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;