first pass of cleanup: removed static classes and cleaned up some bootstrapping code
git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_spring_cleaning@217 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-transactions/pom.xml b/asterix-transactions/pom.xml
index f646356..6e38100 100644
--- a/asterix-transactions/pom.xml
+++ b/asterix-transactions/pom.xml
@@ -25,5 +25,11 @@
</build>
<dependencies>
+ <dependency>
+ <groupId>edu.uci.ics.hyracks</groupId>
+ <artifactId>hyracks-storage-am-btree</artifactId>
+ <version>0.2.1-SNAPSHOT</version>
+ <scope>compile</scope>
+ </dependency>
</dependencies>
</project>
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
index 586f6d6..2bbcc92 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
@@ -32,7 +32,7 @@
*/
public class TransactionalResourceRepository {
- private static Map<ByteBuffer, Object> resourceRepository = new HashMap<ByteBuffer, Object>(); // repository
+ private Map<ByteBuffer, Object> resourceRepository = new HashMap<ByteBuffer, Object>(); // repository
// containing
// resources
// that
@@ -40,13 +40,13 @@
// in
// transactions
- private static Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
+ private Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
// containing
// resource
// managers
- public static void registerTransactionalResource(byte[] resourceBytes, Object resource) {
+ public void registerTransactionalResource(byte[] resourceBytes, Object resource) {
ByteBuffer resourceId = ByteBuffer.wrap(resourceBytes); // need to
// convert to
// ByteBuffer so
@@ -66,7 +66,7 @@
}
}
- public static void registerTransactionalResourceManager(byte id, IResourceManager resourceMgr) {
+ public void registerTransactionalResourceManager(byte id, IResourceManager resourceMgr) {
synchronized (resourceMgrRepository) {
if (resourceMgrRepository.get(id) == null) {
resourceMgrRepository.put(id, resourceMgr);
@@ -79,7 +79,7 @@
}
}
- public static Object getTransactionalResource(byte[] resourceIdBytes) {
+ public Object getTransactionalResource(byte[] resourceIdBytes) {
ByteBuffer buffer = ByteBuffer.wrap(resourceIdBytes);
synchronized (resourceRepository) {
while (resourceRepository.get(buffer) == null) {
@@ -95,17 +95,17 @@
}
}
- public static IResourceManager getTransactionalResourceMgr(byte id) {
+ public IResourceManager getTransactionalResourceMgr(byte id) {
synchronized (resourceMgrRepository) {
- while (resourceMgrRepository.get(id) == null) {
- try {
- resourceMgrRepository.wait();
- } catch (InterruptedException ie) {
- ie.printStackTrace();
- break; // the thread might be interrupted due to other
- // failures occurring elsewhere, break from the loop
- }
- }
+// while (resourceMgrRepository.get(id) == null) {
+// try {
+// resourceMgrRepository.wait();
+// } catch (InterruptedException ie) {
+// ie.printStackTrace();
+// break; // the thread might be interrupted due to other
+// // failures occurring elsewhere, break from the loop
+// }
+// }
return resourceMgrRepository.get(id);
}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
new file mode 100644
index 0000000..3b923dd
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
@@ -0,0 +1,200 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+
+/**
+ * Represents a utility class for generating log records corresponding to
+ * operations on a ITreeIndex implementation. A TreeLogger instance is thread
+ * safe and can be shared across multiple threads that may belong to same or
+ * different transactions.
+ */
+class TransactionState {
+
+ private final Map<Long, TxnThreadState> transactionThreads = new HashMap<Long, TxnThreadState>();
+
+ public synchronized TxnThreadState getTransactionThreadState(long threadId) {
+ return transactionThreads.get(threadId);
+ }
+
+ public synchronized void putTransactionThreadState(long threadId, TxnThreadState txnThreadState) {
+ this.transactionThreads.put(threadId, txnThreadState);
+ }
+
+ public synchronized void remove(long threadId) {
+ transactionThreads.remove(threadId);
+ }
+}
+
+/**
+ * Represents the state of a transaction thread. The state contains information
+ * that includes the tuple being operated, the operation and the location of the
+ * log record corresponding to the operation.
+ */
+class TxnThreadState {
+
+ private ITupleReference tuple;
+ private IndexOp indexOperation;
+ private LogicalLogLocator logicalLogLocator;
+
+ public TxnThreadState(LogicalLogLocator logicalLogLocator, IndexOp indexOperation, ITupleReference tupleReference) {
+ this.tuple = tupleReference;
+ this.indexOperation = indexOperation;
+ this.logicalLogLocator = logicalLogLocator;
+ }
+
+ public synchronized ITupleReference getTuple() {
+ return tuple;
+ }
+
+ public synchronized void setTuple(ITupleReference tuple) {
+ this.tuple = tuple;
+ }
+
+ public synchronized IndexOp getIndexOperation() {
+ return indexOperation;
+ }
+
+ public synchronized void setIndexOperation(IndexOp indexOperation) {
+ this.indexOperation = indexOperation;
+ }
+
+ public synchronized LogicalLogLocator getLogicalLogLocator() {
+ return logicalLogLocator;
+ }
+
+ public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
+ this.logicalLogLocator = logicalLogLocator;
+ }
+
+}
+
+public class TreeLogger implements ILogger, ICloseable {
+
+ private static final byte resourceMgrId = TreeResourceManager.ID;
+ private final Map<Object, Object> arguments = new ConcurrentHashMap<Object, Object>();
+
+ public static final String TREE_INDEX = "TREE_INDEX";
+ public static final String TUPLE_REFERENCE = "TUPLE_REFERENCE";
+ public static final String TUPLE_WRITER = "TUPLE_WRITER";
+ public static final String INDEX_OPERATION = "INDEX_OPERATION";
+ public static final String RESOURCE_ID = "RESOURCE_ID";
+
+ private final ITreeIndex treeIndex;
+ private final ITreeIndexTupleWriter treeIndexTupleWriter;
+ private final byte[] resourceIdBytes;
+ private final byte[] resourceIdLengthBytes;
+
+ public class BTreeOperationCodes {
+ public static final byte INSERT = 0;
+ public static final byte DELETE = 1;
+ }
+
+ public TreeLogger(byte[] resourceIdBytes, ITreeIndex treeIndex) {
+ this.resourceIdBytes = resourceIdBytes;
+ this.treeIndex = treeIndex;
+ treeIndexTupleWriter = treeIndex.getLeafFrameFactory().getTupleWriterFactory().createTupleWriter();
+ this.resourceIdLengthBytes = DataUtil.intToByteArray(resourceIdBytes.length);
+ }
+
+ public synchronized void close(TransactionContext context) {
+ TransactionState txnState = (TransactionState) arguments.get(context.getTransactionID());
+ txnState.remove(Thread.currentThread().getId());
+ arguments.remove(context.getTransactionID());
+ }
+
+ public void generateLogRecord(TransactionProvider provider, TransactionContext context, IndexOp operation,
+ ITupleReference tuple) throws ACIDException {
+ context.addCloseableResource(this); // the close method would be called
+ // on this TreeLogger instance at
+ // the time of transaction
+ // commit/abort.
+ if (operation != IndexOp.INSERT && operation != IndexOp.DELETE) {
+ throw new ACIDException("Loging for Operation " + operation + " not supported");
+
+ }
+
+ TxnThreadState txnThreadState = null;
+ TransactionState txnState;
+ txnState = (TransactionState) arguments.get(context.getTransactionID());
+ if (txnState == null) {
+ synchronized (context) { // threads belonging to different
+ // transaction do not need to
+ // synchronize amongst them.
+ if (txnState == null) {
+ txnState = new TransactionState();
+ arguments.put(context.getTransactionID(), txnState);
+ }
+ }
+ }
+
+ txnThreadState = txnState.getTransactionThreadState(Thread.currentThread().getId());
+ if (txnThreadState == null) {
+ LogicalLogLocator logicalLogLocator = LogUtil.getDummyLogicalLogLocator(provider.getLogManager());
+ txnThreadState = new TxnThreadState(logicalLogLocator, operation, tuple);
+ txnState.putTransactionThreadState(Thread.currentThread().getId(), txnThreadState);
+ }
+ txnThreadState.setIndexOperation(operation);
+ txnThreadState.setTuple(tuple);
+ int tupleSize = treeIndexTupleWriter.bytesRequired(tuple);
+ // Below 4 is for the int representing the length of resource id and 1
+ // is for
+ // the byte representing the operation
+ int logContentLength = 4 + resourceIdBytes.length + 1 + tupleSize;
+ provider.getLogManager().log(txnThreadState.getLogicalLogLocator(), context, resourceMgrId, 0L, LogType.UPDATE,
+ LogActionType.REDO_UNDO, logContentLength, (ILogger) this, arguments);
+ }
+
+ @Override
+ public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logRecordSize,
+ Map<Object, Object> loggerArguments) throws ACIDException {
+ TransactionState txnState = (TransactionState) loggerArguments.get(context.getTransactionID());
+ TxnThreadState state = (TxnThreadState) txnState.getTransactionThreadState(Thread.currentThread().getId());
+ int count = 0;
+ byte[] logBuffer = logicalLogLocator.getBuffer().getArray();
+ System.arraycopy(resourceIdLengthBytes, 0, logBuffer, logicalLogLocator.getMemoryOffset(), 4);
+ count += 4; // count is incremented by 4 because we wrote the length
+ // that is an int and hence 4 bytes
+ System.arraycopy(resourceIdBytes, 0, logBuffer, logicalLogLocator.getMemoryOffset() + count,
+ resourceIdBytes.length);
+ count += resourceIdBytes.length;
+ logBuffer[logicalLogLocator.getMemoryOffset() + count] = (byte) state.getIndexOperation().ordinal();
+ count += 1; // count is incremented by 1 to account for the byte
+ // written.
+ treeIndexTupleWriter.writeTuple(state.getTuple(), logicalLogLocator.getBuffer().getArray(),
+ logicalLogLocator.getMemoryOffset() + count);
+ }
+
+ @Override
+ public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+ }
+
+ @Override
+ public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java
new file mode 100644
index 0000000..34e1466
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+
+public class TreeLoggerRepository {
+
+ private final Map<ByteBuffer, TreeLogger> loggers = new HashMap<ByteBuffer, TreeLogger>();
+ private final TransactionProvider provider;
+
+ public TreeLoggerRepository(TransactionProvider provider) {
+ this.provider = provider;
+ }
+
+ public synchronized TreeLogger getTreeLogger(byte[] resourceIdBytes) {
+ ByteBuffer resourceId = ByteBuffer.wrap(resourceIdBytes);
+ TreeLogger logger = loggers.get(resourceId);
+ if (logger == null) {
+ ITreeIndex treeIndex = (ITreeIndex) provider.getTransactionalResourceRepository().getTransactionalResource(
+ resourceIdBytes);
+ logger = new TreeLogger(resourceIdBytes, treeIndex);
+ loggers.put(resourceId, logger);
+ }
+ return logger;
+ }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
new file mode 100644
index 0000000..7c6d05a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+
+public class TreeResourceManager implements IResourceManager {
+
+ public static final byte ID = (byte) 1;
+
+ private final TransactionProvider provider;
+
+ public TreeResourceManager(TransactionProvider provider) {
+ this.provider = provider;
+ }
+
+ public byte getResourceManagerId() {
+ return ID;
+ }
+
+ public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
+
+ int logContentBeginPos = logRecordHelper.getLogContentBeginPos(logLocator);
+ byte[] logBufferContent = logLocator.getBuffer().getArray();
+ // read the length of resource id byte array
+ int resourceIdLength = DataUtil.byteArrayToInt(logBufferContent, logContentBeginPos);
+ byte[] resourceIdBytes = new byte[resourceIdLength];
+
+ // copy the resource if bytes
+ System.arraycopy(logBufferContent, logContentBeginPos + 4, resourceIdBytes, 0, resourceIdLength);
+
+ // look up the repository to obtain the resource object
+ ITreeIndex treeIndex = (ITreeIndex) provider.getTransactionalResourceRepository().getTransactionalResource(
+ resourceIdBytes);
+ int operationOffset = logContentBeginPos + 4 + resourceIdLength;
+ int tupleBeginPos = operationOffset + 1;
+
+ ITreeIndexTupleReference tupleReference = treeIndex.getLeafFrameFactory().getTupleWriterFactory()
+ .createTupleWriter().createTupleReference();
+ // TODO: remove this call.
+ tupleReference.setFieldCount(tupleReference.getFieldCount());
+ tupleReference.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), tupleBeginPos);
+ byte operation = logBufferContent[operationOffset];
+ IIndexAccessor treeIndexAccessor = treeIndex.createAccessor();
+ try {
+ switch (operation) {
+ case TreeLogger.BTreeOperationCodes.INSERT:
+ treeIndexAccessor.delete(tupleReference);
+ break;
+ case TreeLogger.BTreeOperationCodes.DELETE:
+ treeIndexAccessor.insert(tupleReference);
+ break;
+ }
+ } catch (Exception e) {
+ throw new ACIDException(" could not rollback ", e);
+ }
+ }
+
+ public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException {
+ throw new UnsupportedOperationException(" Redo logic will be implemented as part of crash recovery feature");
+ }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index 085a9da..9c78b95 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -110,7 +110,7 @@
break;
}
byte resourceMgrId = parser.getResourceMgrId(memLSN);
- IResourceManager resourceMgr = TransactionalResourceRepository
+ IResourceManager resourceMgr = transactionProvider.getTransactionalResourceRepository()
.getTransactionalResourceMgr(resourceMgrId);
if (resourceMgr == null) {
throw new ACIDException("unknown resource mgr with id " + resourceMgrId);
@@ -217,7 +217,7 @@
}
// look up the repository to get the resource manager
- IResourceManager resourceMgr = TransactionalResourceRepository
+ IResourceManager resourceMgr = transactionProvider.getTransactionalResourceRepository()
.getTransactionalResourceMgr(resourceMgrId);
if (resourceMgr == null) {
throw new ACIDException(txnContext, " unknown resource manager " + resourceMgrId);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java
index ede93a7..ef843f4 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java
@@ -15,10 +15,12 @@
package edu.uci.ics.asterix.transaction.management.service.transaction;
import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.TreeLoggerRepository;
import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
@@ -32,13 +34,17 @@
private final ILockManager lockManager;
private final ITransactionManager transactionManager;
private final IRecoveryManager recoveryManager;
+ private final TransactionalResourceRepository resourceRepository;
+ private final TreeLoggerRepository loggerRepository;
public TransactionProvider(String id) throws ACIDException {
this.id = id;
- transactionManager = new TransactionManager(this);
- logManager = new LogManager(this);
- lockManager = new LockManager(this);
- recoveryManager = new RecoveryManager(this);
+ this.transactionManager = new TransactionManager(this);
+ this.logManager = new LogManager(this);
+ this.lockManager = new LockManager(this);
+ this.recoveryManager = new RecoveryManager(this);
+ this.loggerRepository = new TreeLoggerRepository(this);
+ this.resourceRepository = new TransactionalResourceRepository();
}
public ILogManager getLogManager() {
@@ -56,6 +62,14 @@
public IRecoveryManager getRecoveryManager() {
return recoveryManager;
}
+
+ public TransactionalResourceRepository getTransactionalResourceRepository() {
+ return resourceRepository;
+ }
+
+ public TreeLoggerRepository getTreeLoggerRepository() {
+ return loggerRepository;
+ }
public String getId() {
return id;
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
index 40ee93f..496260d 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
@@ -51,7 +51,7 @@
provider = new TransactionProvider("nc1");
logManager = provider.getLogManager();
lockManager = provider.getLockManager();
- TransactionalResourceRepository.registerTransactionalResourceManager(DummyResourceMgr.id,
+ provider.getTransactionalResourceRepository().registerTransactionalResourceManager(DummyResourceMgr.id,
new DummyResourceMgr());
Transaction[] transactions = new Transaction[workload.numActiveThreads];
long startTime = System.nanoTime();
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
index fb198fe..ffa5af6 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
@@ -54,8 +54,8 @@
logManager = transactionProvider.getLogManager();
lockManager = transactionProvider.getLockManager();
recoveryManager = transactionProvider.getRecoveryManager();
- TransactionalResourceRepository.registerTransactionalResourceManager(resourceMgr.getResourceManagerId(),
- resourceMgr);
+ transactionProvider.getTransactionalResourceRepository().registerTransactionalResourceManager(
+ resourceMgr.getResourceManagerId(), resourceMgr);
this.resourceMgr = resourceMgr;
this.logger = resource.getLogger();
this.resource = resource;