Deadlock-free locking protocol is enabled

- Added EntityCommitProfiler class in TransactionSubsystem.java file:
This profiler takes a report interval (in seconds) parameter and
reports entity level commit count every report interval (in seconds)
only if IS_PROFILE_MODE is set to true. The profiler runs in a separate
thread. However, the profiler thread doesn't start reporting the count
until the entityCommitCount > 0. The profiler can be used to measure
1) IPS (Inserts Per Second) and
2) IIPS (instantaneous IPS) for the every report interval.

Change-Id: Ie58ae2f519baa53599e99b51bd61ea5f8366dafd
Reviewed-on: https://asterix-gerrit.ics.uci.edu/825
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Murtadha Hubail <hubailmor@gmail.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index a5fa9a6..814c570 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -42,7 +42,6 @@
 import org.apache.asterix.optimizer.rules.IntroduceDynamicTypeCastForExternalFunctionRule;
 import org.apache.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
 import org.apache.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
-import org.apache.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
 import org.apache.asterix.optimizer.rules.IntroduceMaterializationForInsertWithSelfScanRule;
 import org.apache.asterix.optimizer.rules.IntroduceRandomPartitioningFeedComputationRule;
 import org.apache.asterix.optimizer.rules.IntroduceRapidFrameFlushProjectAssignRule;
@@ -291,7 +290,6 @@
         physicalRewritesAllLevels.add(new ReplaceSinkOpWithCommitOpRule());
         physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
-        physicalRewritesAllLevels.add(new IntroduceInstantLockSearchCallbackRule());
         physicalRewritesAllLevels.add(new AddEquivalenceClassForRecordConstructorRule());
         physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
         physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
deleted file mode 100644
index a3d0640..0000000
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.optimizer.rules;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.asterix.algebra.operators.CommitOperator;
-import org.apache.asterix.algebra.operators.physical.BTreeSearchPOperator;
-import org.apache.asterix.metadata.declared.AqlDataSource;
-import org.apache.asterix.metadata.declared.AqlMetadataImplConfig;
-import org.apache.asterix.om.functions.AsterixBuiltinFunctions;
-import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
-import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Triple;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
-import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-
-public class IntroduceInstantLockSearchCallbackRule implements IAlgebraicRewriteRule {
-
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-        return false;
-    }
-
-    private void extractDataSourcesInfo(AbstractLogicalOperator op,
-            Map<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> dataSourcesMap) {
-
-        for (int i = 0; i < op.getInputs().size(); ++i) {
-            AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
-
-            if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
-                UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
-                ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
-                if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
-                    FunctionIdentifier fid = f.getFunctionIdentifier();
-                    if (fid.equals(AsterixBuiltinFunctions.EXTERNAL_LOOKUP)) {
-                        return;
-                    }
-                    if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
-                        throw new IllegalStateException();
-                    }
-                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
-                    jobGenParams.readFromFuncArgs(f.getArguments());
-                    boolean isPrimaryIndex = jobGenParams.isPrimaryIndex();
-                    String indexName = jobGenParams.getIndexName();
-                    if (isPrimaryIndex) {
-                        if (dataSourcesMap.containsKey(indexName)) {
-                            ++(dataSourcesMap.get(indexName).first);
-                        } else {
-                            dataSourcesMap.put(indexName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
-                                    LogicalOperatorTag.UNNEST_MAP, unnestMapOp.getPhysicalOperator()));
-                        }
-                    }
-                }
-            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
-                DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
-                String datasourceName = ((AqlDataSource) dataSourceScanOp.getDataSource()).getId().getDatasourceName();
-                if (dataSourcesMap.containsKey(datasourceName)) {
-                    ++(dataSourcesMap.get(datasourceName).first);
-                } else {
-                    dataSourcesMap.put(datasourceName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
-                            LogicalOperatorTag.DATASOURCESCAN, dataSourceScanOp.getPhysicalOperator()));
-                }
-            }
-            extractDataSourcesInfo(descendantOp, dataSourcesMap);
-        }
-
-    }
-
-    private boolean checkIfRuleIsApplicable(AbstractLogicalOperator op) {
-        if (op.getPhysicalOperator() == null) {
-            return false;
-        }
-        if (op.getOperatorTag() == LogicalOperatorTag.EXTENSION_OPERATOR) {
-            ExtensionOperator extensionOp = (ExtensionOperator) op;
-            if (extensionOp.getDelegate() instanceof CommitOperator) {
-                return true;
-            }
-        }
-        if (op.getOperatorTag() == LogicalOperatorTag.DISTRIBUTE_RESULT
-                || op.getOperatorTag() == LogicalOperatorTag.WRITE_RESULT) {
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
-            throws AlgebricksException {
-
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-
-        if (!checkIfRuleIsApplicable(op)) {
-            return false;
-        }
-        Map<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> dataSourcesMap = new HashMap<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>>();
-        extractDataSourcesInfo(op, dataSourcesMap);
-
-        boolean introducedInstantLock = false;
-
-        Iterator<Map.Entry<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>>> it = dataSourcesMap
-                .entrySet().iterator();
-        while (it.hasNext()) {
-            Entry<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> entry = it.next();
-            Triple<Integer, LogicalOperatorTag, IPhysicalOperator> triple = entry.getValue();
-            if (triple.first == 1) {
-                AqlMetadataImplConfig aqlMetadataImplConfig = new AqlMetadataImplConfig(true);
-                if (triple.second == LogicalOperatorTag.UNNEST_MAP) {
-                    BTreeSearchPOperator pOperator = (BTreeSearchPOperator) triple.third;
-                    pOperator.setImplConfig(aqlMetadataImplConfig);
-                    introducedInstantLock = true;
-                } else {
-                    DataSourceScanPOperator pOperator = (DataSourceScanPOperator) triple.third;
-                    pOperator.setImplConfig(aqlMetadataImplConfig);
-                    introducedInstantLock = true;
-                }
-            }
-
-        }
-        return introducedInstantLock;
-    }
-}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
index 27b19e2..abc03e2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/AsterixTransactionProperties.java
@@ -52,6 +52,9 @@
     private static final String TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_KEY = "txn.lock.timeout.sweepthreshold";
     private static final int TXN_LOCK_TIMEOUT_SWEEPTHRESHOLD_DEFAULT = 10000; // 10s
 
+    private static final String TXN_COMMIT_PROFILER_REPORT_INTERVAL_KEY = "txn.commitprofiler.reportinterval";
+    private static final int TXN_COMMIT_PROFILER_REPORT_INTERVAL_DEFAULT = 5; // 5 seconds
+
     public AsterixTransactionProperties(AsterixPropertiesAccessor accessor) {
         super(accessor);
     }
@@ -114,4 +117,9 @@
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    public int getCommitProfilerReportInterval() {
+        return accessor.getProperty(TXN_COMMIT_PROFILER_REPORT_INTERVAL_KEY,
+                TXN_COMMIT_PROFILER_REPORT_INTERVAL_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index d53236b..5445b11 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -28,6 +28,7 @@
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
+import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.storage.am.common.api.ITupleFilterFactory;
@@ -44,6 +45,20 @@
     private AbstractLSMIndex lsmIndex;
     private int i = 0;
 
+    /**
+     * The following three variables are used to keep track of the information regarding flushing partial frame such as
+     * 1. whether there was a partial frame flush for the current frame,
+     * ==> captured in flushedPartialTuples variabl
+     * 2. the last flushed tuple index in the frame if there was a partial frame flush,
+     * ==> captured in lastFlushedTupleIdx variable
+     * 3. the current tuple index the frame, where this operator is working on the current tuple.
+     * ==> captured in currentTupleIdx variable
+     * These variables are reset for each frame, i.e., whenever nextFrame() is called, these variables are reset.
+     */
+    private boolean flushedPartialTuples;
+    private int currentTupleIdx;
+    private int lastFlushedTupleIdx;
+
     public boolean isPrimary() {
         return isPrimary;
     }
@@ -60,13 +75,14 @@
         RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
         accessor = new FrameTupleAccessor(inputRecDesc);
         writeBuffer = new VSizeFrame(ctx);
+        appender = new FrameTupleAppender(writeBuffer);
         indexHelper.open();
         lsmIndex = (AbstractLSMIndex) indexHelper.getIndexInstance();
         try {
             writer.open();
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
                     indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
-                    lsmIndex, ctx);
+                    lsmIndex, ctx, this);
             indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
             ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
             if (tupleFilterFactory != null) {
@@ -83,11 +99,15 @@
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        currentTupleIdx = 0;
+        lastFlushedTupleIdx = 0;
+        flushedPartialTuples = false;
+
         accessor.reset(buffer);
         ILSMIndexAccessor lsmAccessor = (ILSMIndexAccessor) indexAccessor;
         int tupleCount = accessor.getTupleCount();
         try {
-            for (; i < tupleCount; i++) {
+            for (; i < tupleCount; i++, currentTupleIdx++) {
                 if (tupleFilter != null) {
                     frameTuple.reset(accessor, i);
                     if (!tupleFilter.accept(frameTuple)) {
@@ -120,12 +140,34 @@
             FrameDataException fde = new FrameDataException(i, th);
             throw fde;
         }
+
         writeBuffer.ensureFrameSize(buffer.capacity());
-        FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
-        FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+        if (flushedPartialTuples) {
+            flushPartialFrame();
+        } else {
+            FrameUtils.copyAndFlip(buffer, writeBuffer.getBuffer());
+            FrameUtils.flushFrame(writeBuffer.getBuffer(), writer);
+        }
         i = 0;
     }
 
+    /**
+     * flushes tuples in a frame from lastFlushedTupleIdx(inclusive) to currentTupleIdx(exclusive)
+     */
+    @Override
+    public void flushPartialFrame() throws HyracksDataException {
+        if (lastFlushedTupleIdx == currentTupleIdx) {
+            //nothing to flush
+            return;
+        }
+        for (int i = lastFlushedTupleIdx; i < currentTupleIdx; i++) {
+            FrameUtils.appendToWriter(writer, appender, accessor, i);
+        }
+        appender.write(writer, true);
+        lastFlushedTupleIdx = currentTupleIdx;
+        flushedPartialTuples = true;
+    }
+
     @Override
     public void close() throws HyracksDataException {
         if (lsmIndex != null) {
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java
index 6f54918..d1d869e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/AbstractOperationCallbackFactory.java
@@ -22,7 +22,6 @@
 import java.io.Serializable;
 
 import org.apache.asterix.common.context.ITransactionSubsystemProvider;
-import org.apache.asterix.common.transactions.JobId;
 
 public abstract class AbstractOperationCallbackFactory implements Serializable {
     private static final long serialVersionUID = 1L;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java
index 7909622..495af9a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILockManager.java
@@ -68,7 +68,6 @@
      * @param lockMode
      * @param txnContext
      * @throws ACIDException
-     *             TODO
      * @return
      */
     public void unlock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
@@ -80,7 +79,6 @@
      * @param datasetId
      * @param entityHashValue
      * @param lockMode
-     *            TODO
      * @param context
      * @return
      * @throws ACIDException
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
index 7e27c54..b86aebe 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/ILogRecord.java
@@ -35,6 +35,7 @@
     public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 30;
     public static final int UPDATE_LOG_BASE_SIZE = 59;
     public static final int FLUSH_LOG_SIZE = 18;
+    public static final int WAIT_LOG_SIZE = 14;
 
     public LogRecord.RECORD_STATUS readLogRecord(ByteBuffer buffer);
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
index 003d4c6..2eb8244 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogRecord.java
@@ -65,6 +65,9 @@
  * 3) UPDATE: (Header1(6) + Header2(16) + + Header3(20) + Body(9) + Tail(8)) + PKValueSize + NewValueSize
  *    --> UPDATE_LOG_BASE_SIZE = 59
  * 4) FLUSH: 18 bytes (Header1(6) + DatasetId(4) + Tail(8))
+ * 5) WAIT_LOG_SIZE: 14 bytes (Header1(6) + Tail(8))
+ *    --> WAIT_LOG only requires LogType Field, but in order to conform the log reader protocol
+ *        it also includes LogSource and JobId fields.
  */
 
 public class LogRecord implements ILogRecord {
@@ -265,7 +268,16 @@
         logType = buffer.get();
         jobId = buffer.getInt();
 
-        if (logType != LogType.FLUSH) {
+        if (logType == LogType.FLUSH) {
+            if (buffer.remaining() < DatasetId.BYTES) {
+                return RECORD_STATUS.TRUNCATED;
+            }
+            datasetId = buffer.getInt();
+            resourceId = 0l;
+            computeAndSetLogSize();
+        } else if (logType == LogType.WAIT) {
+            computeAndSetLogSize();
+        } else {
             if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
                 datasetId = -1;
                 PKHashValue = -1;
@@ -306,15 +318,8 @@
             } else {
                 computeAndSetLogSize();
             }
-        } else {
-            computeAndSetLogSize();
-            if (buffer.remaining() < DatasetId.BYTES) {
-                return RECORD_STATUS.TRUNCATED;
-            }
-            datasetId = buffer.getInt();
-            resourceId = 0l;
-            computeAndSetLogSize();
         }
+
         return RECORD_STATUS.OK;
     }
 
@@ -412,6 +417,9 @@
             case LogType.FLUSH:
                 logSize = FLUSH_LOG_SIZE;
                 break;
+            case LogType.WAIT:
+                logSize = WAIT_LOG_SIZE;
+                break;
             default:
                 throw new IllegalStateException("Unsupported Log Type");
         }
@@ -425,7 +433,7 @@
         builder.append(" LogType : ").append(LogType.toString(logType));
         builder.append(" LogSize : ").append(logSize);
         builder.append(" JobId : ").append(jobId);
-        if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
+        if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPSERT_ENTITY_COMMIT || logType == LogType.UPDATE) {
             builder.append(" DatasetId : ").append(datasetId);
             builder.append(" ResourcePartition : ").append(resourcePartition);
             builder.append(" PKHashValue : ").append(PKHashValue);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
index d6d2657..714b8f7 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/LogType.java
@@ -26,6 +26,7 @@
     public static final byte ABORT = 3;
     public static final byte FLUSH = 4;
     public static final byte UPSERT_ENTITY_COMMIT = 5;
+    public static final byte WAIT = 6;
 
     private static final String STRING_UPDATE = "UPDATE";
     private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
@@ -33,6 +34,7 @@
     private static final String STRING_ABORT = "ABORT";
     private static final String STRING_FLUSH = "FLUSH";
     private static final String STRING_UPSERT_ENTITY_COMMIT = "UPSERT_ENTITY_COMMIT";
+    private static final String STRING_WAIT = "WAIT";
 
     private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
 
@@ -50,6 +52,8 @@
                 return STRING_FLUSH;
             case LogType.UPSERT_ENTITY_COMMIT:
                 return STRING_UPSERT_ENTITY_COMMIT;
+            case LogType.WAIT:
+                return STRING_WAIT;
             default:
                 return STRING_INVALID_LOG_TYPE;
         }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
index 5d98961..667bae7 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/indexing/ExternalFileIndexAccessor.java
@@ -92,7 +92,7 @@
 
         // create the accessor  and the cursor using the passed version
         ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
-                .createSearchOperationCallback(indexDataflowHelper.getResourceID(), ctx);
+                .createSearchOperationCallback(indexDataflowHelper.getResourceID(), ctx, null);
         fileIndexAccessor = index.createAccessor(searchCallback, indexDataflowHelper.getVersion());
         fileIndexSearchCursor = fileIndexAccessor.createSearchCursor(false);
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
index 0513f9c..9435387 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalBTreeSearchOperatorNodePushable.java
@@ -73,7 +73,7 @@
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(new VSizeFrame(ctx));
             ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
-                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx, null);
             // The next line is the reason we override this method
             indexAccessor = externalIndex.createAccessor(searchCallback, dataFlowHelper.getTargetVersion());
             cursor = createCursor();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
index 75cc1bf..81e6b17 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalRTreeSearchOperatorNodePushable.java
@@ -72,7 +72,7 @@
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(new VSizeFrame(ctx));
             ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
-                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx, null);
             // The next line is the reason we override this method
             indexAccessor = rTreeIndex.createAccessor(searchCallback, rTreeDataflowHelper.getTargetVersion());
             cursor = createCursor();
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
index 542c12d..54ec084 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/MetadataNode.java
@@ -79,7 +79,6 @@
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallback;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallback;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
@@ -306,15 +305,13 @@
             IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws ACIDException {
         ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
 
-        if (metadataIndex.isPrimaryIndex()) {
-            return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
-                    metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
-                    transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
-        } else {
-            return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
-                    metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
-                    transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
-        }
+        // Regardless of the index type (primary or secondary index), secondary index modification callback is given
+        // This is still correct since metadata index operation doesn't require any lock from ConcurrentLockMgr and
+        // The difference between primaryIndexModCallback and secondaryIndexModCallback is that primary index requires
+        // locks and secondary index doesn't.
+        return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+                metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+                transactionSubsystem, resourceId, metadataStoragePartition, ResourceType.LSM_BTREE, indexOp);
     }
 
     @Override
@@ -980,10 +977,8 @@
             try {
                 while (rangeCursor.hasNext()) {
                     rangeCursor.next();
-                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
-                            new ISerializerDeserializer[] {
-                                    AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(
-                                            BuiltinType.ASTRING),
+                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+                            AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
                             AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
                             AqlSerializerDeserializerProvider.INSTANCE
                                     .getSerializerDeserializer(BuiltinType.ASTRING) }));
@@ -1000,7 +995,7 @@
 
     private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
             IValueExtractor<ResultType> valueExtractor, List<ResultType> results)
-                    throws MetadataException, IndexException, IOException {
+            throws MetadataException, IndexException, IOException {
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
         String resourceName = index.getFile().toString();
         IIndex indexInstance = datasetLifecycleManager.getIndex(resourceName);
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
index 7070a88..ec2e692 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/api/IMetadataNode.java
@@ -148,8 +148,8 @@
      * @throws MetadataException
      *             For example, if the dataverse does not exist. RemoteException
      */
-    public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
-            RemoteException;
+    public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName)
+            throws MetadataException, RemoteException;
 
     /**
      * Deletes the dataverse with given name, and all it's associated datasets,
@@ -194,8 +194,8 @@
      *             For example, if the dataset does not exist.
      * @throws RemoteException
      */
-    public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
-            RemoteException;
+    public Dataset getDataset(JobId jobId, String dataverseName, String datasetName)
+            throws MetadataException, RemoteException;
 
     /**
      * Retrieves all indexes of a dataset, acquiring local locks on behalf of
@@ -229,8 +229,8 @@
      *             For example, if the dataset and/or dataverse does not exist.
      * @throws RemoteException
      */
-    public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
-            RemoteException;
+    public void dropDataset(JobId jobId, String dataverseName, String datasetName)
+            throws MetadataException, RemoteException;
 
     /**
      * Inserts an index into the metadata, acquiring local locks on behalf of
@@ -313,8 +313,8 @@
      *             For example, if the datatype does not exist.
      * @throws RemoteException
      */
-    public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
-            RemoteException;
+    public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName)
+            throws MetadataException, RemoteException;
 
     /**
      * Deletes the given datatype in given dataverse, acquiring local locks on
@@ -331,8 +331,8 @@
      *             deleted.
      * @throws RemoteException
      */
-    public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
-            RemoteException;
+    public void dropDatatype(JobId jobId, String dataverseName, String datatypeName)
+            throws MetadataException, RemoteException;
 
     /**
      * Inserts a node group, acquiring local locks on behalf of the given
@@ -400,8 +400,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
-            RemoteException;
+    public Function getFunction(JobId jobId, FunctionSignature functionSignature)
+            throws MetadataException, RemoteException;
 
     /**
      * Deletes a function, acquiring local locks on behalf of the given
@@ -416,8 +416,8 @@
      *             group to be deleted.
      * @throws RemoteException
      */
-    public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
-            RemoteException;
+    public void dropFunction(JobId jobId, FunctionSignature functionSignature)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -438,8 +438,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
-            RemoteException;
+    public List<Function> getDataverseFunctions(JobId jobId, String dataverseName)
+            throws MetadataException, RemoteException;
 
     /**
      * @param ctx
@@ -448,8 +448,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
-            RemoteException;
+    public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -475,8 +475,8 @@
      *            if the adapter does not exists.
      * @throws RemoteException
      */
-    public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
-            RemoteException;
+    public void dropAdapter(JobId jobId, String dataverseName, String adapterName)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -495,8 +495,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy) throws MetadataException,
-            RemoteException;
+    public void addCompactionPolicy(JobId jobId, CompactionPolicy compactionPolicy)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -506,8 +506,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
-            RemoteException;
+    public CompactionPolicy getCompactionPolicy(JobId jobId, String dataverse, String policy)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -550,7 +550,6 @@
      */
     public void dropFeed(JobId jobId, String dataverse, String feedName) throws MetadataException, RemoteException;
 
-
     /**
      * @param jobId
      * @param feedPolicy
@@ -567,9 +566,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy) throws MetadataException,
-            RemoteException;
-
+    public FeedPolicyEntity getFeedPolicy(JobId jobId, String dataverse, String policy)
+            throws MetadataException, RemoteException;
 
     /**
      * Removes a library , acquiring local locks on behalf of the given
@@ -584,8 +582,8 @@
      *            if the library does not exists.
      * @throws RemoteException
      */
-    public void dropLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
-            RemoteException;
+    public void dropLibrary(JobId jobId, String dataverseName, String libraryName)
+            throws MetadataException, RemoteException;
 
     /**
      * Adds a library, acquiring local locks on behalf of the given
@@ -612,8 +610,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public Library getLibrary(JobId jobId, String dataverseName, String libraryName) throws MetadataException,
-            RemoteException;
+    public Library getLibrary(JobId jobId, String dataverseName, String libraryName)
+            throws MetadataException, RemoteException;
 
     /**
      * Retireve libraries installed in a given dataverse.
@@ -626,8 +624,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public List<Library> getDataverseLibraries(JobId jobId, String dataverseName) throws MetadataException,
-            RemoteException;
+    public List<Library> getDataverseLibraries(JobId jobId, String dataverseName)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -648,8 +646,8 @@
      * @throws RemoteException
      * @throws MetadataException
      */
-    public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName) throws MetadataException,
-            RemoteException;
+    public void dropFeedPolicy(JobId jobId, String dataverseName, String policyName)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -658,8 +656,8 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse) throws MetadataException,
-            RemoteException;
+    public List<FeedPolicyEntity> getDataversePolicies(JobId jobId, String dataverse)
+            throws MetadataException, RemoteException;
 
     /**
      * @param jobId
@@ -731,7 +729,6 @@
     public ExternalFile getExternalFile(JobId jobId, String dataverseName, String datasetName, Integer fileNumber)
             throws MetadataException, RemoteException;
 
-
     /**
      * update an existing dataset in the metadata, acquiring local locks on behalf
      * of the given transaction id.
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
index 9fe72a4..5e2e227 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/AqlMetadataProvider.java
@@ -110,7 +110,6 @@
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
-import org.apache.asterix.transaction.management.opcallbacks.PrimaryIndexSearchOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import org.apache.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
@@ -730,17 +729,15 @@
                     primaryKeyFields[i] = i;
                 }
 
-                AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
                 ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
-                if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
-                    searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                            : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
-                                    txnSubsystemProvider, ResourceType.LSM_BTREE);
-                } else {
-                    searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
-                            : new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
-                                    txnSubsystemProvider, ResourceType.LSM_BTREE);
-                }
+
+                /**
+                 * Due to the read-committed isolation level,
+                 * we may acquire very short duration lock(i.e., instant lock) for readers.
+                 */
+                searchCallbackFactory = temp ? NoOpOperationCallbackFactory.INSTANCE
+                        : new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId, primaryKeyFields,
+                                txnSubsystemProvider, ResourceType.LSM_BTREE);
             }
             Pair<ILSMMergePolicyFactory, Map<String, String>> compactionInfo = DatasetUtils
                     .getMergePolicyFactory(dataset, mdTxnCtx);
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
index 83c6e34..7785978 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/AsterixLSMPrimaryUpsertOperatorNodePushable.java
@@ -128,10 +128,10 @@
             appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
                     indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
-                    index, ctx);
+                    index, ctx, this);
 
             indexAccessor = index.createAccessor(modCallback, opDesc.getSearchOpCallbackFactory()
-                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx));
+                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx, this));
             cursor = indexAccessor.createSearchCursor(false);
             frameTuple = new FrameTupleReference();
             IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
@@ -191,6 +191,7 @@
     //TODO: use tryDelete/tryInsert in order to prevent deadlocks
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+
         accessor.reset(buffer);
         LSMTreeIndexAccessor lsmAccessor = (LSMTreeIndexAccessor) indexAccessor;
         int tupleCount = accessor.getTupleCount();
@@ -239,6 +240,15 @@
         }
     }
 
+    /**
+     * Flushes tuples (which have already been written to tuple appender's buffer in writeOutput() method)
+     * to the next operator/consumer.
+     */
+    @Override
+    public void flushPartialFrame() throws HyracksDataException {
+        appender.write(writer, true);
+    }
+
     private ITupleReference getPrevTupleWithFilter(ITupleReference prevTuple) throws IOException, AsterixException {
         prevRecWithPKWithFilterValue.reset();
         for (int i = 0; i < prevTuple.getFieldCount(); i++) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java
index 49cea94..ef3b218 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallback.java
@@ -20,18 +20,40 @@
 
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.AbstractOperationCallback;
-import org.apache.asterix.common.transactions.ILockManager;
+import org.apache.asterix.common.transactions.ILogManager;
+import org.apache.asterix.common.transactions.ILogRecord;
 import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.ITransactionSubsystem;
+import org.apache.asterix.common.transactions.LogRecord;
+import org.apache.asterix.common.transactions.LogSource;
+import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
+import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
 
 public class LockThenSearchOperationCallback extends AbstractOperationCallback implements ISearchOperationCallback {
 
-    public LockThenSearchOperationCallback(int datasetId, int[] entityIdFields, ILockManager lockManager,
-            ITransactionContext txnCtx) {
-        super(datasetId, entityIdFields, txnCtx, lockManager);
+    /**
+     * variables used for deadlock-free locking protocol
+     */
+    private final LSMIndexInsertUpdateDeleteOperatorNodePushable operatorNodePushable;
+    private final ILogManager logManager;
+    private final ILogRecord logRecord;
+
+    public LockThenSearchOperationCallback(int datasetId, int[] entityIdFields, ITransactionSubsystem txnSubsystem,
+            ITransactionContext txnCtx, IOperatorNodePushable operatorNodePushable) {
+        super(datasetId, entityIdFields, txnCtx, txnSubsystem.getLockManager());
+        this.operatorNodePushable = (LSMIndexInsertUpdateDeleteOperatorNodePushable) operatorNodePushable;
+        this.logManager = txnSubsystem.getLogManager();
+        this.logRecord = new LogRecord();
+        logRecord.setTxnCtx(txnCtx);
+        logRecord.setLogSource(LogSource.LOCAL);
+        logRecord.setLogType(LogType.WAIT);
+        logRecord.setJobId(txnCtx.getJobId().getId());
+        logRecord.computeAndSetLogSize();
     }
 
     @Override
@@ -55,9 +77,49 @@
     public void before(ITupleReference tuple) throws HyracksDataException {
         int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
         try {
-            lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+            if (operatorNodePushable != null) {
+
+                /**********************************************************************************
+                 * In order to achieve deadlock-free locking protocol during any write (insert/delete/upsert) operations,
+                 * the following logic is implemented.
+                 * See https://cwiki.apache.org/confluence/display/ASTERIXDB/Deadlock-Free+Locking+Protocol for more details.
+                 * 1. for each entry in a frame
+                 * 2. returnValue = tryLock() for an entry
+                 * 3. if returnValue == false
+                 * 3-1. flush all entries (which already acquired locks) to the next operator
+                 * : this will make all those entries reach commit operator so that corresponding commit logs will be created.
+                 * 3-2. create a WAIT log and wait until logFlusher thread will flush the WAIT log and gives notification
+                 * : this notification guarantees that all locks acquired by this transactor (or all locks acquired for the entries)
+                 * were released.
+                 * 3-3. acquire lock using lock() instead of tryLock() for the failed entry
+                 * : we know for sure this lock call will not cause deadlock since the transactor doesn't hold any other locks.
+                 * 4. create an update log and insert the entry
+                 * From the above logic, step 2 and 3 are implemented in this before() method.
+                 **********************/
+
+                //release all locks held by this actor (which is a thread) by flushing partial frame.
+                boolean tryLockSucceed = lockManager.tryLock(datasetId, pkHash, LockMode.X, txnCtx);
+                if (!tryLockSucceed) {
+                    //flush entries which have been inserted already to release locks hold by them
+                    operatorNodePushable.flushPartialFrame();
+
+                    //create WAIT log and wait until the WAIT log is flushed and notified by LogFlusher thread
+                    logWait();
+
+                    //acquire lock
+                    lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+                }
+
+            } else {
+                //operatorNodePushable can be null when metadata node operation is executed
+                lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+            }
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
     }
+
+    private void logWait() throws ACIDException {
+        logManager.log(logRecord);
+    }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java
index 6bfb6cd..0d65c16 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/LockThenSearchOperationCallbackFactory.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -40,13 +41,13 @@
     }
 
     @Override
-    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
-            throws HyracksDataException {
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx,
+            IOperatorNodePushable operatorNodePushable) throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
-            return new LockThenSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
-                    txnCtx);
+            return new LockThenSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem, txnCtx,
+                    operatorNodePushable);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
index 9b14807..ba97287 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -41,7 +42,7 @@
     }
 
     @Override
-    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
             throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 780f294..4bde490 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -19,11 +19,14 @@
 
 package org.apache.asterix.transaction.management.opcallbacks;
 
+import org.apache.asterix.common.dataflow.AsterixLSMInsertDeleteOperatorNodePushable;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.ILockManager;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
+import org.apache.asterix.common.transactions.LogType;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -36,18 +39,57 @@
 public class PrimaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback
         implements IModificationOperationCallback {
 
+    private final AsterixLSMInsertDeleteOperatorNodePushable operatorNodePushable;
+
     public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
             ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, int resourcePartition,
-            byte resourceType, IndexOperation indexOp) {
+            byte resourceType, IndexOperation indexOp, IOperatorNodePushable operatorNodePushable) {
         super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourcePartition,
                 resourceType, indexOp);
+        this.operatorNodePushable = (AsterixLSMInsertDeleteOperatorNodePushable) operatorNodePushable;
     }
 
     @Override
     public void before(ITupleReference tuple) throws HyracksDataException {
         int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
         try {
-            lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+            if (operatorNodePushable != null) {
+
+                /**********************************************************************************
+                 * In order to achieve deadlock-free locking protocol during any write (insert/delete/upsert) operations,
+                 * the following logic is implemented.
+                 * See https://cwiki.apache.org/confluence/display/ASTERIXDB/Deadlock-Free+Locking+Protocol for more details.
+                 * 1. for each entry in a frame
+                 * 2. returnValue = tryLock() for an entry
+                 * 3. if returnValue == false
+                 * 3-1. flush all entries (which already acquired locks) to the next operator
+                 * : this will make all those entries reach commit operator so that corresponding commit logs will be created.
+                 * 3-2. create a WAIT log and wait until logFlusher thread will flush the WAIT log and gives notification
+                 * : this notification guarantees that all locks acquired by this transactor (or all locks acquired for the entries)
+                 * were released.
+                 * 3-3. acquire lock using lock() instead of tryLock() for the failed entry
+                 * : we know for sure this lock call will not cause deadlock since the transactor doesn't hold any other locks.
+                 * 4. create an update log and insert the entry
+                 * From the above logic, step 2 and 3 are implemented in this before() method.
+                 **********************/
+
+                //release all locks held by this actor (which is a thread) by flushing partial frame.
+                boolean tryLockSucceed = lockManager.tryLock(datasetId, pkHash, LockMode.X, txnCtx);
+                if (!tryLockSucceed) {
+                    //flush entries which have been inserted already to release locks hold by them
+                    operatorNodePushable.flushPartialFrame();
+
+                    //create WAIT log and wait until the WAIT log is flushed and notified by LogFlusher thread
+                    logWait();
+
+                    //acquire lock
+                    lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+                }
+
+            } else {
+                //operatorNodePushable can be null when metadata node operation is executed
+                lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+            }
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
@@ -62,4 +104,10 @@
             throw new HyracksDataException(e);
         }
     }
+
+    private void logWait() throws ACIDException {
+        logRecord.setLogType(LogType.WAIT);
+        logRecord.computeAndSetLogSize();
+        txnSubsystem.getLogManager().log(logRecord);
+    }
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index db68b26..c406812 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -51,7 +52,8 @@
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+            throws HyracksDataException {
 
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
@@ -65,7 +67,7 @@
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
                     primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId,
-                    resourcePartition, resourceType, indexOp);
+                    resourcePartition, resourceType, indexOp, operatorNodePushable);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
             return modCallback;
         } catch (ACIDException e) {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
index b483674..1dd8368 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -41,7 +42,7 @@
     }
 
     @Override
-    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
             throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index ef2b498..168da99 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -48,7 +49,8 @@
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+            throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getDatasetLifecycleManager();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
index 4e1ee63..5dfdcdc 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
@@ -20,6 +20,7 @@
 package org.apache.asterix.transaction.management.opcallbacks;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallback;
 import org.apache.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
@@ -29,7 +30,7 @@
     private static final long serialVersionUID = 1L;
 
     @Override
-    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
             throws HyracksDataException {
         return new SecondaryIndexSearchOperationCallback();
     }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
index b08798c..8c91c1a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetPrimaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -49,7 +50,8 @@
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+            throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getDatasetLifecycleManager();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
index 403d68d..3e11531 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/TempDatasetSecondaryIndexModificationOperationCallbackFactory.java
@@ -27,6 +27,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -49,7 +50,8 @@
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+            throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getDatasetLifecycleManager();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
index 707f986..87cb8e7 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/opcallbacks/UpsertOperationCallbackFactory.java
@@ -26,6 +26,7 @@
 import org.apache.asterix.common.transactions.ITransactionSubsystem;
 import org.apache.asterix.common.transactions.JobId;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -47,7 +48,8 @@
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourceName, long resourceId,
-            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+            throws HyracksDataException {
 
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
index e268134..8dcc1be 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ConcurrentLockManager.java
@@ -22,7 +22,6 @@
 import java.io.IOException;
 import java.io.OutputStream;
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.logging.Level;
@@ -46,6 +45,10 @@
 
     static final Logger LOGGER = Logger.getLogger(ConcurrentLockManager.class.getName());
     static final Level LVL = Level.FINER;
+    public static final boolean ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL = true;
+
+    public static final int NIL = -1;
+    public static final long NILL = -1L;
 
     public static final boolean DEBUG_MODE = false;//true
     public static final boolean CHECK_CONSISTENCY = false;
@@ -54,8 +57,7 @@
     private ResourceArenaManager resArenaMgr;
     private RequestArenaManager reqArenaMgr;
     private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Integer, Long> jobIdSlotMap;
-    private ThreadLocal<DatasetLockCache> dsLockCache;
+    private ConcurrentHashMap<Integer, Long> jobId2JobSlotMap;
     private LockManagerStats stats = new LockManagerStats(10000);
 
     enum LockAction {
@@ -94,13 +96,7 @@
         resArenaMgr = new ResourceArenaManager(noArenas, lockManagerShrinkTimer);
         reqArenaMgr = new RequestArenaManager(noArenas, lockManagerShrinkTimer);
         jobArenaMgr = new JobArenaManager(noArenas, lockManagerShrinkTimer);
-        jobIdSlotMap = new ConcurrentHashMap<>();
-        dsLockCache = new ThreadLocal<DatasetLockCache>() {
-            @Override
-            protected DatasetLockCache initialValue() {
-                return new DatasetLockCache();
-            }
-        };
+        jobId2JobSlotMap = new ConcurrentHashMap<Integer, Long>();
     }
 
     @Override
@@ -111,17 +107,7 @@
 
         final int dsId = datasetId.getId();
         final int jobId = txnContext.getJobId().getId();
-
-        if (entityHashValue != -1) {
-            lock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext);
-        } else {
-            if (dsLockCache.get().contains(jobId, dsId, lockMode)) {
-                return;
-            }
-        }
-
         final long jobSlot = findOrAllocJobSlot(jobId);
-
         final ResourceGroup group = table.get(dsId, entityHashValue);
         group.getLatch();
         try {
@@ -160,9 +146,6 @@
                         throw new IllegalStateException();
                 }
             }
-            if (entityHashValue == -1) {
-                dsLockCache.get().put(jobId, dsId, lockMode);
-            }
         } catch (InterruptedException e) {
             throw new WaitInterruptedException(txnContext, "interrupted", e);
         } finally {
@@ -282,7 +265,15 @@
      * @return true if a cycle would be introduced, false otherwise
      */
     private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker) {
-        return introducesDeadlock(resSlot, jobSlot, tracker, 0);
+        if (ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL) {
+            /**
+             * Due to the deadlock-free locking protocol, deadlock is not possible.
+             * So, this method always returns false.
+             */
+            return false;
+        } else {
+            return introducesDeadlock(resSlot, jobSlot, tracker, 0);
+        }
     }
 
     private boolean introducesDeadlock(final long resSlot, final long jobSlot, final DeadlockTracker tracker,
@@ -336,15 +327,8 @@
 
         final int dsId = datasetId.getId();
         final int jobId = txnContext.getJobId().getId();
-
-        if (entityHashValue != -1) {
-            lock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext);
-        } else {
-            throw new UnsupportedOperationException("instant locks are not supported on datasets");
-        }
-
         final ResourceGroup group = table.get(dsId, entityHashValue);
-        if (group.firstResourceIndex.get() == -1l) {
+        if (group.firstResourceIndex.get() == NILL) {
             validateJob(txnContext);
             // if we do not have a resource in the group, we know that the
             // resource that we are looking for is not locked
@@ -352,7 +336,7 @@
         }
 
         // we only allocate a request slot if we actually have to wait
-        long reqSlot = -1;
+        long reqSlot = NILL;
 
         group.getLatch();
         try {
@@ -374,7 +358,7 @@
                         return;
                     case WAIT:
                     case CONV:
-                        if (reqSlot == -1) {
+                        if (reqSlot == NILL) {
                             reqSlot = allocRequestSlot(resSlot, jobSlot, lockMode);
                         }
                         enqueueWaiter(group, reqSlot, resSlot, jobSlot, act, txnContext);
@@ -387,7 +371,7 @@
         } catch (InterruptedException e) {
             throw new WaitInterruptedException(txnContext, "interrupted", e);
         } finally {
-            if (reqSlot != -1) {
+            if (reqSlot != NILL) {
                 // deallocate request, if we allocated one earlier
                 if (DEBUG_MODE)
                     LOGGER.finer("del req slot " + TypeUtil.Global.toString(reqSlot));
@@ -405,19 +389,7 @@
 
         final int dsId = datasetId.getId();
         final int jobId = txnContext.getJobId().getId();
-
-        if (entityHashValue != -1) {
-            if (!tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
-                return false;
-            }
-        } else {
-            if (dsLockCache.get().contains(jobId, dsId, lockMode)) {
-                return true;
-            }
-        }
-
         final long jobSlot = findOrAllocJobSlot(jobId);
-
         final ResourceGroup group = table.get(dsId, entityHashValue);
         group.getLatch();
 
@@ -434,9 +406,6 @@
                     // no break
                 case GET:
                     addHolder(reqSlot, resSlot, jobSlot);
-                    if (entityHashValue == -1) {
-                        dsLockCache.get().put(jobId, dsId, lockMode);
-                    }
                     return true;
                 case WAIT:
                 case CONV:
@@ -447,9 +416,6 @@
         } finally {
             group.releaseLatch();
         }
-
-        // if we did acquire the dataset lock, but not the entity lock, we keep
-        // it anyway and clean it up at the end of the job
     }
 
     @Override
@@ -460,17 +426,8 @@
 
         final int dsId = datasetId.getId();
         final int jobId = txnContext.getJobId().getId();
-
-        if (entityHashValue != -1) {
-            if (!tryLock(datasetId, -1, LockMode.intentionMode(lockMode), txnContext)) {
-                return false;
-            }
-        } else {
-            throw new UnsupportedOperationException("instant locks are not supported on datasets");
-        }
-
         final ResourceGroup group = table.get(dsId, entityHashValue);
-        if (group.firstResourceIndex.get() == -1l) {
+        if (group.firstResourceIndex.get() == NILL) {
             validateJob(txnContext);
             // if we do not have a resource in the group, we know that the
             // resource that we are looking for is not locked
@@ -511,7 +468,8 @@
             throws ACIDException {
         log("unlock", datasetId.getId(), entityHashValue, lockMode, txnContext);
         final int jobId = txnContext.getJobId().getId();
-        final long jobSlot = jobIdSlotMap.get(jobId);
+        final long jobSlot = jobId2JobSlotMap.get(jobId);
+
         final int dsId = datasetId.getId();
         unlock(dsId, entityHashValue, lockMode, jobSlot);
     }
@@ -561,19 +519,15 @@
         } finally {
             group.releaseLatch();
         }
-
-        // dataset intention locks are
-        // a) kept in dsLockCache and
-        // b) cleaned up only in releaseLocks at the end of the job
     }
 
     @Override
     public void releaseLocks(ITransactionContext txnContext) throws ACIDException {
-        log("releaseLocks", -1, -1, LockMode.ANY, txnContext);
+        log("releaseLocks", NIL, NIL, LockMode.ANY, txnContext);
         stats.releaseLocks();
 
         int jobId = txnContext.getJobId().getId();
-        Long jobSlot = jobIdSlotMap.get(jobId);
+        Long jobSlot = jobId2JobSlotMap.get(jobId);
         if (jobSlot == null) {
             // we don't know the job, so there are no locks for it - we're done
             return;
@@ -588,7 +542,7 @@
         synchronized (jobArenaMgr) {
             holder = jobArenaMgr.getLastHolder(jobSlot);
         }
-        while (holder != -1) {
+        while (holder != NILL) {
             long resource = reqArenaMgr.getResourceId(holder);
             int dsId = resArenaMgr.getDatasetId(resource);
             int pkHashVal = resArenaMgr.getPkHashVal(resource);
@@ -600,20 +554,20 @@
         if (DEBUG_MODE)
             LOGGER.finer("del job slot " + TypeUtil.Global.toString(jobSlot));
         jobArenaMgr.deallocate(jobSlot);
-        jobIdSlotMap.remove(jobId);
+        jobId2JobSlotMap.remove(jobId);
         stats.logCounters(LOGGER, Level.INFO, true);
     }
 
     private long findOrAllocJobSlot(int jobId) {
-        Long jobSlot = jobIdSlotMap.get(jobId);
+        Long jobSlot = jobId2JobSlotMap.get(jobId);
         if (jobSlot == null) {
             jobSlot = new Long(jobArenaMgr.allocate());
             if (DEBUG_MODE)
                 LOGGER.finer("new job slot " + TypeUtil.Global.toString(jobSlot) + " (" + jobId + ")");
             jobArenaMgr.setJobId(jobSlot, jobId);
-            Long oldSlot = jobIdSlotMap.putIfAbsent(jobId, jobSlot);
+            Long oldSlot = jobId2JobSlotMap.putIfAbsent(jobId, jobSlot);
             if (oldSlot != null) {
-                // if another thread allocated a slot for this jobId between
+                // if another thread allocated a slot for this jobThreadId between
                 // get(..) and putIfAbsent(..), we'll use that slot and
                 // deallocate the one we allocated
                 if (DEBUG_MODE)
@@ -629,7 +583,7 @@
     private long findOrAllocResourceSlot(ResourceGroup group, int dsId, int entityHashValue) {
         long resSlot = findResourceInGroup(group, dsId, entityHashValue);
 
-        if (resSlot == -1) {
+        if (resSlot == NILL) {
             // we don't know about this resource, let's alloc a slot
             resSlot = resArenaMgr.allocate();
             resArenaMgr.setDatasetId(resSlot, dsId);
@@ -690,11 +644,15 @@
         // carefully distinguishing the different lock modes
         long holder = resArenaMgr.getLastHolder(resource);
         LockAction res = LockAction.WAIT;
-        while (holder != -1) {
+        while (holder != NILL) {
             if (job == reqArenaMgr.getJobSlot(holder)) {
                 if (reqArenaMgr.getLockMode(holder) == lockMode) {
                     return LockAction.GET;
                 } else {
+                    if (ENABLED_DEADLOCK_FREE_LOCKING_PROTOCOL) {
+                        throw new IllegalStateException(
+                                "Lock conversion is not supported when deadlock-free locking protocol is enabled!");
+                    }
                     res = LockAction.CONV;
                 }
             }
@@ -706,7 +664,7 @@
     private long findResourceInGroup(ResourceGroup group, int dsId, int entityHashValue) {
         stats.logCounters(LOGGER, Level.INFO, false);
         long resSlot = group.firstResourceIndex.get();
-        while (resSlot != -1) {
+        while (resSlot != NILL) {
             // either we already have a lock on this resource or we have a
             // hash collision
             if (resArenaMgr.getDatasetId(resSlot) == dsId && resArenaMgr.getPkHashVal(resSlot) == entityHashValue) {
@@ -715,7 +673,7 @@
                 resSlot = resArenaMgr.getNext(resSlot);
             }
         }
-        return -1;
+        return NILL;
     }
 
     private void addHolder(long request, long resource, long job) {
@@ -732,7 +690,7 @@
 
     private boolean hasOtherHolders(long resSlot, long jobSlot) {
         long holder = resArenaMgr.getLastHolder(resSlot);
-        while (holder != -1) {
+        while (holder != NILL) {
             if (reqArenaMgr.getJobSlot(holder) != jobSlot) {
                 return true;
             }
@@ -771,10 +729,10 @@
     private long removeRequestFromJob(long holder, long unmodified) {
         long prevForJob = reqArenaMgr.getPrevJobRequest(holder);
         long nextForJob = reqArenaMgr.getNextJobRequest(holder);
-        if (nextForJob != -1) {
+        if (nextForJob != NILL) {
             reqArenaMgr.setPrevJobRequest(nextForJob, prevForJob);
         }
-        if (prevForJob == -1) {
+        if (prevForJob == NILL) {
             return nextForJob;
         } else {
             reqArenaMgr.setNextJobRequest(prevForJob, nextForJob);
@@ -792,8 +750,8 @@
         @Override
         public void add(long request, long resource, long job) {
             long waiter = resArenaMgr.getFirstWaiter(resource);
-            reqArenaMgr.setNextRequest(request, -1);
-            if (waiter == -1) {
+            reqArenaMgr.setNextRequest(request, NILL);
+            if (waiter == NILL) {
                 resArenaMgr.setFirstWaiter(resource, request);
             } else {
                 appendToRequestQueue(waiter, request);
@@ -858,7 +816,7 @@
 
     private void insertIntoJobQueue(long newRequest, long oldRequest) {
         reqArenaMgr.setNextJobRequest(newRequest, oldRequest);
-        reqArenaMgr.setPrevJobRequest(newRequest, -1);
+        reqArenaMgr.setPrevJobRequest(newRequest, NILL);
         if (oldRequest >= 0) {
             reqArenaMgr.setPrevJobRequest(oldRequest, newRequest);
         }
@@ -866,7 +824,7 @@
 
     private void appendToRequestQueue(long head, long appendee) {
         long next = reqArenaMgr.getNextRequest(head);
-        while (next != -1) {
+        while (next != NILL) {
             head = next;
             next = reqArenaMgr.getNextRequest(head);
         }
@@ -876,9 +834,9 @@
     private long removeRequestFromQueueForSlot(long head, long reqSlot) {
         long cur = head;
         long prev = cur;
-        while (prev != -1) {
+        while (prev != NILL) {
             cur = reqArenaMgr.getNextRequest(prev);
-            if (cur == -1) {
+            if (cur == NILL) {
                 throw new IllegalStateException("request " + reqSlot + " not in queue");
             }
             if (cur == reqSlot) {
@@ -907,9 +865,9 @@
     private long removeRequestFromQueueForJob(long head, long jobSlot, byte lockMode) {
         long holder = head;
         long prev = holder;
-        while (prev != -1) {
+        while (prev != NILL) {
             holder = reqArenaMgr.getNextRequest(prev);
-            if (holder == -1) {
+            if (holder == NILL) {
                 throw new IllegalStateException("no entry for job " + jobSlot + " in queue");
             }
             if (requestMatches(holder, jobSlot, lockMode)) {
@@ -925,7 +883,7 @@
     private int determineNewMaxMode(long resource, int oldMaxMode) {
         int newMaxMode = LockMode.NL;
         long holder = resArenaMgr.getLastHolder(resource);
-        while (holder != -1) {
+        while (holder != NILL) {
             int curLockMode = reqArenaMgr.getLockMode(holder);
             if (curLockMode == oldMaxMode) {
                 // we have another lock of the same mode - we're done
@@ -976,10 +934,10 @@
         }
         StringBuilder sb = new StringBuilder();
         sb.append("{ op : ").append(string);
-        if (id != -1) {
+        if (id != NIL) {
             sb.append(" , dataset : ").append(id);
         }
-        if (entityHashValue != -1) {
+        if (entityHashValue != NIL) {
             sb.append(" , entity : ").append(entityHashValue);
         }
         if (lockMode != LockMode.NL) {
@@ -1000,11 +958,11 @@
                 if (group.tryLatch(100, TimeUnit.MILLISECONDS)) {
                     try {
                         long resSlot = group.firstResourceIndex.get();
-                        while (resSlot != -1) {
+                        while (resSlot != NILL) {
                             int dsId = resArenaMgr.getDatasetId(resSlot);
                             int entityHashValue = resArenaMgr.getPkHashVal(resSlot);
                             long reqSlot = resArenaMgr.getLastHolder(resSlot);
-                            while (reqSlot != -1) {
+                            while (reqSlot != NILL) {
                                 byte lockMode = (byte) reqArenaMgr.getLockMode(reqSlot);
                                 long jobSlot = reqArenaMgr.getJobSlot(reqSlot);
                                 int jobId = jobArenaMgr.getJobId(jobSlot);
@@ -1026,7 +984,7 @@
     }
 
     private void assertLockCanBeFoundInJobQueue(int dsId, int entityHashValue, byte lockMode, int jobId) {
-        if (findLockInJobQueue(dsId, entityHashValue, jobId, lockMode) == -1) {
+        if (findLockInJobQueue(dsId, entityHashValue, jobId, lockMode) == NILL) {
             String msg = "request for " + LockMode.toString(lockMode) + " lock on dataset " + dsId + " entity "
                     + entityHashValue + " not found for job " + jobId + " in thread "
                     + Thread.currentThread().getName();
@@ -1046,19 +1004,19 @@
      *            job id
      * @param lockMode
      *            lock mode
-     * @return the slot of the request, if the lock request is found, -1 otherwise
+     * @return the slot of the request, if the lock request is found, NILL otherwise
      */
     private long findLockInJobQueue(final int dsId, final int entityHashValue, final int jobId, byte lockMode) {
-        Long jobSlot = jobIdSlotMap.get(jobId);
+        Long jobSlot = jobId2JobSlotMap.get(jobId);
         if (jobSlot == null) {
-            return -1;
+            return NILL;
         }
 
         long holder;
         synchronized (jobArenaMgr) {
             holder = jobArenaMgr.getLastHolder(jobSlot);
         }
-        while (holder != -1) {
+        while (holder != NILL) {
             long resource = reqArenaMgr.getResourceId(holder);
             if (dsId == resArenaMgr.getDatasetId(resource) && entityHashValue == resArenaMgr.getPkHashVal(resource)
                     && jobSlot == reqArenaMgr.getJobSlot(holder)
@@ -1069,7 +1027,7 @@
                 holder = reqArenaMgr.getNextJobRequest(holder);
             }
         }
-        return -1;
+        return NILL;
     }
 
     private TablePrinter getResourceTablePrinter() {
@@ -1077,7 +1035,7 @@
     }
 
     private TablePrinter getDumpTablePrinter() {
-        return new DumpTablePrinter(table, resArenaMgr, reqArenaMgr, jobArenaMgr, jobIdSlotMap);
+        return new DumpTablePrinter(table, resArenaMgr, reqArenaMgr, jobArenaMgr, jobId2JobSlotMap);
     }
 
     public String printByResource() {
@@ -1115,46 +1073,4 @@
             dumpState(os);
         }
     }
-
-    private static class DatasetLockCache {
-        private long jobId = -1;
-        private HashMap<Integer, Byte> lockCache = new HashMap<Integer, Byte>();
-        // size 1 cache to avoid the boxing/unboxing that comes with the
-        // access to the HashMap
-        private int cDsId = -1;
-        private byte cDsLockMode = -1;
-
-        public boolean contains(final int jobId, final int dsId, byte dsLockMode) {
-            if (this.jobId == jobId) {
-                if (this.cDsId == dsId && this.cDsLockMode == dsLockMode) {
-                    return true;
-                }
-                final Byte cachedLockMode = this.lockCache.get(dsId);
-                if (cachedLockMode != null && cachedLockMode == dsLockMode) {
-                    this.cDsId = dsId;
-                    this.cDsLockMode = dsLockMode;
-                    return true;
-                }
-            } else {
-                this.jobId = -1;
-                this.cDsId = -1;
-                this.cDsLockMode = -1;
-                this.lockCache.clear();
-            }
-            return false;
-        }
-
-        public void put(final int jobId, final int dsId, byte dsLockMode) {
-            this.jobId = jobId;
-            this.cDsId = dsId;
-            this.cDsLockMode = dsLockMode;
-            this.lockCache.put(dsId, dsLockMode);
-        }
-
-        @Override
-        public String toString() {
-            return "[ " + jobId + " : " + lockCache.toString() + "]";
-        }
-    }
-
 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DatasetLockInfo.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DatasetLockInfo.java
deleted file mode 100644
index dfb5d1e..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DatasetLockInfo.java
+++ /dev/null
@@ -1,536 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-
-public class DatasetLockInfo {
-    private EntityLockInfoManager entityLockInfoManager;
-    private EntityInfoManager entityInfoManager;
-    private LockWaiterManager lockWaiterManager;
-    private PrimitiveIntHashMap entityResourceHT;
-    private int IXCount;
-    private int ISCount;
-    private int XCount;
-    private int SCount;
-    private int lastHolder;
-    private int firstWaiter;
-    private int firstUpgrader;
-
-    public DatasetLockInfo(EntityLockInfoManager entityLockInfoManager, EntityInfoManager entityInfoManager,
-            LockWaiterManager lockWaiterManager) {
-        this.entityLockInfoManager = entityLockInfoManager;
-        this.entityInfoManager = entityInfoManager;
-        this.lockWaiterManager = lockWaiterManager;
-        entityResourceHT = new PrimitiveIntHashMap();
-        lastHolder = -1; //-1 stands for end of list
-        firstWaiter = -1;
-        firstUpgrader = -1;
-    }
-
-    public void increaseLockCount(byte lockMode) {
-        switch (lockMode) {
-            case LockMode.IX:
-                IXCount++;
-                break;
-            case LockMode.IS:
-                ISCount++;
-                break;
-            case LockMode.X:
-                XCount++;
-                break;
-            case LockMode.S:
-                SCount++;
-                break;
-            default:
-                throw new IllegalStateException("Invalid dataset lock mode");
-        }
-    }
-
-    public void decreaseLockCount(byte lockMode) {
-        switch (lockMode) {
-            case LockMode.IX:
-                IXCount--;
-                break;
-            case LockMode.IS:
-                ISCount--;
-                break;
-            case LockMode.X:
-                XCount--;
-                break;
-            case LockMode.S:
-                SCount--;
-                break;
-            default:
-                throw new IllegalStateException("Invalid dataset lock mode");
-        }
-    }
-
-    public void increaseLockCount(byte lockMode, int count) {
-        switch (lockMode) {
-            case LockMode.IX:
-                IXCount += count;
-                break;
-            case LockMode.IS:
-                ISCount += count;
-                break;
-            case LockMode.X:
-                XCount += count;
-                break;
-            case LockMode.S:
-                SCount += count;
-                break;
-            default:
-                throw new IllegalStateException("Invalid dataset lock mode");
-        }
-    }
-
-    public void decreaseLockCount(byte lockMode, int count) {
-        switch (lockMode) {
-            case LockMode.IX:
-                IXCount -= count;
-                break;
-            case LockMode.IS:
-                ISCount -= count;
-                break;
-            case LockMode.X:
-                XCount -= count;
-                break;
-            case LockMode.S:
-                SCount -= count;
-                break;
-            default:
-                throw new IllegalStateException("Invalid dataset lock mode");
-        }
-    }
-
-    public boolean isUpgradeCompatible(byte lockMode, int entityInfo) {
-        switch (lockMode) {
-        //upgrade from IS -> IX
-        //XCount is guaranteed to be 0.
-        //upgrade is allowed if SCount is 0.
-            case LockMode.IX:
-                return SCount == 0;
-
-                //upgrade from S -> X
-                //XCount and IXCount are guaranteed to be 0.
-                //upgrade is allowed if ISCount is 0.
-            case LockMode.X:
-                return ISCount == 0;
-
-            default:
-                throw new IllegalStateException("Invalid upgrade lock mode");
-        }
-    }
-
-    public boolean isCompatible(byte lockMode) {
-        switch (lockMode) {
-            case LockMode.IX:
-                return SCount == 0 && XCount == 0;
-
-            case LockMode.IS:
-                return XCount == 0;
-
-            case LockMode.X:
-                return ISCount == 0 && IXCount == 0 && SCount == 0 && XCount == 0;
-
-            case LockMode.S:
-                return IXCount == 0 && XCount == 0;
-
-            default:
-                throw new IllegalStateException("Invalid upgrade lock mode");
-        }
-    }
-
-    public int findEntityInfoFromHolderList(int jobId, int hashVal) {
-        int entityInfo;
-        int eLockInfo;
-        int waiterObjId;
-        if (hashVal == -1) {//dataset-granule lock
-            entityInfo = lastHolder;
-            while (entityInfo != -1) {
-                if (jobId == entityInfoManager.getJobId(entityInfo)) {
-                    return entityInfo;
-                }
-                entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
-            }
-            return -1;
-        } else { //entity-granule lock
-            eLockInfo = entityResourceHT.get(hashVal);
-            if (eLockInfo == -1) {
-                return -1;
-            }
-            entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId, hashVal);
-            if (entityInfo == -1) {
-                //find the entityInfo from the waiter list of entityLockInfo.
-                //There is a case where dataset-granule lock is acquired, but entity-granule lock is not acquired yet.
-                //In this case, the waiter of the entityLockInfo represents the holder of the datasetLockInfo.
-                waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jobId, hashVal);
-                if (waiterObjId != -1) {
-                    entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
-                }
-            }
-            return entityInfo;
-        }
-    }
-
-    public int findWaiterFromWaiterList(int jobId, int hashVal) {
-        int waiterObjId;
-        LockWaiter waiterObj;
-        int entityInfo = 0;
-
-        waiterObjId = firstWaiter;
-        while (waiterObjId != -1) {
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            if (jobId == entityInfoManager.getJobId(entityInfo)
-                    && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
-                return waiterObjId;
-            }
-            waiterObjId = waiterObj.getNextWaiterObjId();
-        }
-
-        return -1;
-    }
-
-    public int findUpgraderFromUpgraderList(int jobId, int hashVal) {
-        int waiterObjId;
-        LockWaiter waiterObj;
-        int entityInfo = 0;
-
-        waiterObjId = firstUpgrader;
-        while (waiterObjId != -1) {
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            if (jobId == entityInfoManager.getJobId(entityInfo)
-                    && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
-                return waiterObjId;
-            }
-            waiterObjId = waiterObj.getNextWaiterObjId();
-        }
-
-        return -1;
-    }
-
-    public boolean isNoHolder() {
-        return ISCount == 0 && IXCount == 0 && SCount == 0 && XCount == 0;
-    }
-
-    public void addHolder(int holder) {
-        entityInfoManager.setPrevEntityActor(holder, lastHolder);
-        lastHolder = holder;
-    }
-
-    /**
-     * Remove holder from linked list of Actor.
-     * Also, remove the corresponding resource from linked list of resource
-     * in order to minimize JobInfo's resource link traversal.
-     *
-     * @param holder
-     * @param jobInfo
-     */
-    public void removeHolder(int holder, JobInfo jobInfo) {
-        int prev = lastHolder;
-        int current = -1;
-        int next;
-
-        //remove holder from linked list of Actor
-        while (prev != holder) {
-            if (LockManager.IS_DEBUG_MODE) {
-                if (prev == -1) {
-                    //shouldn't occur: debugging purpose
-                    try {
-                        throw new Exception();
-                    } catch (Exception e) {
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            current = prev;
-            prev = entityInfoManager.getPrevEntityActor(current);
-        }
-
-        if (current != -1) {
-            //current->prev = prev->prev
-            entityInfoManager.setPrevEntityActor(current, entityInfoManager.getPrevEntityActor(prev));
-        } else {
-            //lastHolder = prev->prev
-            lastHolder = entityInfoManager.getPrevEntityActor(prev);
-        }
-
-        //Notice!!
-        //remove the corresponding resource from linked list of resource.
-        //it is guaranteed that there is no waiter or upgrader in the JobInfo when this function is called.
-        prev = entityInfoManager.getPrevJobResource(holder);
-        next = entityInfoManager.getNextJobResource(holder);
-
-        if (prev != -1) {
-            entityInfoManager.setNextJobResource(prev, next);
-        }
-
-        if (next != -1) {
-            entityInfoManager.setPrevJobResource(next, prev);
-        } else {
-            //This entityInfo(i.e., holder) is the last resource held by this job.
-            jobInfo.setlastHoldingResource(holder);
-        }
-
-        //jobInfo.decreaseDatasetLockCount(holder);
-    }
-
-    /**
-     * append new waiter to the end of waiters
-     *
-     * @param waiterObjId
-     */
-    public void addWaiter(int waiterObjId) {
-        int lastObjId;
-        LockWaiter lastObj = null;
-
-        if (firstWaiter != -1) {
-            //find the lastWaiter
-            lastObjId = firstWaiter;
-            while (lastObjId != -1) {
-                lastObj = lockWaiterManager.getLockWaiter(lastObjId);
-                lastObjId = lastObj.getNextWaiterObjId();
-            }
-            //last->next = new_waiter
-            lastObj.setNextWaiterObjId(waiterObjId);
-        } else {
-            firstWaiter = waiterObjId;
-        }
-        //new_waiter->next = -1
-        lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
-        lastObj.setNextWaiterObjId(-1);
-
-        //        if (LockManager.IS_DEBUG_MODE) {
-        //            System.out.println(printWaiters());
-        //        }
-    }
-
-    public void removeWaiter(int waiterObjId) {
-        int currentObjId = firstWaiter;
-        LockWaiter currentObj;
-        LockWaiter prevObj = null;
-        int prevObjId = -1;
-        int nextObjId;
-
-        while (currentObjId != waiterObjId) {
-
-            if (LockManager.IS_DEBUG_MODE) {
-                if (currentObjId == -1) {
-                    //shouldn't occur: debugging purpose
-                    try {
-                        throw new Exception();
-                    } catch (Exception e) {
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            prevObj = lockWaiterManager.getLockWaiter(currentObjId);
-            prevObjId = currentObjId;
-            currentObjId = prevObj.getNextWaiterObjId();
-        }
-
-        //get current waiter object
-        currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
-        //get next waiterObjId
-        nextObjId = currentObj.getNextWaiterObjId();
-
-        if (prevObjId != -1) {
-            //prev->next = next
-            prevObj.setNextWaiterObjId(nextObjId);
-        } else {
-            //removed first waiter. firstWaiter = current->next
-            firstWaiter = nextObjId;
-        }
-
-        //        if (LockManager.IS_DEBUG_MODE) {
-        //            System.out.println(printWaiters());
-        //        }
-    }
-
-    public void addUpgrader(int waiterObjId) {
-        int lastObjId;
-        LockWaiter lastObj = null;
-
-        if (firstUpgrader != -1) {
-            //find the lastWaiter
-            lastObjId = firstUpgrader;
-            while (lastObjId != -1) {
-                lastObj = lockWaiterManager.getLockWaiter(lastObjId);
-                lastObjId = lastObj.getNextWaiterObjId();
-            }
-            //last->next = new_waiter
-            lastObj.setNextWaiterObjId(waiterObjId);
-        } else {
-            firstUpgrader = waiterObjId;
-        }
-        //new_waiter->next = -1
-        lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
-        lastObj.setNextWaiterObjId(-1);
-    }
-
-    public void removeUpgrader(int waiterObjId) {
-        int currentObjId = firstUpgrader;
-        LockWaiter currentObj;
-        LockWaiter prevObj = null;
-        int prevObjId = -1;
-        int nextObjId;
-
-        while (currentObjId != waiterObjId) {
-
-            if (LockManager.IS_DEBUG_MODE) {
-                if (currentObjId == -1) {
-                    //shouldn't occur: debugging purpose
-                    try {
-                        throw new Exception();
-                    } catch (Exception e) {
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            prevObj = lockWaiterManager.getLockWaiter(currentObjId);
-            prevObjId = currentObjId;
-            currentObjId = prevObj.getNextWaiterObjId();
-        }
-
-        //get current waiter object
-        currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
-        //get next waiterObjId
-        nextObjId = currentObj.getNextWaiterObjId();
-
-        if (prevObjId != -1) {
-            //prev->next = next
-            prevObj.setNextWaiterObjId(nextObjId);
-        } else {
-            //removed first waiter. firstWaiter = current->next
-            firstUpgrader = nextObjId;
-        }
-    }
-
-    //debugging method
-    public String printWaiters() {
-        StringBuilder s = new StringBuilder();
-        int waiterObjId;
-        LockWaiter waiterObj;
-        int entityInfo;
-
-        s.append("WID\tWCT\tEID\tJID\tDID\tPK\n");
-
-        waiterObjId = firstWaiter;
-        while (waiterObjId != -1) {
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            s.append(waiterObjId).append("\t").append(waiterObj.getWaiterCount()).append("\t").append(entityInfo)
-                    .append("\t").append(entityInfoManager.getJobId(entityInfo)).append("\t")
-                    .append(entityInfoManager.getDatasetId(entityInfo)).append("\t")
-                    .append(entityInfoManager.getPKHashVal(entityInfo)).append("\n");
-            waiterObjId = waiterObj.getNextWaiterObjId();
-        }
-
-        return s.toString();
-    }
-
-    public String coreDump() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("\n\t firstUpgrader: " + firstUpgrader);
-        sb.append("\n\t firstWaiter: " + firstWaiter);
-        sb.append("\n\t lastHolder: " + lastHolder);
-        sb.append("\n\t ISCount: " + ISCount);
-        sb.append("\n\t IXCount: " + IXCount);
-        sb.append("\n\t SCount: " + SCount);
-        sb.append("\n\t XCount: " + XCount);
-        sb.append("\n\t entityResourceHT");
-        sb.append(entityResourceHT.prettyPrint());
-        return sb.toString();
-    }
-
-    /////////////////////////////////////////////////////////
-    //  set/get method for private variable
-    /////////////////////////////////////////////////////////
-    public void setIXCount(int count) {
-        IXCount = count;
-    }
-
-    public int getIXCount() {
-        return IXCount;
-    }
-
-    public void setISCount(int count) {
-        ISCount = count;
-    }
-
-    public int getISCount() {
-        return ISCount;
-    }
-
-    public void setXCount(int count) {
-        XCount = count;
-    }
-
-    public int getXCount() {
-        return XCount;
-    }
-
-    public void setSCount(int count) {
-        SCount = count;
-    }
-
-    public int getSCount() {
-        return SCount;
-    }
-
-    public void setLastHolder(int holder) {
-        lastHolder = holder;
-    }
-
-    public int getLastHolder() {
-        return lastHolder;
-    }
-
-    public void setFirstWaiter(int waiter) {
-        firstWaiter = waiter;
-    }
-
-    public int getFirstWaiter() {
-        return firstWaiter;
-    }
-
-    public void setFirstUpgrader(int upgrader) {
-        firstUpgrader = upgrader;
-    }
-
-    public int getFirstUpgrader() {
-        return firstUpgrader;
-    }
-
-    public PrimitiveIntHashMap getEntityResourceHT() {
-        return entityResourceHT;
-    }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DeadlockDetector.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DeadlockDetector.java
deleted file mode 100644
index f8a4f5e..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DeadlockDetector.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.util.HashMap;
-
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.JobId;
-
-/**
- * @author pouria, kisskys Performing a BFS search, upon adding each waiter to a waiting
- *         list to avoid deadlocks this class implements such a loop-detector in
- *         the wait-for-graph
- */
-
-public class DeadlockDetector {
-
-    public static final boolean IS_DEBUG_MODE = true;//false
-    private HashMap<JobId, JobInfo> jobHT;
-    private HashMap<DatasetId, DatasetLockInfo> datasetResourceHT;
-    private EntityLockInfoManager entityLockInfoManager;
-    private EntityInfoManager entityInfoManager;
-    private LockWaiterManager lockWaiterManager;
-
-    private PrimitiveIntHashMap holderList;
-    private PrimitiveIntHashMap nextHolderList;
-    private PrimitiveIntHashMap resourceList;
-    private PrimitiveIntHashMap visitedHolderList;
-    private JobId tempJobIdObj; //temporary object to avoid object creation
-    private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
-
-    public DeadlockDetector(HashMap<JobId, JobInfo> jobHT, HashMap<DatasetId, DatasetLockInfo> datasetResourceHT,
-            EntityLockInfoManager entityLockInfoManager, EntityInfoManager entityInfoManager,
-            LockWaiterManager lockWaiterManager) {
-        this.jobHT = jobHT;
-        this.datasetResourceHT = datasetResourceHT;
-        this.entityLockInfoManager = entityLockInfoManager;
-        this.entityInfoManager = entityInfoManager;
-        this.lockWaiterManager = lockWaiterManager;
-        holderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
-        nextHolderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
-        resourceList = new PrimitiveIntHashMap(1, 1 << 4, 180000);
-        visitedHolderList = new PrimitiveIntHashMap(1 << 6, 1 << 3, 180000);
-        tempJobIdObj = new JobId(0);
-        tempDatasetIdObj = new DatasetId(0);
-    }
-
-    public boolean isSafeToAdd(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo,
-            boolean isUpgrade) {
-        int holder;
-        int visitedHolder;
-        int callerId = entityInfoManager.getJobId(entityInfo);
-        int datasetId = entityInfoManager.getDatasetId(entityInfo);
-        int hashValue = entityInfoManager.getPKHashVal(entityInfo);
-        int resource;
-        PrimitiveIntHashMap tempHolderList;
-
-        holderList.clear(true);
-        visitedHolderList.clear(true);
-
-        //holderlist contains jobId
-        //resourceList contains entityInfo's slot numbers instead of resourceId in order to avoid object creation
-        //since resourceId consists of datasetId and PKHashValue
-
-        //get holder list(jobId list)
-        if (isDatasetLockInfo) {
-            getHolderList(datasetId, -1, holderList);
-        } else {
-            getHolderList(datasetId, hashValue, holderList);
-        }
-
-        //check whether this caller is upgrader or not
-        //if it is upgrader, then handle it as special case in the following manner
-        //if there is another upgrader or waiter of which lock mode is not compatible with the caller's lock mode,
-        //then this caller's wait causes deadlock.
-        if (holderList.get(callerId) != -1) {
-            if (isUpgrade && dLockInfo.getFirstUpgrader() != -1) {
-                return false;
-            }
-            //there is no case such that while a job is holding any mode of lock on a dataset and waits for the same dataset as an waiter.
-            //But the job may wait for the same dataset as an upgrader.
-        }
-
-        //TODO
-        //check whether when there are multiple resources, the waiter and upgrader should be distinguished or not.
-        //The current logic doesn't distinguish these two types of waiter.
-
-        //while holderList is not empty
-        holderList.beginIterate();
-        holder = holderList.getNextKey();
-        while (holder != -1) {
-
-            nextHolderList.clear(true);
-
-            while (holder != -1) {
-                resourceList.clear(true);
-                getWaitingResourceList(holder, resourceList);
-                resourceList.beginIterate();
-                resource = resourceList.getNextKey();
-
-                while (resource != -1) {
-                    //get dataset holder
-                    getHolderList(entityInfoManager.getDatasetId(resource), -1, nextHolderList);
-                    //get entity holder
-                    getHolderList(entityInfoManager.getDatasetId(resource), entityInfoManager.getPKHashVal(resource),
-                            nextHolderList);
-                    if (nextHolderList.get(callerId) != -1) {
-                        return false;
-                    }
-                    resource = resourceList.getNextKey();
-                }
-
-                visitedHolderList.put(holder, -1);
-                holder = holderList.getNextKey();
-            }
-
-            //remove visitedHolder for nextHolderList;
-            visitedHolderList.beginIterate();
-            visitedHolder = visitedHolderList.getNextKey();
-            while (visitedHolder != -1) {
-                nextHolderList.remove(visitedHolder);
-                visitedHolder = visitedHolderList.getNextKey();
-            }
-
-            //swap holder list
-            //set holderList to nextHolderList and nextHolderList to holderList
-            tempHolderList = holderList;
-            holderList = nextHolderList;
-            nextHolderList = tempHolderList;
-            holderList.beginIterate();
-            holder = holderList.getNextKey();
-        }
-
-        return true;
-    }
-
-    /**
-     * Get holder list of dataset if hashValue == -1. Get holder list of entity otherwise.
-     * Where, a holder is a jobId, not entityInfo's slotNum
-     *
-     * @param datasetId
-     * @param hashValue
-     * @param holderList
-     */
-    private void getHolderList(int datasetId, int hashValue, PrimitiveIntHashMap holderList) {
-        PrimitiveIntHashMap entityHT;
-        DatasetLockInfo dLockInfo;
-        int entityLockInfo;
-        int entityInfo;
-        int waiterObjId;
-        LockWaiter waiterObj;
-
-        //get datasetLockInfo
-        tempDatasetIdObj.setId(datasetId);
-        dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
-        if (dLockInfo == null) {
-            return;
-        }
-
-        if (hashValue == -1) {
-            //get S/X-lock holders of dataset
-            entityInfo = dLockInfo.getLastHolder();
-            while (entityInfo != -1) {
-                holderList.put(entityInfoManager.getJobId(entityInfo), 0);
-                entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
-            }
-
-            //get IS/IX-lock holders of dataset
-            entityHT = dLockInfo.getEntityResourceHT();
-            entityHT.beginIterate();
-            entityLockInfo = entityHT.getNextValue();
-            while (entityLockInfo != -1) {
-
-                //1. add holder of eLockInfo to holerList
-                entityInfo = entityLockInfoManager.getLastHolder(entityLockInfo);
-                while (entityInfo != -1) {
-                    holderList.put(entityInfoManager.getJobId(entityInfo), 0);
-                    entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
-                }
-
-                //2. add waiter of eLockInfo to holderList since waiter of entityLock is a holder of datasetLock
-                //(Upgraders need not to be added since upgraders are also holders)
-                waiterObjId = entityLockInfoManager.getFirstWaiter(entityLockInfo);
-                while (waiterObjId != -1) {
-                    waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-                    entityInfo = waiterObj.getEntityInfoSlot();
-                    holderList.put(entityInfoManager.getJobId(entityInfo), 0);
-                    waiterObjId = waiterObj.getNextWaiterObjId();
-                }
-
-                entityLockInfo = entityHT.getNextValue();
-            }
-        } else {
-            //get S/X-lock holders of entity
-            entityHT = dLockInfo.getEntityResourceHT();
-            entityLockInfo = entityHT.get(hashValue);
-            if (entityLockInfo != -1) {
-                entityInfo = entityLockInfoManager.getLastHolder(entityLockInfo);
-                while (entityInfo != -1) {
-                    holderList.put(entityInfoManager.getJobId(entityInfo), 0);
-                    entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
-                }
-            }
-        }
-        return;
-    }
-
-    /**
-     * Get waiting resource list of jobId, where a resource is represented with entityInfo's slot number
-     *
-     * @param jobId
-     * @param resourceList
-     */
-    private void getWaitingResourceList(int jobId, PrimitiveIntHashMap resourceList) {
-        JobInfo jobInfo;
-        int waiterId;
-        LockWaiter waiterObj;
-        int entityInfo;
-
-        //get JobInfo
-        tempJobIdObj.setId(jobId);
-        jobInfo = jobHT.get(tempJobIdObj);
-        if (IS_DEBUG_MODE) {
-            if (jobInfo == null) {
-                System.out.println(Thread.currentThread().getName() + "jobId:" + jobId);
-            }
-        }
-
-        //get WaiterObj
-        waiterId = jobInfo.getFirstWaitingResource();
-        while (waiterId != -1) {
-            waiterObj = lockWaiterManager.getLockWaiter(waiterId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            resourceList.put(entityInfo, -1);
-            waiterId = waiterObj.getNextWaitingResourceObjId();
-        }
-        return;
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DummyLockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DummyLockManager.java
deleted file mode 100644
index 8ffa775..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DummyLockManager.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-
-
-/**
- * A dummy implementation of the ILockManager interface. It assumes that all
- * requests are successful. It can be used to for jobs that are known to be
- * conflict free, but it'll yield terrible results if there are conflicts.
- *
- * @author tillw
- *
- */
-public class DummyLockManager implements ILockManager, ILifeCycleComponent {
-
-    public DummyLockManager(TransactionSubsystem transactionSubsystem) {
-    }
-
-    @Override
-    public void start() {
-    }
-
-    @Override
-    public void stop(boolean dumpState, OutputStream ouputStream) throws IOException {
-    }
-
-    @Override
-    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
-            throws ACIDException {
-    }
-
-    @Override
-    public void releaseLocks(ITransactionContext txnContext) throws ACIDException {
-    }
-
-    @Override
-    public void unlock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
-            throws ACIDException {
-    }
-
-    @Override
-    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext context)
-            throws ACIDException {
-    }
-
-    @Override
-    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext context)
-            throws ACIDException {
-        return true;
-    }
-
-    @Override
-    public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext) throws ACIDException {
-        return true;
-    }
-
-    @Override
-    public String prettyPrint() throws ACIDException {
-        return "DummyLockManager";
-    }
-
-    @Override
-    public void dumpState(OutputStream os) throws IOException {
-    }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
index ffdb151..a75f756 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/DumpTablePrinter.java
@@ -26,18 +26,15 @@
     private ResourceArenaManager resArenaMgr;
     private RequestArenaManager reqArenaMgr;
     private JobArenaManager jobArenaMgr;
-    private ConcurrentHashMap<Integer, Long> jobIdSlotMap;
+    private ConcurrentHashMap<Integer, Long> jobId2JobSlotMap;
 
-    DumpTablePrinter(ResourceGroupTable table,
-                     ResourceArenaManager resArenaMgr,
-                     RequestArenaManager reqArenaMgr,
-                     JobArenaManager jobArenaMgr,
-                     ConcurrentHashMap<Integer, Long> jobIdSlotMap) {
+    DumpTablePrinter(ResourceGroupTable table, ResourceArenaManager resArenaMgr, RequestArenaManager reqArenaMgr,
+            JobArenaManager jobArenaMgr, ConcurrentHashMap<Integer, Long> jobId2JobSlotMap) {
         this.table = table;
         this.resArenaMgr = resArenaMgr;
         this.reqArenaMgr = reqArenaMgr;
         this.jobArenaMgr = jobArenaMgr;
-        this.jobIdSlotMap = jobIdSlotMap;
+        this.jobId2JobSlotMap = jobId2JobSlotMap;
     }
 
     public StringBuilder append(StringBuilder sb) {
@@ -56,9 +53,9 @@
             sb.append(">>dump_end\t>>----- [reqArenaMgr] -----\n");
 
             sb.append(">>dump_begin\t>>----- [jobIdSlotMap] -----\n");
-            for (Integer i : jobIdSlotMap.keySet()) {
+            for (Integer i : jobId2JobSlotMap.keySet()) {
                 sb.append(i).append(" : ");
-                TypeUtil.Global.append(sb, jobIdSlotMap.get(i));
+                TypeUtil.Global.append(sb, jobId2JobSlotMap.get(i));
                 sb.append("\n");
             }
             sb.append(">>dump_end\t>>----- [jobIdSlotMap] -----\n");
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityInfoManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityInfoManager.java
deleted file mode 100644
index 0e607ee..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityInfoManager.java
+++ /dev/null
@@ -1,730 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-
-/**
- * EntityInfoManager provides EntityInfo arrays backed by ByteBuffer.
- * The array grows when the slots are overflowed.
- * Also, the array shrinks according to the following shrink policy
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class EntityInfoManager {
-
-    private ArrayList<ChildEntityInfoArrayManager> pArray;
-    private int allocChild; //used to allocate the next free EntityInfo slot.
-    private long shrinkTimer;
-    private boolean isShrinkTimerOn;
-    private int occupiedSlots;
-    private int shrinkTimerThreshold;
-
-    //    ////////////////////////////////////////////////
-    //    // begin of unit test
-    //    ////////////////////////////////////////////////
-    //
-    //    public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
-    //
-    //    /**
-    //     * @param args
-    //     */
-    //    public static void main(String[] args) {
-    //        final int DataSize = 5000;
-    //
-    //        int i, j;
-    //        int slots = ChildEntityInfoArrayManager.NUM_OF_SLOTS;
-    //        int data[] = new int[DataSize];
-    //        EntityInfoManager eiMgr = new EntityInfoManager();
-    //
-    //        //allocate: 50
-    //        System.out.println("allocate: 50");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = eiMgr.allocate();
-    //            }
-    //
-    //            System.out.println(eiMgr.prettyPrint());
-    //        }
-    //
-    //        //deallocate from the last child to the first child
-    //        System.out.println("deallocate from the last child to the first child");
-    //        for (i = 4; i >= 0; i--) {
-    //            for (j = i * slots + slots - 1; j >= i * slots; j--) {
-    //                eiMgr.deallocate(data[j]);
-    //            }
-    //            System.out.println(eiMgr.prettyPrint());
-    //        }
-    //
-    //        //allocate: 50
-    //        System.out.println("allocate: 50");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = eiMgr.allocate();
-    //            }
-    //
-    //            System.out.println(eiMgr.prettyPrint());
-    //        }
-    //
-    //        //deallocate from the first child to last child
-    //        System.out.println("deallocate from the first child to last child");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                eiMgr.deallocate(data[j]);
-    //            }
-    //
-    //            System.out.println(eiMgr.prettyPrint());
-    //        }
-    //
-    //        //allocate: 50
-    //        System.out.println("allocate: 50");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = eiMgr.allocate();
-    //            }
-    //
-    //            System.out.println(eiMgr.prettyPrint());
-    //        }
-    //
-    //        //deallocate from the first child to 4th child
-    //        System.out.println("deallocate from the first child to 4th child");
-    //        for (i = 0; i < 4; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                eiMgr.deallocate(data[j]);
-    //            }
-    //
-    //            System.out.println(eiMgr.prettyPrint());
-    //        }
-    //
-    //        //allocate: 40
-    //        System.out.println("allocate: 40");
-    //        for (i = 0; i < 4; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = eiMgr.allocate();
-    //            }
-    //
-    //            System.out.println(eiMgr.prettyPrint());
-    //        }
-    //    }
-    //
-    //    ////////////////////////////////////////////////
-    //    // end of unit test
-    //    ////////////////////////////////////////////////
-
-    public EntityInfoManager(int shrinkTimerThreshold) {
-        pArray = new ArrayList<ChildEntityInfoArrayManager>();
-        pArray.add(new ChildEntityInfoArrayManager());
-        allocChild = 0;
-        occupiedSlots = 0;
-        isShrinkTimerOn = false;
-        this.shrinkTimerThreshold = shrinkTimerThreshold;
-    }
-
-    public int allocate(int jobId, int datasetId, int entityHashVal, byte lockMode) {
-        int slotNum = allocate();
-        initEntityInfo(slotNum, jobId, datasetId, entityHashVal, lockMode);
-        return slotNum;
-    }
-
-    public int allocate() {
-        if (pArray.get(allocChild).isFull()) {
-            int size = pArray.size();
-            boolean bAlloc = false;
-            ChildEntityInfoArrayManager child;
-
-            //find a deinitialized child and initialze it
-            for (int i = 0; i < size; i++) {
-                child = pArray.get(i);
-                if (child.isDeinitialized()) {
-                    child.initialize();
-                    allocChild = i;
-                    bAlloc = true;
-                    break;
-                }
-            }
-
-            //allocate new child when there is no deinitialized child
-            if (!bAlloc) {
-                pArray.add(new ChildEntityInfoArrayManager());
-                allocChild = pArray.size() - 1;
-            }
-        }
-
-        occupiedSlots++;
-        return pArray.get(allocChild).allocate() + allocChild * ChildEntityInfoArrayManager.NUM_OF_SLOTS;
-    }
-
-    void deallocate(int slotNum) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).deallocate(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-        occupiedSlots--;
-
-        if (needShrink()) {
-            shrink();
-        }
-    }
-
-    /**
-     * Shrink policy:
-     * Shrink when the resource under-utilization lasts for a certain amount of time.
-     * TODO Need to figure out which of the policies is better
-     * case1.
-     * pArray status : O x x x x x O (O is initialized, x is deinitialized)
-     * In the above status, 'CURRENT' needShrink() returns 'TRUE'
-     * even if there is nothing to shrink or deallocate.
-     * It doesn't distinguish the deinitialized children from initialized children
-     * by calculating totalNumOfSlots = pArray.size() * ChildEntityInfoArrayManager.NUM_OF_SLOTS.
-     * In other words, it doesn't subtract the deinitialized children's slots.
-     * case2.
-     * pArray status : O O x x x x x
-     * However, in the above case, if we subtract the deinitialized children's slots,
-     * needShrink() will return false even if we shrink the pArray at this case.
-     *
-     * @return
-     */
-    private boolean needShrink() {
-        int size = pArray.size();
-        int usedSlots = occupiedSlots;
-        if (usedSlots == 0) {
-            usedSlots = 1;
-        }
-
-        if (size > 1 && size * ChildEntityInfoArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
-            if (isShrinkTimerOn) {
-                if (System.currentTimeMillis() - shrinkTimer >= shrinkTimerThreshold) {
-                    isShrinkTimerOn = false;
-                    return true;
-                }
-            } else {
-                //turn on timer
-                isShrinkTimerOn = true;
-                shrinkTimer = System.currentTimeMillis();
-            }
-        } else {
-            //turn off timer
-            isShrinkTimerOn = false;
-        }
-
-        return false;
-    }
-
-    /**
-     * Shrink() may
-     * deinitialize(:deallocates ByteBuffer of child) Children(s) or
-     * shrink pArray according to the deinitialized children's contiguity status.
-     * It doesn't deinitialze or shrink more than half of children at a time.
-     */
-    private void shrink() {
-        int i;
-        int removeCount = 0;
-        int size = pArray.size();
-        int maxDecreaseCount = size / 2;
-        ChildEntityInfoArrayManager child;
-
-        //The first buffer never be deinitialized.
-        for (i = 1; i < size; i++) {
-            if (pArray.get(i).isEmpty()) {
-                pArray.get(i).deinitialize();
-            }
-        }
-
-        //remove the empty buffers from the end
-        for (i = size - 1; i >= 1; i--) {
-            child = pArray.get(i);
-            if (child.isDeinitialized()) {
-                pArray.remove(i);
-                if (++removeCount == maxDecreaseCount) {
-                    break;
-                }
-            } else {
-                break;
-            }
-        }
-
-        //reset allocChild to the first buffer
-        allocChild = 0;
-
-        isShrinkTimerOn = false;
-    }
-
-    public String prettyPrint() {
-        StringBuilder s = new StringBuilder("\n########### EntityInfoManager Status #############\n");
-        int size = pArray.size();
-        ChildEntityInfoArrayManager child;
-
-        for (int i = 0; i < size; i++) {
-            child = pArray.get(i);
-            if (child.isDeinitialized()) {
-                continue;
-            }
-            s.append("child[" + i + "]");
-            s.append(child.prettyPrint());
-        }
-        return s.toString();
-    }
-
-    public void coreDump(OutputStream os) {
-        StringBuilder sb = new StringBuilder("\n\t########### EntityLockInfoManager Status #############\n");
-        int size = pArray.size();
-        ChildEntityInfoArrayManager child;
-
-        sb.append("Number of Child: " + size + "\n");
-        for (int i = 0; i < size; i++) {
-            try {
-                child = pArray.get(i);
-                sb.append("child[" + i + "]");
-                sb.append(child.prettyPrint());
-
-                os.write(sb.toString().getBytes());
-            } catch (IOException e) {
-                //ignore IOException
-            }
-            sb = new StringBuilder();
-        }
-    }
-
-    public int getShrinkTimerThreshold() {
-        return shrinkTimerThreshold;
-    }
-
-    public void initEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal, byte lockMode) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).initEntityInfo(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, jobId, datasetId, PKHashVal, lockMode);
-    }
-
-    public boolean compareEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal) {
-        return getPKHashVal(slotNum) == PKHashVal && getDatasetId(slotNum) == datasetId && getJobId(slotNum) == jobId;
-    }
-
-    public void increaseDatasetLockCount(int slotNum) {
-        setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) + 1));
-    }
-
-    public void decreaseDatasetLockCount(int slotNum) {
-        setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) - 1));
-    }
-
-    public void increaseEntityLockCount(int slotNum) {
-        setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) + 1));
-    }
-
-    public void decreaseEntityLockCount(int slotNum) {
-        setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) - 1));
-    }
-
-    public void increaseDatasetLockCount(int slotNum, int count) {
-        setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) + count));
-    }
-
-    public void decreaseDatasetLockCount(int slotNum, int count) {
-        setDatasetLockCount(slotNum, (byte) (getDatasetLockCount(slotNum) - count));
-    }
-
-    public void increaseEntityLockCount(int slotNum, int count) {
-        setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) + count));
-    }
-
-    public void decreaseEntityLockCount(int slotNum, int count) {
-        setEntityLockCount(slotNum, (byte) (getEntityLockCount(slotNum) - count));
-    }
-
-    //////////////////////////////////////////////////////////////////
-    //   set/get method for each field of EntityInfo
-    //////////////////////////////////////////////////////////////////
-
-    public void setJobId(int slotNum, int id) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setJobId(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, id);
-    }
-
-    public int getJobId(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getJobId(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setDatasetId(int slotNum, int id) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetId(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, id);
-    }
-
-    public int getDatasetId(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetId(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setPKHashVal(int slotNum, int hashVal) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPKHashVal(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, hashVal);
-    }
-
-    public int getPKHashVal(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPKHashVal(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setDatasetLockMode(int slotNum, byte mode) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetLockMode(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, mode);
-    }
-
-    public byte getDatasetLockMode(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetLockMode(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setDatasetLockCount(int slotNum, byte count) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setDatasetLockCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
-    }
-
-    public byte getDatasetLockCount(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getDatasetLockCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setEntityLockMode(int slotNum, byte mode) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setEntityLockMode(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, mode);
-    }
-
-    public byte getEntityLockMode(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getEntityLockMode(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setEntityLockCount(int slotNum, byte count) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setEntityLockCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
-    }
-
-    public byte getEntityLockCount(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getEntityLockCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    //Used for Waiter/Upgrader
-    public void setNextEntityActor(int slotNum, int nextActorSlotNum) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextEntityActor(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
-    }
-
-    //Used for Waiter/Upgrader
-    public int getNextEntityActor(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextEntityActor(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    //Used for Holder
-    public void setPrevEntityActor(int slotNum, int nextActorSlotNum) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPrevEntityActor(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
-    }
-
-    //Used for Holder
-    public int getPrevEntityActor(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPrevEntityActor(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setPrevJobResource(int slotNum, int prevResourceSlotNum) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setPrevJobResource(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, prevResourceSlotNum);
-    }
-
-    public int getPrevJobResource(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getPrevJobResource(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setNextJobResource(int slotNum, int nextResourceSlotNum) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextJobResource(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextResourceSlotNum);
-    }
-
-    public int getNextJobResource(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextJobResource(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    //    public void setNextDatasetActor(int slotNum, int nextActorSlotNum) {
-    //        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setNextDatasetActor(
-    //                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, nextActorSlotNum);
-    //    }
-    //
-    //    public int getNextDatasetActor(int slotNum) {
-    //        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getNextDatasetActor(
-    //                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    //    }
-}
-
-/******************************************
- * EntityInfo (28 bytes)
- * ****************************************
- * int jobId
- * int datasetId
- * int PKHashValue
- * byte datasetLockMode
- * byte datasetLockCount
- * byte enitityLockMode
- * byte entityLockCount
- * int nextEntityActor : actor can be either holder/waiter/upgrader
- * int prevJobResource : resource can be either dataset or entity and a job is holding/waiting/upgrading lock(s) on it.
- * int nextJobResource : resource can be either dataset or entity and a job is holding/waiting/upgrading lock(s) on it.
- * (int nextDatasetActor : actor can be either holder/waiter/upgrader) --> not used.
- *******************************************/
-
-class ChildEntityInfoArrayManager {
-    public static final int ENTITY_INFO_SIZE = 28; //28bytes
-    public static final int NUM_OF_SLOTS = 1024; //number of entities in a buffer
-    //    public static final int NUM_OF_SLOTS = 10; //for unit test
-    public static final int BUFFER_SIZE = ENTITY_INFO_SIZE * NUM_OF_SLOTS;
-
-    //byte offset of each field of EntityInfo
-    public static final int JOB_ID_OFFSET = 0;
-    public static final int DATASET_ID_OFFSET = 4;
-    public static final int PKHASH_VAL_OFFSET = 8;
-    public static final int DATASET_LOCK_MODE_OFFSET = 12;
-    public static final int DATASET_LOCK_COUNT_OFFSET = 13;
-    public static final int ENTITY_LOCK_MODE_OFFSET = 14;
-    public static final int ENTITY_LOCK_COUNT_OFFSET = 15;
-    public static final int ENTITY_ACTOR_OFFSET = 16;
-    public static final int PREV_JOB_RESOURCE_OFFSET = 20;
-    public static final int NEXT_JOB_RESOURCE_OFFSET = 24;
-    //public static final int DATASET_ACTOR_OFFSET = 28;
-
-    //byte offset of nextFreeSlotNum which shares the same space of JobId
-    //If a slot is in use, the space is used for JobId. Otherwise, it is used for nextFreeSlotNum.
-    public static final int NEXT_FREE_SLOT_OFFSET = 0;
-
-    private ByteBuffer buffer;
-    private int freeSlotNum;
-    private int occupiedSlots; //-1 represents 'deinitialized' state.
-
-    public ChildEntityInfoArrayManager() {
-        initialize();
-    }
-
-    public void initialize() {
-        this.buffer = ByteBuffer.allocate(BUFFER_SIZE);
-        this.freeSlotNum = 0;
-        this.occupiedSlots = 0;
-
-        for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
-            setNextFreeSlot(i, i + 1);
-        }
-        setNextFreeSlot(NUM_OF_SLOTS - 1, -1); //-1 represents EOL(end of link)
-    }
-
-    public int allocate() {
-        int currentSlot = freeSlotNum;
-        freeSlotNum = getNextFreeSlot(currentSlot);
-        occupiedSlots++;
-        if (LockManager.IS_DEBUG_MODE) {
-            System.out.println(Thread.currentThread().getName() + " entity allocate: " + currentSlot);
-        }
-        return currentSlot;
-    }
-
-    public void deallocate(int slotNum) {
-        setNextFreeSlot(slotNum, freeSlotNum);
-        freeSlotNum = slotNum;
-        occupiedSlots--;
-        if (LockManager.IS_DEBUG_MODE) {
-            System.out.println(Thread.currentThread().getName() + " entity deallocate: " + slotNum);
-        }
-    }
-
-    public void deinitialize() {
-        buffer = null;
-        occupiedSlots = -1;
-    }
-
-    public boolean isDeinitialized() {
-        return occupiedSlots == -1;
-    }
-
-    public boolean isFull() {
-        return occupiedSlots == NUM_OF_SLOTS;
-    }
-
-    public boolean isEmpty() {
-        return occupiedSlots == 0;
-    }
-
-    public int getNumOfOccupiedSlots() {
-        return occupiedSlots;
-    }
-
-    public int getFreeSlotNum() {
-        return freeSlotNum;
-    }
-
-    public String prettyPrint() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
-        sb.append("\n\tfreeSlotNum:" + getFreeSlotNum() + "\n");
-        sb.append("\tjid\t").append("did\t").append("PK\t").append("DLM\t").append("DLC\t").append("ELM\t")
-                .append("ELC\t").append("NEA\t").append("PJR\t").append("NJR\n");
-        for (int j = 0; j < ChildEntityInfoArrayManager.NUM_OF_SLOTS; j++) {
-            sb.append(j).append(": ");
-            sb.append("\t" + getJobId(j));
-            sb.append("\t" + getDatasetId(j));
-            sb.append("\t" + getPKHashVal(j));
-            sb.append("\t" + getDatasetLockMode(j));
-            sb.append("\t" + getDatasetLockCount(j));
-            sb.append("\t" + getEntityLockMode(j));
-            sb.append("\t" + getEntityLockCount(j));
-            sb.append("\t" + getNextEntityActor(j));
-            sb.append("\t" + getPrevJobResource(j));
-            sb.append("\t" + getNextJobResource(j));
-            sb.append("\n");
-        }
-        return sb.toString();
-    }
-
-    //////////////////////////////////////////////////////////////////
-    //   set/get method for each field of EntityInfo plus freeSlot
-    //////////////////////////////////////////////////////////////////
-    public void initEntityInfo(int slotNum, int jobId, int datasetId, int PKHashVal, byte lockMode) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET, jobId);
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET, datasetId);
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET, PKHashVal);
-        buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET, lockMode);
-        buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET, (byte) 0);
-        buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET, lockMode);
-        buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET, (byte) 0);
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, -1);
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET, -1);
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET, -1);
-        //buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET, -1);
-    }
-
-    public void setNextFreeSlot(int slotNum, int nextFreeSlot) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_FREE_SLOT_OFFSET, nextFreeSlot);
-    }
-
-    public int getNextFreeSlot(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + NEXT_FREE_SLOT_OFFSET);
-    }
-
-    public void setJobId(int slotNum, int id) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET, id);
-    }
-
-    public int getJobId(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + JOB_ID_OFFSET);
-    }
-
-    public void setDatasetId(int slotNum, int id) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET, id);
-    }
-
-    public int getDatasetId(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + DATASET_ID_OFFSET);
-    }
-
-    public void setPKHashVal(int slotNum, int hashVal) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET, hashVal);
-    }
-
-    public int getPKHashVal(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + PKHASH_VAL_OFFSET);
-    }
-
-    public void setDatasetLockMode(int slotNum, byte mode) {
-        buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET, mode);
-    }
-
-    public byte getDatasetLockMode(int slotNum) {
-        return buffer.get(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_MODE_OFFSET);
-    }
-
-    public void setDatasetLockCount(int slotNum, byte count) {
-        buffer.put(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET, count);
-    }
-
-    public byte getDatasetLockCount(int slotNum) {
-        return buffer.get(slotNum * ENTITY_INFO_SIZE + DATASET_LOCK_COUNT_OFFSET);
-    }
-
-    public void setEntityLockMode(int slotNum, byte mode) {
-        buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET, mode);
-    }
-
-    public byte getEntityLockMode(int slotNum) {
-        return buffer.get(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_MODE_OFFSET);
-    }
-
-    public void setEntityLockCount(int slotNum, byte count) {
-        buffer.put(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET, count);
-    }
-
-    public byte getEntityLockCount(int slotNum) {
-        return buffer.get(slotNum * ENTITY_INFO_SIZE + ENTITY_LOCK_COUNT_OFFSET);
-    }
-
-    //Used for Waiter/Upgrader
-    public void setNextEntityActor(int slotNum, int nextActorSlotNum) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, nextActorSlotNum);
-    }
-
-    //Used for Waiter/Upgrader
-    public int getNextEntityActor(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET);
-    }
-
-    //Used for Holder
-    public void setPrevEntityActor(int slotNum, int nextActorSlotNum) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET, nextActorSlotNum);
-    }
-
-    //Used for Holder
-    public int getPrevEntityActor(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + ENTITY_ACTOR_OFFSET);
-    }
-
-    public void setPrevJobResource(int slotNum, int prevResourceSlotNum) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET, prevResourceSlotNum);
-    }
-
-    public int getPrevJobResource(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + PREV_JOB_RESOURCE_OFFSET);
-    }
-
-    public void setNextJobResource(int slotNum, int prevResourceSlotNum) {
-        buffer.putInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET, prevResourceSlotNum);
-    }
-
-    public int getNextJobResource(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + NEXT_JOB_RESOURCE_OFFSET);
-    }
-
-    //    public void setNextDatasetActor(int slotNum, int nextActorSlotNum) {
-    //        buffer.putInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET, nextActorSlotNum);
-    //    }
-    //
-    //    public int getNextDatasetActor(int slotNum) {
-    //        return buffer.getInt(slotNum * ENTITY_INFO_SIZE + DATASET_ACTOR_OFFSET);
-    //    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityLockInfoManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityLockInfoManager.java
deleted file mode 100644
index c3de3c4..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/EntityLockInfoManager.java
+++ /dev/null
@@ -1,827 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-
-/**
- * EntityLockInfoManager provides EntityLockInfo arrays backed by ByteBuffer.
- * The array grows when the slots are overflowed.
- * Also, the array shrinks according to the following shrink policy
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class EntityLockInfoManager {
-
-    public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
-
-    private ArrayList<ChildEntityLockInfoArrayManager> pArray;
-    private int allocChild; //used to allocate the next free EntityInfo slot.
-    private long shrinkTimer;
-    private boolean isShrinkTimerOn;
-    private int occupiedSlots;
-    private EntityInfoManager entityInfoManager;
-    LockWaiterManager lockWaiterManager;
-
-    //        ////////////////////////////////////////////////
-    //        // begin of unit test
-    //        ////////////////////////////////////////////////
-    //
-    //        public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
-    //
-    //        /**
-    //         * @param args
-    //         */
-    //        public static void main(String[] args) {
-    //            final int DataSize = 5000;
-    //
-    //            int i, j;
-    //            int slots = ChildEntityLockInfoArrayManager.NUM_OF_SLOTS;
-    //            int data[] = new int[DataSize];
-    //            EntityLockInfoManager eliMgr = new EntityLockInfoManager();
-    //
-    //            //allocate: 50
-    //            System.out.println("allocate: 50");
-    //            for (i = 0; i < 5; i++) {
-    //                for (j = i * slots; j < i * slots + slots; j++) {
-    //                    data[j] = eliMgr.allocate();
-    //                }
-    //
-    //                System.out.println(eliMgr.prettyPrint());
-    //            }
-    //
-    //            //deallocate from the last child to the first child
-    //            System.out.println("deallocate from the last child to the first child");
-    //            for (i = 4; i >= 0; i--) {
-    //                for (j = i * slots + slots - 1; j >= i * slots; j--) {
-    //                    eliMgr.deallocate(data[j]);
-    //                }
-    //                System.out.println(eliMgr.prettyPrint());
-    //            }
-    //
-    //            //allocate: 50
-    //            System.out.println("allocate: 50");
-    //            for (i = 0; i < 5; i++) {
-    //                for (j = i * slots; j < i * slots + slots; j++) {
-    //                    data[j] = eliMgr.allocate();
-    //                }
-    //
-    //                System.out.println(eliMgr.prettyPrint());
-    //            }
-    //
-    //            //deallocate from the first child to last child
-    //            System.out.println("deallocate from the first child to last child");
-    //            for (i = 0; i < 5; i++) {
-    //                for (j = i * slots; j < i * slots + slots; j++) {
-    //                    eliMgr.deallocate(data[j]);
-    //                }
-    //
-    //                System.out.println(eliMgr.prettyPrint());
-    //            }
-    //
-    //            //allocate: 50
-    //            System.out.println("allocate: 50");
-    //            for (i = 0; i < 5; i++) {
-    //                for (j = i * slots; j < i * slots + slots; j++) {
-    //                    data[j] = eliMgr.allocate();
-    //                }
-    //
-    //                System.out.println(eliMgr.prettyPrint());
-    //            }
-    //
-    //            //deallocate from the first child to 4th child
-    //            System.out.println("deallocate from the first child to 4th child");
-    //            for (i = 0; i < 4; i++) {
-    //                for (j = i * slots; j < i * slots + slots; j++) {
-    //                    eliMgr.deallocate(data[j]);
-    //                }
-    //
-    //                System.out.println(eliMgr.prettyPrint());
-    //            }
-    //
-    //            //allocate: 40
-    //            System.out.println("allocate: 40");
-    //            for (i = 0; i < 4; i++) {
-    //                for (j = i * slots; j < i * slots + slots; j++) {
-    //                    data[j] = eliMgr.allocate();
-    //                }
-    //
-    //                System.out.println(eliMgr.prettyPrint());
-    //            }
-    //        }
-    //
-    //        ////////////////////////////////////////////////
-    //        // end of unit test
-    //        ////////////////////////////////////////////////
-
-    public EntityLockInfoManager(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager) {
-        pArray = new ArrayList<ChildEntityLockInfoArrayManager>();
-        pArray.add(new ChildEntityLockInfoArrayManager());
-        allocChild = 0;
-        occupiedSlots = 0;
-        isShrinkTimerOn = false;
-        this.entityInfoManager = entityInfoManager;
-        this.lockWaiterManager = lockWaiterManager;
-    }
-
-    public int allocate() {
-        if (pArray.get(allocChild).isFull()) {
-            int size = pArray.size();
-            boolean bAlloc = false;
-            ChildEntityLockInfoArrayManager child;
-
-            //find a deinitialized child and initialze it
-            for (int i = 0; i < size; i++) {
-                child = pArray.get(i);
-                if (child.isDeinitialized()) {
-                    child.initialize();
-                    allocChild = i;
-                    bAlloc = true;
-                    break;
-                }
-            }
-
-            //allocate new child when there is no deinitialized child
-            if (!bAlloc) {
-                pArray.add(new ChildEntityLockInfoArrayManager());
-                allocChild = pArray.size() - 1;
-            }
-        }
-        occupiedSlots++;
-        return pArray.get(allocChild).allocate() + allocChild * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS;
-    }
-
-    void deallocate(int slotNum) {
-        pArray.get(slotNum / ChildEntityLockInfoArrayManager.NUM_OF_SLOTS).deallocate(
-                slotNum % ChildEntityLockInfoArrayManager.NUM_OF_SLOTS);
-        occupiedSlots--;
-
-        if (needShrink()) {
-            shrink();
-        }
-    }
-
-    /**
-     * Shrink policy:
-     * Shrink when the resource under-utilization lasts for a certain amount of time.
-     * TODO Need to figure out which of the policies is better
-     * case1.
-     * pArray status : O x x x x x O (O is initialized, x is deinitialized)
-     * In the above status, 'CURRENT' needShrink() returns 'TRUE'
-     * even if there is nothing to shrink or deallocate.
-     * It doesn't distinguish the deinitialized children from initialized children
-     * by calculating totalNumOfSlots = pArray.size() * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS.
-     * In other words, it doesn't subtract the deinitialized children's slots.
-     * case2.
-     * pArray status : O O x x x x x
-     * However, in the above case, if we subtract the deinitialized children's slots,
-     * needShrink() will return false even if we shrink the pArray at this case.
-     *
-     * @return
-     */
-    private boolean needShrink() {
-        int size = pArray.size();
-        int usedSlots = occupiedSlots;
-        if (usedSlots == 0) {
-            usedSlots = 1;
-        }
-
-        if (size > 1 && size * ChildEntityLockInfoArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
-            if (isShrinkTimerOn) {
-                if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
-                    isShrinkTimerOn = false;
-                    return true;
-                }
-            } else {
-                //turn on timer
-                isShrinkTimerOn = true;
-                shrinkTimer = System.currentTimeMillis();
-            }
-        } else {
-            //turn off timer
-            isShrinkTimerOn = false;
-        }
-
-        return false;
-    }
-
-    /**
-     * Shrink() may
-     * deinitialize(:deallocates ByteBuffer of child) Children(s) or
-     * shrink pArray according to the deinitialized children's contiguity status.
-     * It doesn't deinitialze or shrink more than half of children at a time.
-     */
-    private void shrink() {
-        int i;
-        int removeCount = 0;
-        int size = pArray.size();
-        int maxDecreaseCount = size / 2;
-        ChildEntityLockInfoArrayManager child;
-
-        //The first buffer never be deinitialized.
-        for (i = 1; i < size; i++) {
-            if (pArray.get(i).isEmpty()) {
-                pArray.get(i).deinitialize();
-            }
-        }
-
-        //remove the empty buffers from the end
-        for (i = size - 1; i >= 1; i--) {
-            child = pArray.get(i);
-            if (child.isDeinitialized()) {
-                pArray.remove(i);
-                if (++removeCount == maxDecreaseCount) {
-                    break;
-                }
-            } else {
-                break;
-            }
-        }
-
-        //reset allocChild to the first buffer
-        allocChild = 0;
-
-        isShrinkTimerOn = false;
-    }
-
-    public String prettyPrint() {
-        StringBuilder s = new StringBuilder("\n########### EntityLockInfoManager Status #############\n");
-        int size = pArray.size();
-        ChildEntityLockInfoArrayManager child;
-
-        for (int i = 0; i < size; i++) {
-            child = pArray.get(i);
-            if (child.isDeinitialized()) {
-                continue;
-            }
-            s.append("child[" + i + "]");
-            s.append(child.prettyPrint());
-        }
-        return s.toString();
-    }
-
-    public void coreDump(OutputStream os) {
-        StringBuilder sb = new StringBuilder("\n\t########### EntityLockInfoManager Status #############\n");
-        int size = pArray.size();
-        ChildEntityLockInfoArrayManager child;
-
-        sb.append("Number of Child: " + size + "\n");
-        for (int i = 0; i < size; i++) {
-            try {
-                child = pArray.get(i);
-                sb.append("child[" + i + "]");
-                sb.append(child.prettyPrint());
-
-                os.write(sb.toString().getBytes());
-            } catch (IOException e) {
-                //ignore IOException
-            }
-            sb = new StringBuilder();
-        }
-    }
-
-    public int getShrinkTimerThreshold() {
-        return SHRINK_TIMER_THRESHOLD;
-    }
-
-    //debugging method
-    public String printWaiters(int slotNum) {
-        StringBuilder s = new StringBuilder();
-        int waiterObjId;
-        LockWaiter waiterObj;
-        int entityInfo;
-
-        s.append("WID\tWCT\tEID\tJID\tDID\tPK\n");
-
-        waiterObjId = getFirstWaiter(slotNum);
-        while (waiterObjId != -1) {
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            s.append(waiterObjId).append("\t").append(waiterObj.getWaiterCount()).append("\t").append(entityInfo)
-                    .append("\t").append(entityInfoManager.getJobId(entityInfo)).append("\t")
-                    .append(entityInfoManager.getDatasetId(entityInfo)).append("\t")
-                    .append(entityInfoManager.getPKHashVal(entityInfo)).append("\n");
-            waiterObjId = waiterObj.getNextWaiterObjId();
-        }
-
-        return s.toString();
-    }
-
-    public void addHolder(int slotNum, int holder) {
-        entityInfoManager.setPrevEntityActor(holder, getLastHolder(slotNum));
-        setLastHolder(slotNum, holder);
-    }
-
-    /**
-     * Remove holder from linked list of Actor.
-     * Also, remove the corresponding resource from linked list of resource
-     * in order to minimize JobInfo's resource link traversal.
-     *
-     * @param slotNum
-     * @param holder
-     * @param jobInfo
-     */
-    public void removeHolder(int slotNum, int holder, JobInfo jobInfo) {
-        int prev = getLastHolder(slotNum);
-        int current = -1;
-        int next;
-
-        //remove holder from linked list of Actor
-        while (prev != holder) {
-            if (LockManager.IS_DEBUG_MODE) {
-                if (prev == -1) {
-                    //shouldn't occur: debugging purpose
-                    try {
-                        throw new Exception();
-                    } catch (Exception e) {
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            current = prev;
-            prev = entityInfoManager.getPrevEntityActor(current);
-        }
-
-        if (current != -1) {
-            //current->prev = prev->prev
-            entityInfoManager.setPrevEntityActor(current, entityInfoManager.getPrevEntityActor(prev));
-        } else {
-            //lastHolder = prev->prev
-            setLastHolder(slotNum, entityInfoManager.getPrevEntityActor(prev));
-        }
-
-        //Notice!!
-        //remove the corresponding resource from linked list of resource.
-        prev = entityInfoManager.getPrevJobResource(holder);
-        next = entityInfoManager.getNextJobResource(holder);
-
-        if (prev != -1) {
-            entityInfoManager.setNextJobResource(prev, next);
-        }
-
-        if (next != -1) {
-            entityInfoManager.setPrevJobResource(next, prev);
-        } else {
-            //This entityInfo(i.e., holder) is the last resource held by this job.
-            jobInfo.setlastHoldingResource(prev);
-        }
-
-        //jobInfo.decreaseDatasetLockCount(holder);
-    }
-
-    public void addWaiter(int slotNum, int waiterObjId) {
-        int lastObjId;
-        LockWaiter lastObj = null;
-        int firstWaiter = getFirstWaiter(slotNum);
-
-        if (firstWaiter != -1) {
-            //find the lastWaiter
-            lastObjId = firstWaiter;
-            while (lastObjId != -1) {
-                lastObj = lockWaiterManager.getLockWaiter(lastObjId);
-                lastObjId = lastObj.getNextWaiterObjId();
-            }
-            //last->next = new_waiter
-            lastObj.setNextWaiterObjId(waiterObjId);
-        } else {
-            setFirstWaiter(slotNum, waiterObjId);
-        }
-        //new_waiter->next = -1
-        lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
-        lastObj.setNextWaiterObjId(-1);
-    }
-
-    public void removeWaiter(int slotNum, int waiterObjId) {
-        int currentObjId = getFirstWaiter(slotNum);
-        LockWaiter currentObj;
-        LockWaiter prevObj = null;
-        int prevObjId = -1;
-        int nextObjId;
-
-        while (currentObjId != waiterObjId) {
-
-            if (LockManager.IS_DEBUG_MODE) {
-                if (currentObjId == -1) {
-                    //shouldn't occur: debugging purpose
-                    try {
-                        throw new Exception();
-                    } catch (Exception e) {
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            prevObj = lockWaiterManager.getLockWaiter(currentObjId);
-            prevObjId = currentObjId;
-            currentObjId = prevObj.getNextWaiterObjId();
-        }
-
-        //get current waiter object
-        currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
-        //get next waiterObjId
-        nextObjId = currentObj.getNextWaiterObjId();
-
-        if (prevObjId != -1) {
-            //prev->next = next
-            prevObj.setNextWaiterObjId(nextObjId);
-        } else {
-            //removed first waiter. firstWaiter = current->next
-            setFirstWaiter(slotNum, nextObjId);
-        }
-    }
-
-    public void addUpgrader(int slotNum, int waiterObjId) {
-        //[Notice]
-        //Even if there are multiple threads in a job try to upgrade lock mode on same resource which is entity-granule,
-        //while the first upgrader is waiting, all the incoming upgrade requests from other threads should be rejected by aborting them.
-        //Therefore, there is no actual "ADD" upgrader method. Instead, it only has "SET" upgrader method.
-        if (LockManager.IS_DEBUG_MODE) {
-            if (getUpgrader(slotNum) != -1) {
-                throw new IllegalStateException("Invalid lock upgrade request. This call should be handled as deadlock");
-            }
-        }
-
-        setUpgrader(slotNum, waiterObjId);
-    }
-
-    public void removeUpgrader(int slotNum, int waiterObjId) {
-        setUpgrader(slotNum, -1);
-    }
-
-    public boolean isUpgradeCompatible(int slotNum, byte lockMode, int entityInfo) {
-        switch (lockMode) {
-            case LockMode.X:
-                return getSCount(slotNum) - entityInfoManager.getEntityLockCount(entityInfo) == 0;
-
-            default:
-                throw new IllegalStateException("Invalid upgrade lock mode");
-        }
-    }
-
-    public boolean isCompatible(int slotNum, byte lockMode) {
-        switch (lockMode) {
-            case LockMode.X:
-                return getSCount(slotNum) == 0 && getXCount(slotNum) == 0;
-
-            case LockMode.S:
-                return getXCount(slotNum) == 0;
-
-            default:
-                throw new IllegalStateException("Invalid upgrade lock mode");
-        }
-    }
-
-    public int findEntityInfoFromHolderList(int eLockInfo, int jobId, int hashVal) {
-        int entityInfo = getLastHolder(eLockInfo);
-
-        while (entityInfo != -1) {
-            if (jobId == entityInfoManager.getJobId(entityInfo)
-                    && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
-                return entityInfo;
-            }
-            //            if (LockManager.IS_DEBUG_MODE) {
-            //                System.out.println("eLockInfo(" + eLockInfo + "),entityInfo(" + entityInfo + "), Request[" + jobId
-            //                        + "," + hashVal + "]:Result[" + entityInfoManager.getJobId(entityInfo) + ","
-            //                        + entityInfoManager.getPKHashVal(entityInfo) + "]");
-            //            }
-            entityInfo = entityInfoManager.getPrevEntityActor(entityInfo);
-        }
-
-        return -1;
-    }
-
-    public int findWaiterFromWaiterList(int eLockInfo, int jobId, int hashVal) {
-        int waiterObjId = getFirstWaiter(eLockInfo);
-        LockWaiter waiterObj;
-        int entityInfo;
-
-        while (waiterObjId != -1) {
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            if (jobId == entityInfoManager.getJobId(entityInfo)
-                    && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
-                return waiterObjId;
-            }
-            waiterObjId = waiterObj.getNextWaiterObjId();
-        }
-
-        return -1;
-    }
-
-    public int findUpgraderFromUpgraderList(int eLockInfo, int jobId, int hashVal) {
-        int waiterObjId = getUpgrader(eLockInfo);
-        LockWaiter waiterObj;
-        int entityInfo;
-
-        if (waiterObjId != -1) {
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            if (jobId == entityInfoManager.getJobId(entityInfo)
-                    && hashVal == entityInfoManager.getPKHashVal(entityInfo)) {
-                return waiterObjId;
-            }
-        }
-
-        return -1;
-    }
-
-    public void increaseLockCount(int slotNum, byte lockMode) {
-        switch (lockMode) {
-            case LockMode.X:
-                setXCount(slotNum, (short) (getXCount(slotNum) + 1));
-                break;
-            case LockMode.S:
-                setSCount(slotNum, (short) (getSCount(slotNum) + 1));
-                break;
-            default:
-                throw new IllegalStateException("Invalid entity lock mode " + lockMode);
-        }
-    }
-
-    public void decreaseLockCount(int slotNum, byte lockMode) {
-        switch (lockMode) {
-            case LockMode.X:
-                setXCount(slotNum, (short) (getXCount(slotNum) - 1));
-                break;
-            case LockMode.S:
-                setSCount(slotNum, (short) (getSCount(slotNum) - 1));
-                break;
-            default:
-                throw new IllegalStateException("Invalid entity lock mode " + lockMode);
-        }
-    }
-
-    public void increaseLockCount(int slotNum, byte lockMode, short count) {
-        switch (lockMode) {
-            case LockMode.X:
-                setXCount(slotNum, (short) (getXCount(slotNum) + count));
-                break;
-            case LockMode.S:
-                setSCount(slotNum, (short) (getSCount(slotNum) + count));
-                break;
-            default:
-                throw new IllegalStateException("Invalid entity lock mode " + lockMode);
-        }
-    }
-
-    public void decreaseLockCount(int slotNum, byte lockMode, short count) {
-        switch (lockMode) {
-            case LockMode.X:
-                setXCount(slotNum, (short) (getXCount(slotNum) - count));
-                break;
-            case LockMode.S:
-                setSCount(slotNum, (short) (getSCount(slotNum) - count));
-                break;
-            default:
-                throw new IllegalStateException("Invalid entity lock mode " + lockMode);
-        }
-    }
-
-    //////////////////////////////////////////////////////////////////
-    //   set/get method for each field of EntityLockInfo
-    //////////////////////////////////////////////////////////////////
-
-    public void setXCount(int slotNum, short count) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setXCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
-    }
-
-    public short getXCount(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getXCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setSCount(int slotNum, short count) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setSCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, count);
-    }
-
-    public short getSCount(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getSCount(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setLastHolder(int slotNum, int holder) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setLastHolder(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, holder);
-    }
-
-    public int getLastHolder(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getLastHolder(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setFirstWaiter(int slotNum, int waiter) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setFirstWaiter(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, waiter);
-    }
-
-    public int getFirstWaiter(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getFirstWaiter(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-    public void setUpgrader(int slotNum, int upgrader) {
-        pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).setUpgrader(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS, upgrader);
-    }
-
-    public int getUpgrader(int slotNum) {
-        return pArray.get(slotNum / ChildEntityInfoArrayManager.NUM_OF_SLOTS).getUpgrader(
-                slotNum % ChildEntityInfoArrayManager.NUM_OF_SLOTS);
-    }
-
-}
-
-/******************************************
- * EntityLockInfo (16 bytes)
- * ****************************************
- * short XCount : used to represent the count of X mode lock if it is allocated. Otherwise, it represents next free slot.
- * short SCount
- * int lastHolder
- * int firstWaiter
- * int upgrader : may exist only one since there are only S and X mode lock in Entity-level
- *******************************************/
-
-class ChildEntityLockInfoArrayManager {
-    public static final int ENTITY_LOCK_INFO_SIZE = 16; //16bytes
-    public static final int NUM_OF_SLOTS = 1024; //number of entityLockInfos in a buffer
-    //public static final int NUM_OF_SLOTS = 10; //for unit test
-    public static final int BUFFER_SIZE = ENTITY_LOCK_INFO_SIZE * NUM_OF_SLOTS;
-
-    //byte offset of each field of EntityLockInfo
-    public static final int XCOUNT_OFFSET = 0;
-    public static final int SCOUNT_OFFSET = 2;
-    public static final int LAST_HOLDER_OFFSET = 4;
-    public static final int FIRST_WAITER_OFFSET = 8;
-    public static final int UPGRADER_OFFSET = 12;
-
-    //byte offset of nextFreeSlotNum which shares the same space with LastHolder field
-    //If a slot is in use, the space is used for LastHolder. Otherwise, it is used for nextFreeSlotNum.
-    public static final int NEXT_FREE_SLOT_OFFSET = 4;
-
-    private ByteBuffer buffer;
-    private int freeSlotNum;
-    private int occupiedSlots; //-1 represents 'deinitialized' state.
-
-    public ChildEntityLockInfoArrayManager() {
-        initialize();
-    }
-
-    public void initialize() {
-        this.buffer = ByteBuffer.allocate(BUFFER_SIZE);
-        this.freeSlotNum = 0;
-        this.occupiedSlots = 0;
-
-        for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
-            setNextFreeSlot(i, i + 1);
-        }
-        setNextFreeSlot(NUM_OF_SLOTS - 1, -1); //-1 represents EOL(end of link)
-    }
-
-    public int allocate() {
-        int currentSlot = freeSlotNum;
-        freeSlotNum = getNextFreeSlot(currentSlot);
-        //initialize values
-        setXCount(currentSlot, (short) 0);
-        setSCount(currentSlot, (short) 0);
-        setLastHolder(currentSlot, -1);
-        setFirstWaiter(currentSlot, -1);
-        setUpgrader(currentSlot, -1);
-        occupiedSlots++;
-        if (LockManager.IS_DEBUG_MODE) {
-            System.out.println(Thread.currentThread().getName() + " Allocated ELockInfo[" + currentSlot + "]");
-        }
-        return currentSlot;
-    }
-
-    public void deallocate(int slotNum) {
-        setNextFreeSlot(slotNum, freeSlotNum);
-        freeSlotNum = slotNum;
-        occupiedSlots--;
-        if (LockManager.IS_DEBUG_MODE) {
-            System.out.println(Thread.currentThread().getName() + " Deallocated ELockInfo[" + slotNum + "]");
-        }
-    }
-
-    public void deinitialize() {
-        buffer = null;
-        occupiedSlots = -1;
-    }
-
-    public boolean isDeinitialized() {
-        return occupiedSlots == -1;
-    }
-
-    public boolean isFull() {
-        return occupiedSlots == NUM_OF_SLOTS;
-    }
-
-    public boolean isEmpty() {
-        return occupiedSlots == 0;
-    }
-
-    public int getNumOfOccupiedSlots() {
-        return occupiedSlots;
-    }
-
-    public int getFreeSlotNum() {
-        return freeSlotNum;
-    }
-
-    public String prettyPrint() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
-        sb.append("\n\tfreeSlotNum:" + getFreeSlotNum());
-        sb.append("\n\tX\t").append("S\t").append("LH\t").append("FW\t").append("UP\n");
-        for (int j = 0; j < ChildEntityLockInfoArrayManager.NUM_OF_SLOTS; j++) {
-            sb.append(j).append(": ");
-            sb.append("\t" + getXCount(j));
-            sb.append("\t" + getSCount(j));
-            sb.append("\t" + getLastHolder(j));
-            sb.append("\t" + getFirstWaiter(j));
-            sb.append("\t" + getUpgrader(j));
-            sb.append("\n");
-        }
-        return sb.toString();
-    }
-
-    //////////////////////////////////////////////////////////////////
-    //   set/get method for each field of EntityLockInfo plus freeSlot
-    //////////////////////////////////////////////////////////////////
-
-    public void setNextFreeSlot(int slotNum, int nextFreeSlot) {
-        buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + NEXT_FREE_SLOT_OFFSET, nextFreeSlot);
-    }
-
-    public int getNextFreeSlot(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + NEXT_FREE_SLOT_OFFSET);
-    }
-
-    public void setXCount(int slotNum, short count) {
-        buffer.putShort(slotNum * ENTITY_LOCK_INFO_SIZE + XCOUNT_OFFSET, count);
-    }
-
-    public short getXCount(int slotNum) {
-        return buffer.getShort(slotNum * ENTITY_LOCK_INFO_SIZE + XCOUNT_OFFSET);
-    }
-
-    public void setSCount(int slotNum, short count) {
-        buffer.putShort(slotNum * ENTITY_LOCK_INFO_SIZE + SCOUNT_OFFSET, count);
-    }
-
-    public short getSCount(int slotNum) {
-        return buffer.getShort(slotNum * ENTITY_LOCK_INFO_SIZE + SCOUNT_OFFSET);
-    }
-
-    public void setLastHolder(int slotNum, int holder) {
-        buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + LAST_HOLDER_OFFSET, holder);
-    }
-
-    public int getLastHolder(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + LAST_HOLDER_OFFSET);
-    }
-
-    public void setFirstWaiter(int slotNum, int waiter) {
-        buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + FIRST_WAITER_OFFSET, waiter);
-    }
-
-    public int getFirstWaiter(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + FIRST_WAITER_OFFSET);
-    }
-
-    public void setUpgrader(int slotNum, int upgrader) {
-        buffer.putInt(slotNum * ENTITY_LOCK_INFO_SIZE + UPGRADER_OFFSET, upgrader);
-    }
-
-    public int getUpgrader(int slotNum) {
-        return buffer.getInt(slotNum * ENTITY_LOCK_INFO_SIZE + UPGRADER_OFFSET);
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockHashTable.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockHashTable.java
deleted file mode 100644
index bf85e69..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockHashTable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-/**
- * @author pouria Interface for a hashTable, used in the internal data
- *         structures of lockManager
- * @param <K>
- *            Type of the objects, used as keys
- * @param <V>
- *            Type of the objects, used as values
- */
-public interface ILockHashTable<K, V> {
-
-    public void put(K key, V value);
-
-    public V get(K key);
-
-    public V remove(K key);
-
-    public int getKeysetSize();
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockMatrix.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockMatrix.java
deleted file mode 100644
index e2e2e3a..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/ILockMatrix.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-/**
- * @author pouria
- *         Shows: - The conflict matrix for the locking protocol (whether two
- *         lock modes conflict with each other or not on a single resource) -
- *         Whether request to convert a lock mode to a new one is a conversion
- *         (i.e. the new lock mode is stringer than the current one) or not
- *         Each lock mode is shown/interpreted as an integer
- */
-
-public interface ILockMatrix {
-
-    /**
-     * @param mask
-     *            (current/expected) lock mask on the resource
-     * @param reqLockMode
-     *            index of the requested lockMode
-     * @return true if the lock request conflicts with the mask
-     */
-    public boolean conflicts(int mask, int reqLockMode);
-
-    /**
-     * @param currentLockMode
-     * @param reqLockMode
-     * @return true if the request is a conversion
-     */
-    public boolean isConversion(int currentLockMode, int reqLockMode);
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/JobInfo.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/JobInfo.java
deleted file mode 100644
index ebbb0b8..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/JobInfo.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-
-public class JobInfo {
-    private EntityInfoManager entityInfoManager;
-    private LockWaiterManager lockWaiterManager;
-    private ITransactionContext jobCtx;
-    private int lastHoldingResource; //resource(entity or dataset) which is held by this job lastly
-    private int firstWaitingResource; //resource(entity or dataset) which this job is waiting for
-    private int upgradingResource; //resource(entity or dataset) which this job is waiting for to upgrade
-
-    private PrimitiveIntHashMap datasetISLockHT; //used for keeping dataset-granule-lock's count acquired by this job.
-
-    public JobInfo(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager, ITransactionContext txnCtx) {
-        this.entityInfoManager = entityInfoManager;
-        this.lockWaiterManager = lockWaiterManager;
-        this.jobCtx = txnCtx;
-        this.lastHoldingResource = -1;
-        this.firstWaitingResource = -1;
-        this.upgradingResource = -1;
-        if (LockManager.ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-            //This table maintains the number of locks acquired by this jobInfo.
-            //[Notice] But this doesn't decrease the count even if the lock is released.
-            this.datasetISLockHT = new PrimitiveIntHashMap(1 << 4, 1 << 2, Integer.MAX_VALUE);
-        }
-    }
-
-    public void addHoldingResource(int resource) {
-
-        if (LockManager.IS_DEBUG_MODE) {
-            if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
-                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
-                        + entityInfoManager.getJobId(resource) + "'s resource!!!");
-            }
-            //System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddHolder:"+ resource);
-        }
-
-        if (lastHoldingResource != -1) {
-            entityInfoManager.setNextJobResource(lastHoldingResource, resource);
-        }
-        entityInfoManager.setPrevJobResource(resource, lastHoldingResource);
-        entityInfoManager.setNextJobResource(resource, -1);
-        lastHoldingResource = resource;
-    }
-
-    public void removeHoldingResource(int resource) {
-        int current = lastHoldingResource;
-        int prev;
-        int next;
-
-        if (LockManager.IS_DEBUG_MODE) {
-            if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
-                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
-                        + entityInfoManager.getJobId(resource) + "'s resource!!!");
-            }
-            //System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveHolder:"+ resource);
-        }
-
-        while (current != resource) {
-
-            if (LockManager.IS_DEBUG_MODE) {
-                if (current == -1) {
-                    //shouldn't occur: debugging purpose
-                    try {
-                        throw new Exception();
-                    } catch (Exception e) {
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            current = entityInfoManager.getPrevJobResource(current);
-        }
-
-        prev = entityInfoManager.getPrevJobResource(current);
-        next = entityInfoManager.getNextJobResource(current);
-        //update prev->next = next
-        if (prev != -1) {
-            entityInfoManager.setNextJobResource(prev, next);
-        }
-        if (next != -1) {
-            entityInfoManager.setPrevJobResource(next, prev);
-        }
-        if (lastHoldingResource == resource) {
-            lastHoldingResource = prev;
-        }
-
-        //decreaseDatasetLockCount(resource);
-    }
-
-    public void addWaitingResource(int waiterObjId) {
-        int lastObjId;
-        LockWaiter lastObj = null;
-
-        if (firstWaitingResource != -1) {
-            //find the lastWaiter
-            lastObjId = firstWaitingResource;
-            while (lastObjId != -1) {
-                lastObj = lockWaiterManager.getLockWaiter(lastObjId);
-                if (LockManager.IS_DEBUG_MODE) {
-                    int entityInfo = lastObj.getEntityInfoSlot();
-                    if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
-                        throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId()
-                                + ") has diffrent Job(JID:" + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
-                    }
-                }
-                lastObjId = lastObj.getNextWaitingResourceObjId();
-            }
-            //last->next = new_waiter
-            lastObj.setNextWaitingResourceObjId(waiterObjId);
-        } else {
-            firstWaitingResource = waiterObjId;
-        }
-        //new_waiter->next = -1
-        lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
-        if (LockManager.IS_DEBUG_MODE) {
-            int entityInfo = lastObj.getEntityInfoSlot();
-            if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
-                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
-                        + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
-            }
-        }
-        lastObj.setNextWaitingResourceObjId(-1);
-
-        //        if (LockManager.IS_DEBUG_MODE) {
-        //            System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);
-        //        }
-    }
-
-    public void removeWaitingResource(int waiterObjId) {
-        int currentObjId = firstWaitingResource;
-        LockWaiter currentObj;
-        LockWaiter prevObj = null;
-        int prevObjId = -1;
-        int nextObjId;
-
-        while (currentObjId != waiterObjId) {
-
-            if (LockManager.IS_DEBUG_MODE) {
-                if (currentObjId == -1) {
-                    //shouldn't occur: debugging purpose
-                    try {
-                        throw new Exception();
-                    } catch (Exception e) {
-                        // TODO Auto-generated catch block
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            prevObj = lockWaiterManager.getLockWaiter(currentObjId);
-            prevObjId = currentObjId;
-            currentObjId = prevObj.getNextWaitingResourceObjId();
-        }
-
-        //get current waiter object
-        currentObj = lockWaiterManager.getLockWaiter(currentObjId);
-
-        if (LockManager.IS_DEBUG_MODE) {
-            int entityInfo = currentObj.getEntityInfoSlot();
-            if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
-                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
-                        + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
-            }
-        }
-
-        //get next waiterObjId
-        nextObjId = currentObj.getNextWaitingResourceObjId();
-
-        if (prevObjId != -1) {
-            //prev->next = next
-            prevObj.setNextWaitingResourceObjId(nextObjId);
-        } else {
-            //removed first waiter. firstWaiter = current->next
-            firstWaitingResource = nextObjId;
-        }
-
-        //        if (LockManager.IS_DEBUG_MODE) {
-        //            System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);
-        //        }
-    }
-
-    public void increaseDatasetISLockCount(int datasetId) {
-        int count = datasetISLockHT.get(datasetId);
-        if (count == -1) {
-            datasetISLockHT.put(datasetId, 1);
-        } else {
-            datasetISLockHT.upsert(datasetId, count + 1);
-        }
-    }
-
-    public void decreaseDatasetISLockCount(int datasetId, int entityToDatasetLockEscalationThreshold) {
-        int count = datasetISLockHT.get(datasetId);
-        if (count >= entityToDatasetLockEscalationThreshold) {
-            //do not decrease the count since it is already escalated.
-        } else if (count > 1) {
-            datasetISLockHT.upsert(datasetId, count - 1);
-        } else if (count == 1) {
-            datasetISLockHT.remove(datasetId);
-        } else if (count <= 0) {
-            throw new IllegalStateException("Illegal state of datasetLock count(" + count + ") in JobInfo's dLockHT");
-        }
-    }
-
-    public int getDatasetISLockCount(int datasetId) {
-        int count = datasetISLockHT.get(datasetId);
-        if (count == -1) {
-            return 0;
-        } else {
-            return count;
-        }
-    }
-
-    /**********************************************************************************
-     * public boolean isDatasetLockGranted(int datasetId) {
-     * return dLockHT.get(datasetId) == -1 ? false : true;
-     * }
-     **********************************************************************************/
-
-    public boolean isDatasetLockGranted(int datasetId, byte lockMode) {
-        int entityInfo = lastHoldingResource;
-        byte datasetLockMode;
-
-        while (entityInfo != -1) {
-            datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-            datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? datasetLockMode
-                    : datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-            if (entityInfoManager.getDatasetId(entityInfo) == datasetId
-                    && isStrongerOrEqualToLockMode(datasetLockMode, lockMode)) {
-                return true;
-            }
-            entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
-        }
-        return false;
-    }
-
-    //check whether LockMode modeA is stronger than or equal to LockMode modeB
-    private boolean isStrongerOrEqualToLockMode(byte modeA, byte modeB) {
-        switch (modeB) {
-            case LockMode.X:
-                return modeA == LockMode.X;
-
-            case LockMode.IX:
-                return modeA == LockMode.IX || modeA == LockMode.X;
-
-            case LockMode.S:
-                return modeA == LockMode.S || modeA == LockMode.X;
-
-            case LockMode.IS:
-                return true;
-
-            default:
-                throw new IllegalStateException("Unsupported dataset lock mode.");
-        }
-    }
-
-    public String printHoldingResource() {
-        StringBuilder s = new StringBuilder();
-        int entityInfo = lastHoldingResource;
-
-        while (entityInfo != -1) {
-            s.append("entityInfo[").append(entityInfo).append("] ");
-            s.append(entityInfoManager.getJobId(entityInfo)).append(" ");
-            s.append(entityInfoManager.getDatasetId(entityInfo)).append(" ");
-            s.append(entityInfoManager.getPKHashVal(entityInfo)).append(" ");
-            s.append(entityInfoManager.getDatasetLockMode(entityInfo)).append(" ");
-            s.append(entityInfoManager.getDatasetLockCount(entityInfo)).append(" ");
-            s.append(entityInfoManager.getEntityLockCount(entityInfo)).append(" ");
-            s.append(entityInfoManager.getEntityLockMode(entityInfo)).append(" ");
-            s.append("\n");
-            entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
-        }
-        return s.toString();
-    }
-
-    public String coreDump() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("\n\t datasetISLockHT");
-        sb.append(datasetISLockHT.prettyPrint());
-        sb.append("\n\t firstWaitingResource: " + firstWaitingResource);
-        sb.append("\n\t lastHoldingResource: " + lastHoldingResource);
-        sb.append("\n\t upgradingResource: " + upgradingResource);
-        sb.append("\n\t jobCtx.jobId: " + jobCtx.getJobId());
-        return sb.toString();
-    }
-
-    /////////////////////////////////////////////////////////
-    //  set/get method for private variable
-    /////////////////////////////////////////////////////////
-    public void setlastHoldingResource(int resource) {
-        lastHoldingResource = resource;
-    }
-
-    public int getLastHoldingResource() {
-        return lastHoldingResource;
-    }
-
-    public void setFirstWaitingResource(int resource) {
-        firstWaitingResource = resource;
-    }
-
-    public int getFirstWaitingResource() {
-        return firstWaitingResource;
-    }
-
-    public void setUpgradingResource(int resource) {
-        upgradingResource = resource;
-    }
-
-    public int getUpgradingResource() {
-        return upgradingResource;
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManager.java
deleted file mode 100644
index f1e1915..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManager.java
+++ /dev/null
@@ -1,2289 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.apache.asterix.common.config.AsterixTransactionProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.common.transactions.LogRecord;
-import org.apache.asterix.common.transactions.LogType;
-import org.apache.asterix.transaction.management.service.logging.LogBuffer;
-import org.apache.asterix.transaction.management.service.logging.LogBufferTailReader;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import org.apache.hyracks.api.lifecycle.ILifeCycleComponent;
-
-/**
- * An implementation of the ILockManager interface for the
- * specific case of locking protocol with two lock modes: (S) and (X),
- * where S lock mode is shown by 0, and X lock mode is shown by 1.
- *
- * @author pouria, kisskys
- */
-
-public class LockManager implements ILockManager, ILifeCycleComponent {
-
-    public static final boolean IS_DEBUG_MODE = false;//true
-    //This variable indicates that the dataset granule X lock request is allowed when
-    //there are concurrent lock requests. As of 4/16/2013, we only allow the dataset granule X lock
-    //during DDL operation which is preceded by holding X latch on metadata.
-    //Therefore, we don't allow the concurrent lock requests with the dataset granule X lock.
-    public static final boolean ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS = false;
-
-    public static final boolean ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET = true;
-    private static final int DO_ESCALATE = 0;
-    private static final int ESCALATED = 1;
-    private static final int DONOT_ESCALATE = 2;
-
-    private final TransactionSubsystem txnSubsystem;
-
-    //all threads accessing to LockManager's tables such as jobHT and datasetResourceHT
-    //are serialized through LockTableLatch. All threads waiting the latch will be fairly served
-    //in FIFO manner when the latch is available.
-    private final ReadWriteLock lockTableLatch;
-    private final ReadWriteLock waiterLatch;
-    private final HashMap<JobId, JobInfo> jobHT;
-    private final HashMap<DatasetId, DatasetLockInfo> datasetResourceHT;
-
-    private final EntityLockInfoManager entityLockInfoManager;
-    private final EntityInfoManager entityInfoManager;
-    private final LockWaiterManager lockWaiterManager;
-
-    private final DeadlockDetector deadlockDetector;
-    private final TimeOutDetector toutDetector;
-    private final DatasetId tempDatasetIdObj; //temporary object to avoid object creation
-    private final JobId tempJobIdObj;
-
-    private int tryLockDatasetGranuleRevertOperation;
-
-    private LockRequestTracker lockRequestTracker; //for debugging
-    private final ConsecutiveWakeupContext consecutiveWakeupContext;
-
-    public LockManager(TransactionSubsystem txnSubsystem) throws ACIDException {
-        this.txnSubsystem = txnSubsystem;
-        this.lockTableLatch = new ReentrantReadWriteLock(true);
-        this.waiterLatch = new ReentrantReadWriteLock(true);
-        this.jobHT = new HashMap<JobId, JobInfo>();
-        this.datasetResourceHT = new HashMap<DatasetId, DatasetLockInfo>();
-        this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties()
-                .getLockManagerShrinkTimer());
-        this.lockWaiterManager = new LockWaiterManager();
-        this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
-        this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
-                entityInfoManager, lockWaiterManager);
-        this.toutDetector = new TimeOutDetector(this, txnSubsystem.getAsterixAppRuntimeContextProvider()
-                .getThreadExecutor());
-        this.tempDatasetIdObj = new DatasetId(0);
-        this.tempJobIdObj = new JobId(0);
-        this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
-        if (IS_DEBUG_MODE) {
-            this.lockRequestTracker = new LockRequestTracker();
-        }
-    }
-
-    public AsterixTransactionProperties getTransactionProperties() {
-        return this.txnSubsystem.getTransactionProperties();
-    }
-
-    @Override
-    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
-            throws ACIDException {
-        internalLock(datasetId, entityHashValue, lockMode, txnContext, false);
-    }
-
-    private void internalLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext,
-            boolean isInstant) throws ACIDException {
-
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int dId = datasetId.getId(); //int-type datasetId
-        int entityInfo;
-        int eLockInfo = -1;
-        DatasetLockInfo dLockInfo = null;
-        JobInfo jobInfo;
-        byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-        boolean doEscalate = false;
-        boolean caughtLockMgrLatchException = false;
-
-        latchLockTable();
-        try {
-            validateJob(txnContext);
-
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Requested", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
-                        dLockInfo, eLockInfo);
-            }
-
-            dLockInfo = datasetResourceHT.get(datasetId);
-            jobInfo = jobHT.get(jobId);
-
-            if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                if (!isInstant && datasetLockMode == LockMode.IS && jobInfo != null && dLockInfo != null) {
-                    int escalateStatus = needEscalateFromEntityToDataset(jobInfo, dId, lockMode);
-                    switch (escalateStatus) {
-                        case DO_ESCALATE:
-                            entityHashValue = -1;
-                            doEscalate = true;
-                            break;
-
-                        case ESCALATED:
-                            return;
-
-                        default:
-                            break;
-                    }
-                }
-            }
-
-            //#. if the datasetLockInfo doesn't exist in datasetResourceHT
-            if (dLockInfo == null || dLockInfo.isNoHolder()) {
-                if (dLockInfo == null) {
-                    dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
-                    datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
-                }
-                entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
-
-                //if dataset-granule lock
-                if (entityHashValue == -1) { //-1 stands for dataset-granule
-                    entityInfoManager.increaseDatasetLockCount(entityInfo);
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                    dLockInfo.addHolder(entityInfo);
-                } else {
-                    entityInfoManager.increaseDatasetLockCount(entityInfo);
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                    //add entityLockInfo
-                    eLockInfo = entityLockInfoManager.allocate();
-                    dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
-                    entityInfoManager.increaseEntityLockCount(entityInfo);
-                    entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
-                    entityLockInfoManager.addHolder(eLockInfo, entityInfo);
-                }
-
-                if (jobInfo == null) {
-                    jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
-                    jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
-                }
-                jobInfo.addHoldingResource(entityInfo);
-
-                if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                    if (!isInstant && datasetLockMode == LockMode.IS) {
-                        jobInfo.increaseDatasetISLockCount(dId);
-                        if (doEscalate) {
-                            throw new IllegalStateException(
-                                    "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + txnSubsystem.getTransactionProperties()
-                                                    .getEntityToDatasetLockEscalationThreshold());
-                        }
-                    }
-                }
-
-                if (IS_DEBUG_MODE) {
-                    trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
-                            dLockInfo, eLockInfo);
-                }
-
-                return;
-            }
-
-            //#. the datasetLockInfo exists in datasetResourceHT.
-            //1. handle dataset-granule lock
-            entityInfo = lockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext);
-
-            //2. handle entity-granule lock
-            if (entityHashValue != -1) {
-                lockEntityGranule(datasetId, entityHashValue, lockMode, entityInfo, txnContext);
-            }
-
-            if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                if (!isInstant) {
-                    if (doEscalate) {
-                        //jobInfo must not be null.
-                        assert jobInfo != null;
-                        jobInfo.increaseDatasetISLockCount(dId);
-                        //release pre-acquired locks
-                        releaseDatasetISLocks(jobInfo, jobId, datasetId, txnContext);
-                    } else if (datasetLockMode == LockMode.IS) {
-                        if (jobInfo == null) {
-                            jobInfo = jobHT.get(jobId);
-                            //jobInfo must not be null;
-                            assert jobInfo != null;
-                        }
-                        jobInfo.increaseDatasetISLockCount(dId);
-                    }
-                }
-            }
-
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
-                        dLockInfo, eLockInfo);
-            }
-        } catch (Exception e) {
-            if (e instanceof LockMgrLatchHandlerException) {
-                // don't unlatch
-                caughtLockMgrLatchException = true;
-                throw new ACIDException(((LockMgrLatchHandlerException) e).getInternalException());
-            }
-        } finally {
-            if (!caughtLockMgrLatchException) {
-                unlatchLockTable();
-            }
-        }
-
-        return;
-    }
-
-    private void releaseDatasetISLocks(JobInfo jobInfo, JobId jobId, DatasetId datasetId, ITransactionContext txnContext)
-            throws ACIDException {
-        int entityInfo;
-        int prevEntityInfo;
-        int entityHashValue;
-        int did;//int-type dataset id
-
-        //while traversing all holding resources,
-        //release IS locks on the escalated dataset and
-        //release S locks on the corresponding enttites
-        //by calling unlock() method.
-        entityInfo = jobInfo.getLastHoldingResource();
-        while (entityInfo != -1) {
-            prevEntityInfo = entityInfoManager.getPrevJobResource(entityInfo);
-
-            //release a lock only if the datset is the escalated dataset and
-            //the entityHashValue is not -1("not -1" means a non-dataset-level lock)
-            did = entityInfoManager.getDatasetId(entityInfo);
-            entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
-            if (did == datasetId.getId() && entityHashValue != -1) {
-                this.unlock(datasetId, entityHashValue, LockMode.ANY, txnContext);
-            }
-
-            entityInfo = prevEntityInfo;
-        }
-    }
-
-    private int needEscalateFromEntityToDataset(JobInfo jobInfo, int datasetId, byte lockMode) {
-        //we currently allow upgrade only if the lockMode is S.
-        if (lockMode != LockMode.S) {
-            return DONOT_ESCALATE;
-        }
-
-        int count = jobInfo.getDatasetISLockCount(datasetId);
-        if (count == txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold()) {
-            return DO_ESCALATE;
-        } else if (count > txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold()) {
-            return ESCALATED;
-        } else {
-            return DONOT_ESCALATE;
-        }
-    }
-
-    private void validateJob(ITransactionContext txnContext) throws ACIDException {
-        if (txnContext.getTxnState() == ITransactionManager.ABORTED) {
-            throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
-        } else if (txnContext.isTimeout()) {
-            requestAbort(txnContext);
-        }
-    }
-
-    private int lockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int dId = datasetId.getId(); //int-type datasetId
-        int waiterObjId;
-        int entityInfo = -1;
-        DatasetLockInfo dLockInfo;
-        JobInfo jobInfo;
-        boolean isUpgrade = false;
-        int weakerModeLockCount;
-        int waiterCount = 0;
-        byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-
-        dLockInfo = datasetResourceHT.get(datasetId);
-        jobInfo = jobHT.get(jobId);
-
-        //check duplicated call
-
-        //1. lock request causing duplicated upgrading requests from different threads in a same job
-        waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
-        if (waiterObjId != -1) {
-            //make the caller wait on the same LockWaiter object
-            entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
-            waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo, waiterObjId);
-
-            //Only for the first-get-up thread, the waiterCount will be more than 0 and
-            //the thread updates lock count on behalf of the all other waiting threads.
-            //Therefore, all the next-get-up threads will not update any lock count.
-            if (waiterCount > 0) {
-                //add ((the number of waiting upgrader) - 1) to entityInfo's dataset lock count and datasetLockInfo's lock count
-                //where -1 is for not counting the first upgrader's request since the lock count for the first upgrader's request
-                //is already counted.
-                weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-                entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
-                entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount - 1);
-
-                if (entityHashValue == -1) { //dataset-granule lock
-                    dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount + waiterCount - 1);//new lock mode
-                    dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
-                } else {
-                    dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount + waiterCount - 1);
-                    dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
-                }
-            }
-
-            return entityInfo;
-        }
-
-        //2. lock request causing duplicated waiting requests from different threads in a same job
-        waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
-        if (waiterObjId != -1) {
-            //make the caller wait on the same LockWaiter object
-            entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
-            waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, false, true, txnContext, jobInfo, waiterObjId);
-
-            if (waiterCount > 0) {
-                entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount);
-                if (entityHashValue == -1) {
-                    dLockInfo.increaseLockCount(datasetLockMode, waiterCount);
-                    dLockInfo.addHolder(entityInfo);
-                } else {
-                    dLockInfo.increaseLockCount(datasetLockMode, waiterCount);
-                    //IS and IX holders are implicitly handled.
-                }
-                //add entityInfo to JobInfo's holding-resource list
-                jobInfo.addHoldingResource(entityInfo);
-            }
-
-            return entityInfo;
-        }
-
-        //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
-        entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
-        if (entityInfo == -1) {
-
-            entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
-            if (jobInfo == null) {
-                jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
-                jobHT.put(jobId, jobInfo);
-            }
-
-            //wait if any upgrader exists or upgrading lock mode is not compatible
-            if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
-                    || !dLockInfo.isCompatible(datasetLockMode)) {
-
-                /////////////////////////////////////////////////////////////////////////////////////////////
-                //[Notice] Mimicking SIX mode
-                //When the lock escalation from IS to S in dataset-level is allowed, the following case occurs
-                //DatasetLockInfo's SCount = 1 and the same job who carried out the escalation tries to insert,
-                //then the job should be able to insert without being blocked by itself.
-                //Our approach is to introduce SIX mode, but we don't have currently,
-                //so I simply mimicking SIX by allowing S and IX coexist in the dataset level
-                //only if their job id is identical for the requests.
-                if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                    if (datasetLockMode == LockMode.IX && dLockInfo.getSCount() == 1
-                            && jobInfo.isDatasetLockGranted(dId, LockMode.S)) {
-                        entityInfoManager.increaseDatasetLockCount(entityInfo);
-                        //IX holders are implicitly handled without adding holder
-                        dLockInfo.increaseLockCount(datasetLockMode);
-                        //add entityInfo to JobInfo's holding-resource list
-                        jobInfo.addHoldingResource(entityInfo);
-                        return entityInfo;
-                    }
-                }
-                ///////////////////////////////////////////////////////////////////////////////////////////////
-
-                /////////////////////////////////////////////////////////////////////////////////////////////
-                if (ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS) {
-                    //The following case only may occur when the dataset level X lock is requested
-                    //with the other lock
-
-                    //[Notice]
-                    //There has been no same caller as (jId, dId, entityHashValue) triplet.
-                    //But there could be the same caller in terms of (jId, dId) pair.
-                    //For example,
-                    //1) (J1, D1, E1) acquires IS in Dataset D1
-                    //2) (J2, D1, -1) requests X  in Dataset D1, but waits
-                    //3) (J1, D1, E2) requests IS in Dataset D1, but should wait
-                    //The 3) may cause deadlock if 1) and 3) are under the same thread.
-                    //Even if (J1, D1, E1) and (J1, D1, E2) are two different thread, instead of
-                    //aborting (J1, D1, E1) triggered by the deadlock, we give higher priority to 3) than 2)
-                    //as long as the dataset level lock D1 is being held by the same jobId.
-                    //The above consideration is covered in the following code.
-                    //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
-                    if (jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
-                        if (dLockInfo.isCompatible(datasetLockMode)) {
-                            //this is duplicated call
-                            entityInfoManager.increaseDatasetLockCount(entityInfo);
-                            if (entityHashValue == -1) {
-                                dLockInfo.increaseLockCount(datasetLockMode);
-                                dLockInfo.addHolder(entityInfo);
-                            } else {
-                                dLockInfo.increaseLockCount(datasetLockMode);
-                                //IS and IX holders are implicitly handled.
-                            }
-                            //add entityInfo to JobInfo's holding-resource list
-                            jobInfo.addHoldingResource(entityInfo);
-
-                            return entityInfo;
-                        } else {
-                            //considered as upgrader
-                            waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo,
-                                    -1);
-                            if (waiterCount > 0) {
-                                entityInfoManager.increaseDatasetLockCount(entityInfo);
-                                if (entityHashValue == -1) {
-                                    dLockInfo.increaseLockCount(datasetLockMode);
-                                    dLockInfo.addHolder(entityInfo);
-                                } else {
-                                    dLockInfo.increaseLockCount(datasetLockMode);
-                                    //IS and IX holders are implicitly handled.
-                                }
-                                //add entityInfo to JobInfo's holding-resource list
-                                jobInfo.addHoldingResource(entityInfo);
-                            }
-                            return entityInfo;
-                        }
-                    }
-                }
-                /////////////////////////////////////////////////////////////////////////////////////////////
-
-                waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, false, true, txnContext, jobInfo, -1);
-            } else {
-                waiterCount = 1;
-            }
-
-            if (waiterCount > 0) {
-                entityInfoManager.increaseDatasetLockCount(entityInfo);
-                if (entityHashValue == -1) {
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                    dLockInfo.addHolder(entityInfo);
-                } else {
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                    //IS and IX holders are implicitly handled.
-                }
-                //add entityInfo to JobInfo's holding-resource list
-                jobInfo.addHoldingResource(entityInfo);
-            }
-        } else {
-            isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
-            if (isUpgrade) { //upgrade call
-                //wait if any upgrader exists or upgrading lock mode is not compatible
-                if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
-                    waiterCount = handleLockWaiter(dLockInfo, -1, entityInfo, true, true, txnContext, jobInfo, -1);
-                } else {
-                    waiterCount = 1;
-                }
-
-                if (waiterCount > 0) {
-                    //add ((the number of waiting upgrader) - 1) to entityInfo's dataset lock count and datasetLockInfo's lock count
-                    //where -1 is for not counting the first upgrader's request since the lock count for the first upgrader's request
-                    //is already counted.
-                    weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-                    entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
-                    entityInfoManager.increaseDatasetLockCount(entityInfo, waiterCount - 1);
-
-                    if (entityHashValue == -1) { //dataset-granule lock
-                        dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount + waiterCount - 1);//new lock mode
-                        dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
-                    } else {
-                        dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount + waiterCount - 1);
-                        dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
-                    }
-                }
-            } else { //duplicated call
-                entityInfoManager.increaseDatasetLockCount(entityInfo);
-                datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-
-                if (entityHashValue == -1) { //dataset-granule
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                } else { //entity-granule
-                    datasetLockMode = datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                }
-            }
-        }
-
-        return entityInfo;
-    }
-
-    private void lockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            int entityInfoFromDLockInfo, ITransactionContext txnContext) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int waiterObjId;
-        int eLockInfo = -1;
-        int entityInfo;
-        DatasetLockInfo dLockInfo;
-        JobInfo jobInfo;
-        boolean isUpgrade = false;
-        int waiterCount = 0;
-        int weakerModeLockCount;
-
-        dLockInfo = datasetResourceHT.get(datasetId);
-        jobInfo = jobHT.get(jobId);
-        eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
-        if (eLockInfo != -1) {
-            //check duplicated call
-
-            //1. lock request causing duplicated upgrading requests from different threads in a same job
-            waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
-            if (waiterObjId != -1) {
-                entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
-                waiterCount = handleLockWaiter(dLockInfo, eLockInfo, -1, true, false, txnContext, jobInfo, waiterObjId);
-
-                if (waiterCount > 0) {
-                    weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
-                    entityInfoManager.setEntityLockMode(entityInfo, LockMode.X);
-                    entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount - 1);
-
-                    entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) (weakerModeLockCount
-                            + waiterCount - 1));//new lock mode
-                    entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
-                }
-                return;
-            }
-
-            //2. lock request causing duplicated waiting requests from different threads in a same job
-            waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
-            if (waiterObjId != -1) {
-                entityInfo = lockWaiterManager.getLockWaiter(waiterObjId).getEntityInfoSlot();
-                waiterCount = handleLockWaiter(dLockInfo, eLockInfo, -1, false, false, txnContext, jobInfo, waiterObjId);
-
-                if (waiterCount > 0) {
-                    entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount);
-                    entityLockInfoManager.increaseLockCount(eLockInfo, lockMode, (short) waiterCount);
-                    entityLockInfoManager.addHolder(eLockInfo, entityInfo);
-                }
-                return;
-            }
-
-            //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
-            entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
-            if (entityInfo != -1) {//duplicated call or upgrader
-
-                isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
-                if (isUpgrade) {//upgrade call
-                    //wait if any upgrader exists or upgrading lock mode is not compatible
-                    if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
-                            || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
-                        waiterCount = handleLockWaiter(dLockInfo, eLockInfo, entityInfo, true, false, txnContext,
-                                jobInfo, -1);
-                    } else {
-                        waiterCount = 1;
-                    }
-
-                    if (waiterCount > 0) {
-                        weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
-                        entityInfoManager.setEntityLockMode(entityInfo, lockMode);
-                        entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount - 1);
-
-                        entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) (weakerModeLockCount
-                                + waiterCount - 1));//new lock mode
-                        entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
-                    }
-
-                } else {//duplicated call
-                    entityInfoManager.increaseEntityLockCount(entityInfo);
-                    entityLockInfoManager.increaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
-                }
-            } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
-                entityInfo = entityInfoFromDLockInfo;
-                if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
-                        || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
-                        || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
-                    waiterCount = handleLockWaiter(dLockInfo, eLockInfo, entityInfo, false, false, txnContext, jobInfo,
-                            -1);
-                } else {
-                    waiterCount = 1;
-                }
-
-                if (waiterCount > 0) {
-                    entityInfoManager.increaseEntityLockCount(entityInfo, waiterCount);
-                    entityLockInfoManager.increaseLockCount(eLockInfo, lockMode, (short) waiterCount);
-                    entityLockInfoManager.addHolder(eLockInfo, entityInfo);
-                }
-            }
-        } else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
-            eLockInfo = entityLockInfoManager.allocate();
-            dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
-            entityInfoManager.increaseEntityLockCount(entityInfoFromDLockInfo);
-            entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
-            entityLockInfoManager.addHolder(eLockInfo, entityInfoFromDLockInfo);
-        }
-    }
-
-    @Override
-    public void unlock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
-            throws ACIDException {
-        internalUnlock(datasetId, entityHashValue, txnContext, false);
-    }
-
-    private void instantUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
-            throws ACIDException {
-        internalUnlock(datasetId, entityHashValue, txnContext, true);
-    }
-
-    private void internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
-            boolean isInstant) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int eLockInfo = -1;
-        DatasetLockInfo dLockInfo = null;
-        JobInfo jobInfo;
-        int entityInfo = -1;
-        byte datasetLockMode;
-
-        if (IS_DEBUG_MODE) {
-            if (entityHashValue == -1) {
-                throw new UnsupportedOperationException(
-                        "Unsupported unlock request: dataset-granule unlock is not supported");
-            }
-        }
-
-        latchLockTable();
-        try {
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
-                        dLockInfo, eLockInfo);
-            }
-
-            //find the resource to be unlocked
-            dLockInfo = datasetResourceHT.get(datasetId);
-            jobInfo = jobHT.get(jobId);
-            if (dLockInfo == null || jobInfo == null) {
-                throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
-            }
-
-            eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
-            if (eLockInfo == -1) {
-                throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
-            }
-
-            //find the corresponding entityInfo
-            entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId.getId(), entityHashValue);
-            if (entityInfo == -1) {
-                throw new IllegalStateException("Invalid unlock request[" + jobId.getId() + "," + datasetId.getId()
-                        + "," + entityHashValue + "]: Corresponding lock info doesn't exist.");
-            }
-
-            datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo) == LockMode.S ? LockMode.IS
-                    : LockMode.IX;
-
-            //decrease the corresponding count of dLockInfo/eLockInfo/entityInfo
-            dLockInfo.decreaseLockCount(datasetLockMode);
-            entityLockInfoManager.decreaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
-            entityInfoManager.decreaseDatasetLockCount(entityInfo);
-            entityInfoManager.decreaseEntityLockCount(entityInfo);
-
-            if (entityInfoManager.getEntityLockCount(entityInfo) == 0
-                    && entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
-                int threadCount = 0; //number of threads(in the same job) waiting on the same resource
-                int waiterObjId = jobInfo.getFirstWaitingResource();
-                int waitingEntityInfo;
-                LockWaiter waiterObj;
-
-                //1) wake up waiters and remove holder
-                //wake up waiters of dataset-granule lock
-                wakeUpDatasetLockWaiters(dLockInfo);
-                //wake up waiters of entity-granule lock
-                wakeUpEntityLockWaiters(eLockInfo);
-                //remove the holder from eLockInfo's holder list and remove the holding resource from jobInfo's holding resource list
-                //this can be done in the following single function call.
-                entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
-
-                //2) if
-                //      there is no waiting thread on the same resource (this can be checked through jobInfo)
-                //   then
-                //      a) delete the corresponding entityInfo
-                //      b) write commit log for the unlocked resource(which is a committed txn).
-                while (waiterObjId != -1) {
-                    waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-                    waitingEntityInfo = waiterObj.getEntityInfoSlot();
-                    if (entityInfoManager.getDatasetId(waitingEntityInfo) == datasetId.getId()
-                            && entityInfoManager.getPKHashVal(waitingEntityInfo) == entityHashValue) {
-                        threadCount++;
-                        break;
-                    }
-                    waiterObjId = waiterObj.getNextWaiterObjId();
-                }
-                if (threadCount == 0) {
-                    entityInfoManager.deallocate(entityInfo);
-                }
-            }
-
-            //deallocate entityLockInfo's slot if there is no txn referring to the entityLockInfo.
-            if (entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
-                    && entityLockInfoManager.getLastHolder(eLockInfo) == -1
-                    && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
-                dLockInfo.getEntityResourceHT().remove(entityHashValue);
-                entityLockInfoManager.deallocate(eLockInfo);
-            }
-
-            //we don't deallocate datasetLockInfo even if there is no txn referring to the datasetLockInfo
-            //since the datasetLockInfo is likely to be referred to again.
-
-            if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                if (!isInstant && datasetLockMode == LockMode.IS) {
-                    jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties()
-                            .getEntityToDatasetLockEscalationThreshold());
-                }
-            }
-
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Granted", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
-                        dLockInfo, eLockInfo);
-            }
-        } finally {
-            unlatchLockTable();
-        }
-    }
-
-    @Override
-    public void releaseLocks(ITransactionContext txnContext) throws ACIDException {
-        LockWaiter waiterObj;
-        int entityInfo;
-        int prevEntityInfo;
-        int entityHashValue;
-        DatasetLockInfo dLockInfo = null;
-        int eLockInfo = -1;
-        int did;//int-type dataset id
-        int datasetLockCount;
-        int entityLockCount;
-        byte lockMode;
-        boolean existWaiter = false;
-
-        JobId jobId = txnContext.getJobId();
-
-        latchLockTable();
-        try {
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Requested", RequestType.RELEASE_LOCKS, new DatasetId(0), 0, (byte) 0, txnContext,
-                        dLockInfo, eLockInfo);
-            }
-
-            JobInfo jobInfo = jobHT.get(jobId);
-            if (jobInfo == null) {
-                return;
-            }
-
-            //remove waiterObj of JobInfo
-            //[Notice]
-            //waiterObjs may exist if aborted thread is the caller of this function.
-            //Even if there are the waiterObjs, there is no waiting thread on the objects.
-            //If the caller of this function is an aborted thread, it is guaranteed that there is no waiting threads
-            //on the waiterObjs since when the aborted caller thread is waken up, all other waiting threads are
-            //also waken up at the same time through 'notifyAll()' call.
-            //In contrast, if the caller of this function is not an aborted thread, then there is no waiting object.
-            int waiterObjId = jobInfo.getFirstWaitingResource();
-            int nextWaiterObjId;
-            while (waiterObjId != -1) {
-                existWaiter = true;
-                waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-                nextWaiterObjId = waiterObj.getNextWaitingResourceObjId();
-                entityInfo = waiterObj.getEntityInfoSlot();
-                if (IS_DEBUG_MODE) {
-                    if (jobId.getId() != entityInfoManager.getJobId(entityInfo)) {
-                        throw new IllegalStateException("JobInfo(" + jobId + ") has diffrent Job(JID:"
-                                + entityInfoManager.getJobId(entityInfo) + "'s lock request!!!");
-                    }
-                }
-
-                //1. remove from waiter(or upgrader)'s list of dLockInfo or eLockInfo.
-                did = entityInfoManager.getDatasetId(entityInfo);
-                tempDatasetIdObj.setId(did);
-                dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
-
-                if (waiterObj.isWaitingOnEntityLock()) {
-                    entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
-                    eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-                    if (waiterObj.isWaiter()) {
-                        entityLockInfoManager.removeWaiter(eLockInfo, waiterObjId);
-                    } else {
-                        entityLockInfoManager.removeUpgrader(eLockInfo, waiterObjId);
-                    }
-                } else {
-                    if (waiterObj.isWaiter()) {
-                        dLockInfo.removeWaiter(waiterObjId);
-                    } else {
-                        dLockInfo.removeUpgrader(waiterObjId);
-                    }
-                }
-
-                //2. wake-up waiters
-                latchWaitNotify();
-                synchronized (waiterObj) {
-                    unlatchWaitNotify();
-                    waiterObj.setWait(false);
-                    if (IS_DEBUG_MODE) {
-                        System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID(" + waiterObjId
-                                + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
-                    }
-                    waiterObj.notifyAll();
-                }
-
-                //3. deallocate waiterObj
-                lockWaiterManager.deallocate(waiterObjId);
-
-                //4. deallocate entityInfo only if this waiter is not an upgrader
-                if (entityInfoManager.getDatasetLockCount(entityInfo) == 0
-                        && entityInfoManager.getEntityLockCount(entityInfo) == 0) {
-                    entityInfoManager.deallocate(entityInfo);
-                }
-                waiterObjId = nextWaiterObjId;
-            }
-
-            //release holding resources
-            entityInfo = jobInfo.getLastHoldingResource();
-            while (entityInfo != -1) {
-                prevEntityInfo = entityInfoManager.getPrevJobResource(entityInfo);
-
-                //decrease lock count of datasetLock and entityLock
-                did = entityInfoManager.getDatasetId(entityInfo);
-                tempDatasetIdObj.setId(did);
-                dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
-                entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
-
-                if (entityHashValue == -1) {
-                    //decrease datasetLockCount
-                    lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-                    datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-                    if (datasetLockCount != 0) {
-                        dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
-
-                        //wakeup waiters of datasetLock and remove holder from datasetLockInfo
-                        wakeUpDatasetLockWaiters(dLockInfo);
-
-                        //remove the holder from datasetLockInfo only if the lock is dataset-granule lock.
-                        //--> this also removes the holding resource from jobInfo
-                        //(Because the IX and IS lock's holders are handled implicitly,
-                        //those are not in the holder list of datasetLockInfo.)
-                        dLockInfo.removeHolder(entityInfo, jobInfo);
-                    }
-                } else {
-                    //decrease datasetLockCount
-                    lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-                    lockMode = lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-                    datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-
-                    if (datasetLockCount != 0) {
-                        dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
-                    }
-
-                    //decrease entityLockCount
-                    lockMode = entityInfoManager.getEntityLockMode(entityInfo);
-                    entityLockCount = entityInfoManager.getEntityLockCount(entityInfo);
-                    eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-                    if (IS_DEBUG_MODE) {
-                        if (eLockInfo < 0) {
-                            System.out.println("eLockInfo:" + eLockInfo);
-                        }
-                    }
-
-                    if (entityLockCount != 0) {
-                        entityLockInfoManager.decreaseLockCount(eLockInfo, lockMode, (short) entityLockCount);
-                    }
-
-                    if (datasetLockCount != 0) {
-                        //wakeup waiters of datasetLock and don't remove holder from datasetLockInfo
-                        wakeUpDatasetLockWaiters(dLockInfo);
-                    }
-
-                    if (entityLockCount != 0) {
-                        //wakeup waiters of entityLock
-                        wakeUpEntityLockWaiters(eLockInfo);
-
-                        //remove the holder from entityLockInfo
-                        //--> this also removes the holding resource from jobInfo
-                        entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
-                    }
-
-                    //deallocate entityLockInfo if there is no holder and waiter.
-                    if (entityLockInfoManager.getLastHolder(eLockInfo) == -1
-                            && entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
-                            && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
-                        dLockInfo.getEntityResourceHT().remove(entityHashValue);
-                        entityLockInfoManager.deallocate(eLockInfo);
-                    }
-                }
-
-                //deallocate entityInfo
-                entityInfoManager.deallocate(entityInfo);
-
-                entityInfo = prevEntityInfo;
-            }
-
-            //remove JobInfo
-            jobHT.remove(jobId);
-
-            if (existWaiter) {
-                txnContext.setTimeout(true);
-                txnContext.setTxnState(ITransactionManager.ABORTED);
-            }
-
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Granted", RequestType.RELEASE_LOCKS, new DatasetId(0), 0, (byte) 0, txnContext,
-                        dLockInfo, eLockInfo);
-            }
-        } finally {
-            unlatchLockTable();
-        }
-    }
-
-    @Override
-    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
-            throws ACIDException {
-
-        //        try {
-        //            internalLock(datasetId, entityHashValue, lockMode, txnContext);
-        //            return;
-        //        } finally {
-        //            unlock(datasetId, entityHashValue, txnContext);
-        //        }
-        internalLock(datasetId, entityHashValue, lockMode, txnContext, true);
-        instantUnlock(datasetId, entityHashValue, txnContext);
-    }
-
-    @Override
-    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, ITransactionContext txnContext)
-            throws ACIDException {
-        return internalTryLock(datasetId, entityHashValue, lockMode, txnContext, false);
-    }
-
-    @Override
-    public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext) throws ACIDException {
-        return internalInstantTryLock(datasetId, entityHashValue, lockMode, txnContext);
-    }
-
-    private boolean internalInstantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext) throws ACIDException {
-        DatasetLockInfo dLockInfo = null;
-        boolean isSuccess = true;
-
-        latchLockTable();
-        try {
-            validateJob(txnContext);
-
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Requested", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
-                        txnContext, dLockInfo, -1);
-            }
-
-            dLockInfo = datasetResourceHT.get(datasetId);
-
-            //#. if the datasetLockInfo doesn't exist in datasetResourceHT
-            if (dLockInfo == null || dLockInfo.isNoHolder()) {
-                if (IS_DEBUG_MODE) {
-                    trackLockRequest("Granted", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
-                            txnContext, dLockInfo, -1);
-                }
-                return true;
-            }
-
-            //#. the datasetLockInfo exists in datasetResourceHT.
-            //1. handle dataset-granule lock
-            byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS
-                    : LockMode.IX;
-            if (datasetLockMode == LockMode.IS) {
-                //[Notice]
-                //Skip checking the dataset level lock compatibility if the requested LockMode is IS lock.
-                //We know that this internalInstantTryLock() call with IS lock mode will be always granted
-                //because we don't allow X lock on dataset-level except DDL operation.
-                //During DDL operation, all other operations will be pending, so there is no conflict.
-                isSuccess = true;
-            } else {
-                isSuccess = instantTryLockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext, dLockInfo,
-                        datasetLockMode);
-            }
-
-            if (isSuccess && entityHashValue != -1) {
-                //2. handle entity-granule lock
-                isSuccess = instantTryLockEntityGranule(datasetId, entityHashValue, lockMode, txnContext, dLockInfo);
-            }
-
-            if (IS_DEBUG_MODE) {
-                if (isSuccess) {
-                    trackLockRequest("Granted", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
-                            txnContext, dLockInfo, -1);
-                } else {
-                    trackLockRequest("Failed", RequestType.INSTANT_TRY_LOCK, datasetId, entityHashValue, lockMode,
-                            txnContext, dLockInfo, -1);
-                }
-            }
-
-        } finally {
-            unlatchLockTable();
-        }
-
-        return isSuccess;
-    }
-
-    private boolean instantTryLockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext, DatasetLockInfo dLockInfo, byte datasetLockMode) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int dId = datasetId.getId(); //int-type datasetId
-        int waiterObjId;
-        int entityInfo = -1;
-        JobInfo jobInfo;
-        boolean isUpgrade = false;
-
-        jobInfo = jobHT.get(jobId);
-
-        //check duplicated call
-
-        //1. lock request causing duplicated upgrading requests from different threads in a same job
-        waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
-        if (waiterObjId != -1) {
-            return false;
-        }
-
-        //2. lock request causing duplicated waiting requests from different threads in a same job
-        waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
-        if (waiterObjId != -1) {
-            return false;
-        }
-
-        //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
-        entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
-        if (entityInfo == -1) { //new call from this job -> doesn't mean that eLockInfo doesn't exist since another thread might have create the eLockInfo already.
-
-            //return fail if any upgrader exists or upgrading lock mode is not compatible
-            if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
-                    || !dLockInfo.isCompatible(datasetLockMode)) {
-
-                if (ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS) {
-                    //The following case only may occur when the dataset level X lock is requested
-                    //with the other lock
-
-                    //[Notice]
-                    //There has been no same caller as (jId, dId, entityHashValue) triplet.
-                    //But there could be the same caller in terms of (jId, dId) pair.
-                    //For example,
-                    //1) (J1, D1, E1) acquires IS in Dataset D1
-                    //2) (J2, D1, -1) requests X  in Dataset D1, but waits
-                    //3) (J1, D1, E2) requests IS in Dataset D1, but should wait
-                    //The 3) may cause deadlock if 1) and 3) are under the same thread.
-                    //Even if (J1, D1, E1) and (J1, D1, E2) are two different thread, instead of
-                    //aborting (J1, D1, E1) triggered by the deadlock, we give higher priority to 3) than 2)
-                    //as long as the dataset level lock D1 is being held by the same jobId.
-                    //The above consideration is covered in the following code.
-                    //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
-                    if (jobInfo != null && jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
-                        if (dLockInfo.isCompatible(datasetLockMode)) {
-                            //this is duplicated call
-                            return true;
-                        }
-                    }
-                }
-
-                return false;
-            }
-        } else {
-            isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
-            if (isUpgrade) { //upgrade call
-                //return fail if any upgrader exists or upgrading lock mode is not compatible
-                if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
-                    return false;
-                }
-            }
-            /************************************
-             * else { //duplicated call
-             * //do nothing
-             * }
-             *************************************/
-        }
-
-        return true;
-    }
-
-    private boolean instantTryLockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext, DatasetLockInfo dLockInfo) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int waiterObjId;
-        int eLockInfo = -1;
-        int entityInfo;
-        boolean isUpgrade = false;
-
-        dLockInfo = datasetResourceHT.get(datasetId);
-        eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
-        if (eLockInfo != -1) {
-            //check duplicated call
-
-            //1. lock request causing duplicated upgrading requests from different threads in a same job
-            waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
-            if (waiterObjId != -1) {
-                return false;
-            }
-
-            //2. lock request causing duplicated waiting requests from different threads in a same job
-            waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
-            if (waiterObjId != -1) {
-                return false;
-            }
-
-            //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
-            entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
-            if (entityInfo != -1) {//duplicated call or upgrader
-
-                isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
-                if (isUpgrade) {//upgrade call
-                    //wait if any upgrader exists or upgrading lock mode is not compatible
-                    if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
-                            || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
-                        return false;
-                    }
-                }
-                /***************************
-                 * else {//duplicated call
-                 * //do nothing
-                 * }
-                 ****************************/
-            } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
-                if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
-                        || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
-                        || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
-                    return false;
-                }
-            }
-        }
-        /*******************************
-         * else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
-         * //do nothing
-         * }
-         *********************************/
-
-        return true;
-    }
-
-    private boolean internalTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext, boolean isInstant) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int dId = datasetId.getId(); //int-type datasetId
-        int entityInfo;
-        int eLockInfo = -1;
-        DatasetLockInfo dLockInfo = null;
-        JobInfo jobInfo;
-        byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-        boolean isSuccess = true;
-        boolean doEscalate = false;
-
-        latchLockTable();
-        try {
-            validateJob(txnContext);
-
-            if (IS_DEBUG_MODE) {
-                trackLockRequest("Requested", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
-                        dLockInfo, eLockInfo);
-            }
-
-            dLockInfo = datasetResourceHT.get(datasetId);
-            jobInfo = jobHT.get(jobId);
-
-            if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                if (!isInstant && datasetLockMode == LockMode.IS && jobInfo != null && dLockInfo != null) {
-                    int upgradeStatus = needEscalateFromEntityToDataset(jobInfo, dId, lockMode);
-                    switch (upgradeStatus) {
-                        case DO_ESCALATE:
-                            entityHashValue = -1;
-                            doEscalate = true;
-                            break;
-
-                        case ESCALATED:
-                            return true;
-
-                        default:
-                            break;
-                    }
-                }
-            }
-
-            //#. if the datasetLockInfo doesn't exist in datasetResourceHT
-            if (dLockInfo == null || dLockInfo.isNoHolder()) {
-                if (dLockInfo == null) {
-                    dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
-                    datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
-                }
-                entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
-
-                //if dataset-granule lock
-                if (entityHashValue == -1) { //-1 stands for dataset-granule
-                    entityInfoManager.increaseDatasetLockCount(entityInfo);
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                    dLockInfo.addHolder(entityInfo);
-                } else {
-                    entityInfoManager.increaseDatasetLockCount(entityInfo);
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                    //add entityLockInfo
-                    eLockInfo = entityLockInfoManager.allocate();
-                    dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
-                    entityInfoManager.increaseEntityLockCount(entityInfo);
-                    entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
-                    entityLockInfoManager.addHolder(eLockInfo, entityInfo);
-                }
-
-                if (jobInfo == null) {
-                    jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
-                    jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
-                }
-                jobInfo.addHoldingResource(entityInfo);
-
-                if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                    if (!isInstant && datasetLockMode == LockMode.IS) {
-                        jobInfo.increaseDatasetISLockCount(dId);
-                        if (doEscalate) {
-                            //This exception is thrown when the threshold value is set to 1.
-                            //We don't want to allow the lock escalation when there is a first lock request on a dataset.
-                            throw new IllegalStateException(
-                                    "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + txnSubsystem.getTransactionProperties()
-                                                    .getEntityToDatasetLockEscalationThreshold());
-                        }
-                    }
-                }
-
-                if (IS_DEBUG_MODE) {
-                    trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
-                            dLockInfo, eLockInfo);
-                }
-
-                return true;
-            }
-
-            //#. the datasetLockInfo exists in datasetResourceHT.
-            //1. handle dataset-granule lock
-            tryLockDatasetGranuleRevertOperation = 0;
-            entityInfo = tryLockDatasetGranule(datasetId, entityHashValue, lockMode, txnContext);
-            if (entityInfo == -2) {//-2 represents fail
-                isSuccess = false;
-            } else {
-                //2. handle entity-granule lock
-                if (entityHashValue != -1) {
-                    isSuccess = tryLockEntityGranule(datasetId, entityHashValue, lockMode, entityInfo, txnContext);
-                    if (!isSuccess) {
-                        revertTryLockDatasetGranuleOperation(datasetId, entityHashValue, lockMode, entityInfo,
-                                txnContext);
-                    }
-                }
-            }
-
-            if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
-                if (!isInstant) {
-                    if (doEscalate) {
-                        //jobInfo must not be null.
-                        assert jobInfo != null;
-                        jobInfo.increaseDatasetISLockCount(dId);
-                        //release pre-acquired locks
-                        releaseDatasetISLocks(jobInfo, jobId, datasetId, txnContext);
-                    } else if (datasetLockMode == LockMode.IS) {
-                        if (jobInfo == null) {
-                            jobInfo = jobHT.get(jobId);
-                            //jobInfo must not be null;
-                            assert jobInfo != null;
-                        }
-                        jobInfo.increaseDatasetISLockCount(dId);
-                    }
-                }
-            }
-
-            if (IS_DEBUG_MODE) {
-                if (isSuccess) {
-                    trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
-                            dLockInfo, eLockInfo);
-                } else {
-                    trackLockRequest("Failed", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
-                            dLockInfo, eLockInfo);
-                }
-            }
-
-        } finally {
-            unlatchLockTable();
-        }
-
-        return isSuccess;
-    }
-
-    private void trackLockRequest(String msg, int requestType, DatasetId datasetIdObj, int entityHashValue,
-            byte lockMode, ITransactionContext txnContext, DatasetLockInfo dLockInfo, int eLockInfo) {
-        StringBuilder s = new StringBuilder();
-        LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, datasetIdObj,
-                entityHashValue, lockMode, txnContext);
-        s.append(Thread.currentThread().getId() + ":");
-        s.append(msg);
-        if (msg.equals("Granted")) {
-            if (dLockInfo != null) {
-                s.append("\t|D| ");
-                s.append(dLockInfo.getIXCount()).append(",");
-                s.append(dLockInfo.getISCount()).append(",");
-                s.append(dLockInfo.getXCount()).append(",");
-                s.append(dLockInfo.getSCount()).append(",");
-                if (dLockInfo.getFirstUpgrader() != -1) {
-                    s.append("+");
-                } else {
-                    s.append("-");
-                }
-                s.append(",");
-                if (dLockInfo.getFirstWaiter() != -1) {
-                    s.append("+");
-                } else {
-                    s.append("-");
-                }
-            }
-
-            if (eLockInfo != -1) {
-                s.append("\t|E| ");
-                s.append(entityLockInfoManager.getXCount(eLockInfo)).append(",");
-                s.append(entityLockInfoManager.getSCount(eLockInfo)).append(",");
-                if (entityLockInfoManager.getUpgrader(eLockInfo) != -1) {
-                    s.append("+");
-                } else {
-                    s.append("-");
-                }
-                s.append(",");
-                if (entityLockInfoManager.getFirstWaiter(eLockInfo) != -1) {
-                    s.append("+");
-                } else {
-                    s.append("-");
-                }
-            }
-        }
-
-        lockRequestTracker.addEvent(s.toString(), request);
-        if (msg.equals("Requested")) {
-            lockRequestTracker.addRequest(request);
-        }
-        System.out.println(request.prettyPrint() + "--> " + s.toString());
-    }
-
-    public String getHistoryForAllJobs() {
-        if (IS_DEBUG_MODE) {
-            return lockRequestTracker.getHistoryForAllJobs();
-        }
-        return null;
-    }
-
-    public String getHistoryPerJob() {
-        if (IS_DEBUG_MODE) {
-            return lockRequestTracker.getHistoryPerJob();
-        }
-        return null;
-    }
-
-    public String getRequestHistoryForAllJobs() {
-        if (IS_DEBUG_MODE) {
-            return lockRequestTracker.getRequestHistoryForAllJobs();
-        }
-        return null;
-    }
-
-    private void revertTryLockDatasetGranuleOperation(DatasetId datasetId, int entityHashValue, byte lockMode,
-            int entityInfo, ITransactionContext txnContext) {
-        JobId jobId = txnContext.getJobId();
-        DatasetLockInfo dLockInfo;
-        JobInfo jobInfo;
-        int lockCount;
-        byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-
-        dLockInfo = datasetResourceHT.get(datasetId);
-        jobInfo = jobHT.get(jobId);
-
-        //see tryLockDatasetGranule() function to know the revert operation
-        switch (tryLockDatasetGranuleRevertOperation) {
-
-            case 1://[revertOperation1]: reverting 'adding a holder'
-
-                if (entityHashValue == -1) {
-                    dLockInfo.decreaseLockCount(datasetLockMode);
-                    dLockInfo.removeHolder(entityInfo, jobInfo); //--> this call removes entityInfo from JobInfo's holding-resource-list as well.
-                } else {
-                    dLockInfo.decreaseLockCount(datasetLockMode);
-                    jobInfo.removeHoldingResource(entityInfo);
-                }
-                entityInfoManager.decreaseDatasetLockCount(entityInfo);
-                if (jobInfo.getLastHoldingResource() == -1 && jobInfo.getFirstWaitingResource() == -1) {
-                    jobHT.remove(jobId);
-                }
-                entityInfoManager.deallocate(entityInfo);
-                break;
-
-            case 2://[revertOperation2]: reverting 'adding an upgrader'
-                lockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-                if (entityHashValue == -1) { //dataset-granule lock
-                    dLockInfo.decreaseLockCount(LockMode.X, lockCount);
-                    dLockInfo.increaseLockCount(LockMode.S, lockCount);
-                } else {
-                    dLockInfo.decreaseLockCount(LockMode.IX, lockCount);
-                    dLockInfo.increaseLockCount(LockMode.IS, lockCount);
-                }
-                entityInfoManager.setDatasetLockMode(entityInfo, LockMode.S);
-                break;
-
-            case 3://[revertOperation3]: reverting 'adding a duplicated call'
-                entityInfoManager.decreaseDatasetLockCount(entityInfo);
-                datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-                if (entityHashValue == -1) { //dataset-granule
-                    dLockInfo.decreaseLockCount(datasetLockMode);
-                } else { //entity-granule
-                    datasetLockMode = datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-                    dLockInfo.decreaseLockCount(datasetLockMode);
-                }
-
-                break;
-            default:
-                //do nothing;
-        }
-    }
-
-    private int tryLockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int dId = datasetId.getId(); //int-type datasetId
-        int waiterObjId;
-        int entityInfo = -1;
-        DatasetLockInfo dLockInfo;
-        JobInfo jobInfo;
-        boolean isUpgrade = false;
-        int weakerModeLockCount;
-        byte datasetLockMode = entityHashValue == -1 ? lockMode : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-
-        dLockInfo = datasetResourceHT.get(datasetId);
-        jobInfo = jobHT.get(jobId);
-
-        //check duplicated call
-
-        //1. lock request causing duplicated upgrading requests from different threads in a same job
-        waiterObjId = dLockInfo.findUpgraderFromUpgraderList(jId, entityHashValue);
-        if (waiterObjId != -1) {
-            return -2;
-        }
-
-        //2. lock request causing duplicated waiting requests from different threads in a same job
-        waiterObjId = dLockInfo.findWaiterFromWaiterList(jId, entityHashValue);
-        if (waiterObjId != -1) {
-            return -2;
-        }
-
-        //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
-        entityInfo = dLockInfo.findEntityInfoFromHolderList(jId, entityHashValue);
-        if (entityInfo == -1) { //new call from this job -> doesn't mean that eLockInfo doesn't exist since another thread might have create the eLockInfo already.
-
-            //////////////////////////////////////////////////////////////////////////////////////
-            //[part of revertOperation1]
-            entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
-            if (jobInfo == null) {
-                jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
-                jobHT.put(jobId, jobInfo);
-            }
-            //////////////////////////////////////////////////////////////////////////////////////
-
-            //return fail if any upgrader exists or upgrading lock mode is not compatible
-            if (dLockInfo.getFirstUpgrader() != -1 || dLockInfo.getFirstWaiter() != -1
-                    || !dLockInfo.isCompatible(datasetLockMode)) {
-
-                if (ALLOW_DATASET_GRANULE_X_LOCK_WITH_OTHER_CONCURRENT_LOCK_REQUESTS) {
-                    //The following case only may occur when the dataset level X lock is requested
-                    //with the other lock
-
-                    //[Notice]
-                    //There has been no same caller as (jId, dId, entityHashValue) triplet.
-                    //But there could be the same caller in terms of (jId, dId) pair.
-                    //For example,
-                    //1) (J1, D1, E1) acquires IS in Dataset D1
-                    //2) (J2, D1, -1) requests X  in Dataset D1, but waits
-                    //3) (J1, D1, E2) requests IS in Dataset D1, but should wait
-                    //The 3) may cause deadlock if 1) and 3) are under the same thread.
-                    //Even if (J1, D1, E1) and (J1, D1, E2) are two different thread, instead of
-                    //aborting (J1, D1, E1) triggered by the deadlock, we give higher priority to 3) than 2)
-                    //as long as the dataset level lock D1 is being held by the same jobId.
-                    //The above consideration is covered in the following code.
-                    //find the same dataset-granule lock request, that is, (J1, D1) pair in the above example.
-                    if (jobInfo.isDatasetLockGranted(dId, LockMode.IS)) {
-                        if (dLockInfo.isCompatible(datasetLockMode)) {
-                            //this is duplicated call
-                            entityInfoManager.increaseDatasetLockCount(entityInfo);
-                            if (entityHashValue == -1) {
-                                dLockInfo.increaseLockCount(datasetLockMode);
-                                dLockInfo.addHolder(entityInfo);
-                            } else {
-                                dLockInfo.increaseLockCount(datasetLockMode);
-                                //IS and IX holders are implicitly handled.
-                            }
-                            //add entityInfo to JobInfo's holding-resource list
-                            jobInfo.addHoldingResource(entityInfo);
-
-                            tryLockDatasetGranuleRevertOperation = 1;
-
-                            return entityInfo;
-                        }
-                    }
-                }
-
-                //revert [part of revertOperation1] before return
-                if (jobInfo.getLastHoldingResource() == -1 && jobInfo.getFirstWaitingResource() == -1) {
-                    jobHT.remove(jobId);
-                }
-                entityInfoManager.deallocate(entityInfo);
-
-                return -2;
-            }
-
-            //////////////////////////////////////////////////////////////////////////////////////
-            //revert the following operations if the caller thread has to wait during this call.
-            //[revertOperation1]
-            entityInfoManager.increaseDatasetLockCount(entityInfo);
-            if (entityHashValue == -1) {
-                dLockInfo.increaseLockCount(datasetLockMode);
-                dLockInfo.addHolder(entityInfo);
-            } else {
-                dLockInfo.increaseLockCount(datasetLockMode);
-                //IS and IX holders are implicitly handled.
-            }
-            //add entityInfo to JobInfo's holding-resource list
-            jobInfo.addHoldingResource(entityInfo);
-
-            //set revert operation to be reverted when tryLock() fails
-            tryLockDatasetGranuleRevertOperation = 1;
-            //////////////////////////////////////////////////////////////////////////////////////
-
-        } else {
-            isUpgrade = isLockUpgrade(entityInfoManager.getDatasetLockMode(entityInfo), lockMode);
-            if (isUpgrade) { //upgrade call
-                //return fail if any upgrader exists or upgrading lock mode is not compatible
-                if (dLockInfo.getFirstUpgrader() != -1 || !dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)) {
-                    return -2;
-                }
-
-                //update entityInfo's dataset lock count and datasetLockInfo's lock count
-                weakerModeLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
-
-                //////////////////////////////////////////////////////////////////////////////////////
-                //revert the following operations if the caller thread has to wait during this call.
-                //[revertOperation2]
-                entityInfoManager.setDatasetLockMode(entityInfo, lockMode);
-
-                if (entityHashValue == -1) { //dataset-granule lock
-                    dLockInfo.increaseLockCount(LockMode.X, weakerModeLockCount);//new lock mode
-                    dLockInfo.decreaseLockCount(LockMode.S, weakerModeLockCount);//current lock mode
-                } else {
-                    dLockInfo.increaseLockCount(LockMode.IX, weakerModeLockCount);
-                    dLockInfo.decreaseLockCount(LockMode.IS, weakerModeLockCount);
-                }
-                tryLockDatasetGranuleRevertOperation = 2;
-                //////////////////////////////////////////////////////////////////////////////////////
-
-            } else { //duplicated call
-
-                //////////////////////////////////////////////////////////////////////////////////////
-                //revert the following operations if the caller thread has to wait during this call.
-                //[revertOperation3]
-                entityInfoManager.increaseDatasetLockCount(entityInfo);
-                datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-
-                if (entityHashValue == -1) { //dataset-granule
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                } else { //entity-granule
-                    datasetLockMode = datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-                    dLockInfo.increaseLockCount(datasetLockMode);
-                }
-
-                tryLockDatasetGranuleRevertOperation = 3;
-                //////////////////////////////////////////////////////////////////////////////////////
-
-            }
-        }
-
-        return entityInfo;
-    }
-
-    private boolean tryLockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
-            int entityInfoFromDLockInfo, ITransactionContext txnContext) throws ACIDException {
-        JobId jobId = txnContext.getJobId();
-        int jId = jobId.getId(); //int-type jobId
-        int waiterObjId;
-        int eLockInfo = -1;
-        int entityInfo;
-        DatasetLockInfo dLockInfo;
-        boolean isUpgrade = false;
-        int weakerModeLockCount;
-
-        dLockInfo = datasetResourceHT.get(datasetId);
-        eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
-
-        if (eLockInfo != -1) {
-            //check duplicated call
-
-            //1. lock request causing duplicated upgrading requests from different threads in a same job
-            waiterObjId = entityLockInfoManager.findUpgraderFromUpgraderList(eLockInfo, jId, entityHashValue);
-            if (waiterObjId != -1) {
-                return false;
-            }
-
-            //2. lock request causing duplicated waiting requests from different threads in a same job
-            waiterObjId = entityLockInfoManager.findWaiterFromWaiterList(eLockInfo, jId, entityHashValue);
-            if (waiterObjId != -1) {
-                return false;
-            }
-
-            //3. lock request causing duplicated holding requests from different threads or a single thread in a same job
-            entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jId, entityHashValue);
-            if (entityInfo != -1) {//duplicated call or upgrader
-
-                isUpgrade = isLockUpgrade(entityInfoManager.getEntityLockMode(entityInfo), lockMode);
-                if (isUpgrade) {//upgrade call
-                    //wait if any upgrader exists or upgrading lock mode is not compatible
-                    if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
-                            || !entityLockInfoManager.isUpgradeCompatible(eLockInfo, lockMode, entityInfo)) {
-                        return false;
-                    }
-
-                    weakerModeLockCount = entityInfoManager.getEntityLockCount(entityInfo);
-                    entityInfoManager.setEntityLockMode(entityInfo, lockMode);
-
-                    entityLockInfoManager.increaseLockCount(eLockInfo, LockMode.X, (short) weakerModeLockCount);//new lock mode
-                    entityLockInfoManager.decreaseLockCount(eLockInfo, LockMode.S, (short) weakerModeLockCount);//old lock mode
-
-                } else {//duplicated call
-                    entityInfoManager.increaseEntityLockCount(entityInfo);
-                    entityLockInfoManager.increaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
-                }
-            } else {//new call from this job, but still eLockInfo exists since other threads hold it or wait on it
-                entityInfo = entityInfoFromDLockInfo;
-                if (entityLockInfoManager.getUpgrader(eLockInfo) != -1
-                        || entityLockInfoManager.getFirstWaiter(eLockInfo) != -1
-                        || !entityLockInfoManager.isCompatible(eLockInfo, lockMode)) {
-                    return false;
-                }
-
-                entityInfoManager.increaseEntityLockCount(entityInfo);
-                entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
-                entityLockInfoManager.addHolder(eLockInfo, entityInfo);
-            }
-        } else {//eLockInfo doesn't exist, so this lock request is the first request and can be granted without waiting.
-            eLockInfo = entityLockInfoManager.allocate();
-            dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
-            entityInfoManager.increaseEntityLockCount(entityInfoFromDLockInfo);
-            entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
-            entityLockInfoManager.addHolder(eLockInfo, entityInfoFromDLockInfo);
-        }
-
-        return true;
-    }
-
-    private void latchLockTable() {
-        lockTableLatch.writeLock().lock();
-    }
-
-    private void unlatchLockTable() {
-        lockTableLatch.writeLock().unlock();
-    }
-
-    private void latchWaitNotify() {
-        waiterLatch.writeLock().lock();
-    }
-
-    private void unlatchWaitNotify() {
-        waiterLatch.writeLock().unlock();
-    }
-
-    private int handleLockWaiter(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isUpgrade,
-            boolean isDatasetLockInfo, ITransactionContext txnContext, JobInfo jobInfo, int duplicatedWaiterObjId)
-            throws ACIDException {
-        int waiterId = -1;
-        LockWaiter waiter;
-        int waiterCount = 0;
-        boolean isInterruptedExceptionOccurred = false;
-
-        if (duplicatedWaiterObjId != -1
-                || isDeadlockFree(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade)) {//deadlock free -> wait
-            if (duplicatedWaiterObjId == -1) {
-                waiterId = lockWaiterManager.allocate(); //initial value of waiterObj: wait = true, victim = false
-                waiter = lockWaiterManager.getLockWaiter(waiterId);
-                waiter.setEntityInfoSlot(entityInfo);
-                jobInfo.addWaitingResource(waiterId);
-                waiter.setBeginWaitTime(System.currentTimeMillis());
-            } else {
-                waiterId = duplicatedWaiterObjId;
-                waiter = lockWaiterManager.getLockWaiter(waiterId);
-            }
-
-            if (duplicatedWaiterObjId == -1) {
-                //add actor properly
-                if (isDatasetLockInfo) {
-                    waiter.setWaitingOnEntityLock(false);
-                    if (isUpgrade) {
-                        dLockInfo.addUpgrader(waiterId);
-                        waiter.setWaiter(false);
-                    } else {
-                        dLockInfo.addWaiter(waiterId);
-                        waiter.setWaiter(true);
-                    }
-                } else {
-                    waiter.setWaitingOnEntityLock(true);
-                    if (isUpgrade) {
-                        waiter.setWaiter(false);
-                        entityLockInfoManager.addUpgrader(eLockInfo, waiterId);
-                    } else {
-                        waiter.setWaiter(true);
-                        entityLockInfoManager.addWaiter(eLockInfo, waiterId);
-                    }
-                }
-            }
-            waiter.increaseWaiterCount();
-            waiter.setFirstGetUp(true);
-
-            latchWaitNotify();
-            unlatchLockTable();
-            try {
-                synchronized (waiter) {
-                    unlatchWaitNotify();
-                    while (waiter.needWait()) {
-                        try {
-                            if (IS_DEBUG_MODE) {
-                                System.out.println("" + Thread.currentThread().getName() + "\twaits("
-                                        + waiter.getWaiterCount() + "): WID(" + waiterId + "),EID("
-                                        + waiter.getEntityInfoSlot() + ")");
-                            }
-                            waiter.wait();
-                        } catch (InterruptedException e) {
-                            //TODO figure-out what is the appropriate way to handle this exception
-                            e.printStackTrace();
-                            isInterruptedExceptionOccurred = true;
-                            waiter.setWait(false);
-                        }
-                    }
-                }
-
-                if (isInterruptedExceptionOccurred) {
-                    throw new ACIDException("InterruptedException is caught");
-                }
-            } catch (Exception e) {
-                throw new LockMgrLatchHandlerException(e);
-            }
-
-            //waiter woke up -> remove/deallocate waiter object and abort if timeout
-            latchLockTable();
-
-            if (txnContext.isTimeout() || waiter.isVictim()) {
-                requestAbort(txnContext);
-            }
-
-            if (waiter.isFirstGetUp()) {
-                waiter.setFirstGetUp(false);
-                waiterCount = waiter.getWaiterCount();
-            } else {
-                waiterCount = 0;
-            }
-
-            waiter.decreaseWaiterCount();
-            if (IS_DEBUG_MODE) {
-                System.out.println("" + Thread.currentThread().getName() + "\tgot-up!(" + waiter.getWaiterCount()
-                        + "): WID(" + waiterId + "),EID(" + waiter.getEntityInfoSlot() + ")");
-            }
-            if (waiter.getWaiterCount() == 0) {
-                //remove actor properly
-                if (isDatasetLockInfo) {
-                    if (isUpgrade) {
-                        dLockInfo.removeUpgrader(waiterId);
-                    } else {
-                        dLockInfo.removeWaiter(waiterId);
-                    }
-                } else {
-                    if (isUpgrade) {
-                        entityLockInfoManager.removeUpgrader(eLockInfo, waiterId);
-                    } else {
-                        entityLockInfoManager.removeWaiter(eLockInfo, waiterId);
-                    }
-                }
-
-                //if (!isUpgrade && isDatasetLockInfo) {
-                jobInfo.removeWaitingResource(waiterId);
-                //}
-                lockWaiterManager.deallocate(waiterId);
-            }
-
-        } else { //deadlock -> abort
-            //[Notice]
-            //Before requesting abort, the entityInfo for waiting datasetLock request is deallocated.
-            if (!isUpgrade && isDatasetLockInfo) {
-                //deallocate the entityInfo
-                entityInfoManager.deallocate(entityInfo);
-            }
-            requestAbort(txnContext);
-        }
-
-        return waiterCount;
-    }
-
-    private boolean isDeadlockFree(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo,
-            boolean isUpgrade) {
-        return deadlockDetector.isSafeToAdd(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade);
-    }
-
-    private void requestAbort(ITransactionContext txnContext) throws ACIDException {
-        txnContext.setTimeout(true);
-        throw new ACIDException("Transaction " + txnContext.getJobId()
-                + " should abort (requested by the Lock Manager)");
-    }
-
-    /**
-     * For now, upgrading lock granule from entity-granule to dataset-granule is not supported!!
-     *
-     * @param fromLockMode
-     * @param toLockMode
-     * @return
-     */
-    private boolean isLockUpgrade(byte fromLockMode, byte toLockMode) {
-        return fromLockMode == LockMode.S && toLockMode == LockMode.X;
-    }
-
-    /**
-     * wake up upgraders first, then waiters.
-     * Criteria to wake up upgraders: if the upgrading lock mode is compatible, then wake up the upgrader.
-     */
-    private void wakeUpDatasetLockWaiters(DatasetLockInfo dLockInfo) {
-        int waiterObjId = dLockInfo.getFirstUpgrader();
-        int entityInfo;
-        LockWaiter waiterObj;
-        byte datasetLockMode;
-        byte lockMode;
-        boolean areAllUpgradersAwaken = true;
-
-        consecutiveWakeupContext.reset();
-        while (waiterObjId != -1) {
-            //wake up upgraders
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? LockMode.X : LockMode.IX;
-            if (dLockInfo.isUpgradeCompatible(datasetLockMode, entityInfo)
-                    && consecutiveWakeupContext.isCompatible(datasetLockMode)) {
-                consecutiveWakeupContext.setLockMode(datasetLockMode);
-                //compatible upgrader is waken up
-                latchWaitNotify();
-                synchronized (waiterObj) {
-                    unlatchWaitNotify();
-                    waiterObj.setWait(false);
-                    if (IS_DEBUG_MODE) {
-                        System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID(" + waiterObjId
-                                + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
-                    }
-                    waiterObj.notifyAll();
-                }
-                waiterObjId = waiterObj.getNextWaiterObjId();
-            } else {
-                areAllUpgradersAwaken = false;
-                break;
-            }
-        }
-
-        if (areAllUpgradersAwaken) {
-            //wake up waiters
-            waiterObjId = dLockInfo.getFirstWaiter();
-            while (waiterObjId != -1) {
-                waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-                entityInfo = waiterObj.getEntityInfoSlot();
-                lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
-                datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? lockMode
-                        : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
-                if (dLockInfo.isCompatible(datasetLockMode) && consecutiveWakeupContext.isCompatible(datasetLockMode)) {
-                    consecutiveWakeupContext.setLockMode(datasetLockMode);
-                    //compatible waiter is waken up
-                    latchWaitNotify();
-                    synchronized (waiterObj) {
-                        unlatchWaitNotify();
-                        waiterObj.setWait(false);
-                        if (IS_DEBUG_MODE) {
-                            System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID("
-                                    + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
-                        }
-                        waiterObj.notifyAll();
-                    }
-                    waiterObjId = waiterObj.getNextWaiterObjId();
-                } else {
-                    break;
-                }
-            }
-        }
-    }
-
-    private void wakeUpEntityLockWaiters(int eLockInfo) {
-        boolean areAllUpgradersAwaken = true;
-        int waiterObjId = entityLockInfoManager.getUpgrader(eLockInfo);
-        int entityInfo;
-        LockWaiter waiterObj;
-        byte entityLockMode;
-
-        consecutiveWakeupContext.reset();
-        while (waiterObjId != -1) {
-            //wake up upgraders
-            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-            entityInfo = waiterObj.getEntityInfoSlot();
-            if (entityLockInfoManager.isUpgradeCompatible(eLockInfo, LockMode.X, entityInfo)
-                    && consecutiveWakeupContext.isCompatible(LockMode.X)) {
-                consecutiveWakeupContext.setLockMode(LockMode.X);
-                latchWaitNotify();
-                synchronized (waiterObj) {
-                    unlatchWaitNotify();
-                    waiterObj.setWait(false);
-                    if (IS_DEBUG_MODE) {
-                        System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID(" + waiterObjId
-                                + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
-                    }
-                    waiterObj.notifyAll();
-                }
-                waiterObjId = waiterObj.getNextWaiterObjId();
-            } else {
-                areAllUpgradersAwaken = false;
-                break;
-            }
-        }
-
-        if (areAllUpgradersAwaken) {
-            //wake up waiters
-            waiterObjId = entityLockInfoManager.getFirstWaiter(eLockInfo);
-            while (waiterObjId != -1) {
-                waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-                entityInfo = waiterObj.getEntityInfoSlot();
-                entityLockMode = entityInfoManager.getEntityLockMode(entityInfo);
-                if (entityLockInfoManager.isCompatible(eLockInfo, entityLockMode)
-                        && consecutiveWakeupContext.isCompatible(entityLockMode)) {
-                    consecutiveWakeupContext.setLockMode(entityLockMode);
-                    //compatible waiter is waken up
-                    latchWaitNotify();
-                    synchronized (waiterObj) {
-                        unlatchWaitNotify();
-                        waiterObj.setWait(false);
-                        if (IS_DEBUG_MODE) {
-                            System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID("
-                                    + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
-                        }
-                        waiterObj.notifyAll();
-                    }
-                } else {
-                    break;
-                }
-                waiterObjId = waiterObj.getNextWaiterObjId();
-            }
-        }
-    }
-
-    @Override
-    public String prettyPrint() throws ACIDException {
-        StringBuilder s = new StringBuilder("\n########### LockManager Status #############\n");
-        return s + "\n";
-    }
-
-    public void sweepForTimeout() throws ACIDException {
-        JobInfo jobInfo;
-        int waiterObjId;
-        LockWaiter waiterObj;
-
-        latchLockTable();
-        try {
-
-            Iterator<Entry<JobId, JobInfo>> iter = jobHT.entrySet().iterator();
-            while (iter.hasNext()) {
-                Map.Entry<JobId, JobInfo> pair = iter.next();
-                jobInfo = pair.getValue();
-                waiterObjId = jobInfo.getFirstWaitingResource();
-                while (waiterObjId != -1) {
-                    waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
-                    toutDetector.checkAndSetVictim(waiterObj);
-                    waiterObjId = waiterObj.getNextWaiterObjId();
-                }
-            }
-        } finally {
-            unlatchLockTable();
-        }
-    }
-
-    @Override
-    public void start() {
-        //no op
-    }
-
-    @Override
-    public void stop(boolean dumpState, OutputStream os) {
-        if (dumpState) {
-            dumpState(os);
-        }
-    }
-
-    @Override
-    public void dumpState(OutputStream os) {
-        //#. dump Configurable Variables
-        dumpConfVars(os);
-
-        //#. dump jobHT
-        dumpJobInfo(os);
-
-        //#. dump datasetResourceHT
-        dumpDatasetLockInfo(os);
-
-        //#. dump entityLockInfoManager
-        dumpEntityLockInfo(os);
-
-        //#. dump entityInfoManager
-        dumpEntityInfo(os);
-
-        //#. dump lockWaiterManager
-        dumpLockWaiterInfo(os);
-    }
-
-    private void dumpConfVars(OutputStream os) {
-        try {
-            StringBuilder sb = new StringBuilder();
-            sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
-            sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: "
-                    + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
-            sb.append("\nSHRINK_TIMER_THRESHOLD (entityLockInfoManager): "
-                    + entityLockInfoManager.getShrinkTimerThreshold());
-            sb.append("\nSHRINK_TIMER_THRESHOLD (entityInfoManager): " + entityInfoManager.getShrinkTimerThreshold());
-            sb.append("\nSHRINK_TIMER_THRESHOLD (lockWaiterManager): " + lockWaiterManager.getShrinkTimerThreshold());
-            sb.append("\n>>dump_end\t>>----- [ConfVars] -----\n");
-            os.write(sb.toString().getBytes());
-        } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
-            if (IS_DEBUG_MODE) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    private void dumpJobInfo(OutputStream os) {
-        JobId jobId;
-        JobInfo jobInfo;
-        StringBuilder sb = new StringBuilder();
-
-        try {
-            sb.append("\n>>dump_begin\t>>----- [JobInfo] -----");
-            Set<Map.Entry<JobId, JobInfo>> entrySet = jobHT.entrySet();
-            if (entrySet != null) {
-                for (Map.Entry<JobId, JobInfo> entry : entrySet) {
-                    if (entry != null) {
-                        jobId = entry.getKey();
-                        if (jobId != null) {
-                            sb.append("\n" + jobId);
-                        } else {
-                            sb.append("\nJID:null");
-                        }
-
-                        jobInfo = entry.getValue();
-                        if (jobInfo != null) {
-                            sb.append(jobInfo.coreDump());
-                        } else {
-                            sb.append("\nJobInfo:null");
-                        }
-                    }
-                }
-            }
-            sb.append("\n>>dump_end\t>>----- [JobInfo] -----\n");
-            os.write(sb.toString().getBytes());
-        } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
-            if (IS_DEBUG_MODE) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    private void dumpDatasetLockInfo(OutputStream os) {
-        DatasetId datasetId;
-        DatasetLockInfo datasetLockInfo;
-        StringBuilder sb = new StringBuilder();
-
-        try {
-            sb.append("\n>>dump_begin\t>>----- [DatasetLockInfo] -----");
-            Set<Map.Entry<DatasetId, DatasetLockInfo>> entrySet = datasetResourceHT.entrySet();
-            if (entrySet != null) {
-                for (Map.Entry<DatasetId, DatasetLockInfo> entry : entrySet) {
-                    if (entry != null) {
-                        datasetId = entry.getKey();
-                        if (datasetId != null) {
-                            sb.append("\nDatasetId:" + datasetId.getId());
-                        } else {
-                            sb.append("\nDatasetId:null");
-                        }
-
-                        datasetLockInfo = entry.getValue();
-                        if (datasetLockInfo != null) {
-                            sb.append(datasetLockInfo.coreDump());
-                        } else {
-                            sb.append("\nDatasetLockInfo:null");
-                        }
-                    }
-                    sb.append("\n>>dump_end\t>>----- [DatasetLockInfo] -----\n");
-                    os.write(sb.toString().getBytes());
-
-                    //create a new sb to avoid possible OOM exception
-                    sb = new StringBuilder();
-                }
-            }
-        } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
-            if (IS_DEBUG_MODE) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    private void dumpEntityLockInfo(OutputStream os) {
-        StringBuilder sb = new StringBuilder();
-        try {
-            sb.append("\n>>dump_begin\t>>----- [EntityLockInfo] -----");
-            entityLockInfoManager.coreDump(os);
-            sb.append("\n>>dump_end\t>>----- [EntityLockInfo] -----\n");
-            os.write(sb.toString().getBytes());
-        } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
-            if (IS_DEBUG_MODE) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    private void dumpEntityInfo(OutputStream os) {
-        StringBuilder sb = new StringBuilder();
-        try {
-            sb.append("\n>>dump_begin\t>>----- [EntityInfo] -----");
-            entityInfoManager.coreDump(os);
-            sb.append("\n>>dump_end\t>>----- [EntityInfo] -----\n");
-            os.write(sb.toString().getBytes());
-        } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
-            if (IS_DEBUG_MODE) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    private void dumpLockWaiterInfo(OutputStream os) {
-        StringBuilder sb = new StringBuilder();
-        try {
-            sb.append("\n>>dump_begin\t>>----- [LockWaiterInfo] -----");
-            lockWaiterManager.coreDump(os);
-            sb.append("\n>>dump_end\t>>----- [LockWaiterInfo] -----\n");
-            os.write(sb.toString().getBytes());
-        } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
-            if (IS_DEBUG_MODE) {
-                e.printStackTrace();
-            }
-        }
-    }
-
-    public void batchUnlock(LogBuffer logPage, LogBufferTailReader logBufferTailReader) throws ACIDException {
-        latchLockTable();
-        try {
-            ITransactionContext txnCtx = null;
-            LogRecord logRecord = logBufferTailReader.next();
-            while (logRecord != null) {
-                if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
-                    tempDatasetIdObj.setId(logRecord.getDatasetId());
-                    tempJobIdObj.setId(logRecord.getJobId());
-                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
-                    unlock(tempDatasetIdObj, logRecord.getPKHashValue(), LockMode.ANY, txnCtx);
-                    txnCtx.notifyOptracker(false);
-                } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
-                    tempJobIdObj.setId(logRecord.getJobId());
-                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
-                    txnCtx.notifyOptracker(true);
-                    logPage.notifyJobTerminator();
-                }
-                logRecord = logBufferTailReader.next();
-            }
-        } finally {
-            unlatchLockTable();
-        }
-    }
-}
-
-class ConsecutiveWakeupContext {
-    private boolean IS;
-    private boolean IX;
-    private boolean S;
-    private boolean X;
-
-    public void reset() {
-        IS = false;
-        IX = false;
-        S = false;
-        X = false;
-    }
-
-    public boolean isCompatible(byte lockMode) {
-        switch (lockMode) {
-            case LockMode.IX:
-                return !S && !X;
-
-            case LockMode.IS:
-                return !X;
-
-            case LockMode.X:
-                return !IS && !IX && !S && !X;
-
-            case LockMode.S:
-                return !IX && !X;
-
-            default:
-                throw new IllegalStateException("Invalid upgrade lock mode");
-        }
-    }
-
-    public void setLockMode(byte lockMode) {
-        switch (lockMode) {
-            case LockMode.IX:
-                IX = true;
-                return;
-
-            case LockMode.IS:
-                IS = true;
-                return;
-
-            case LockMode.X:
-                X = true;
-                return;
-
-            case LockMode.S:
-                S = true;
-                return;
-
-            default:
-                throw new IllegalStateException("Invalid lock mode");
-        }
-
-    }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
deleted file mode 100644
index 7ccd6b9..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
+++ /dev/null
@@ -1,664 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.NoSuchElementException;
-import java.util.Scanner;
-
-import org.apache.commons.io.FileUtils;
-
-import org.apache.asterix.common.config.AsterixPropertiesAccessor;
-import org.apache.asterix.common.config.AsterixTransactionProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.transaction.management.service.logging.LogManager;
-import org.apache.asterix.transaction.management.service.transaction.TransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class LockManagerDeterministicUnitTest {
-
-    public static void main(String args[]) throws ACIDException, IOException, AsterixException {
-        //prepare configuration file
-        File cwd = new File(System.getProperty("user.dir"));
-        File asterixdbDir = cwd.getParentFile();
-        File srcFile = new File(asterixdbDir.getAbsoluteFile(),
-                "asterix-app/src/main/resources/asterix-build-configuration.xml");
-        File destFile = new File(cwd, "target/classes/asterix-configuration.xml");
-        FileUtils.copyFile(srcFile, destFile);
-
-        //initialize controller thread
-        String requestFileName = new String(
-                "src/main/java/org.apache/asterix/transaction/management/service/locking/LockRequestFile");
-        Thread t = new Thread(new LockRequestController(requestFileName));
-        t.start();
-    }
-}
-
-class LockRequestController implements Runnable {
-
-    public static final boolean IS_DEBUG_MODE = false;
-    TransactionSubsystem txnProvider;
-    WorkerReadyQueue workerReadyQueue;
-    ArrayList<LockRequest> requestList;
-    ArrayList<ArrayList<Integer>> expectedResultList;
-    int resultListIndex;
-    ILockManager lockMgr;
-    String requestFileName;
-    long defaultWaitTime;
-
-    public LockRequestController(String requestFileName) throws ACIDException, AsterixException {
-        this.txnProvider = new TransactionSubsystem("nc1", new TestRuntimeContextProvider(), new AsterixTransactionProperties(
-                new AsterixPropertiesAccessor()));
-        this.workerReadyQueue = new WorkerReadyQueue();
-        this.requestList = new ArrayList<LockRequest>();
-        this.expectedResultList = new ArrayList<ArrayList<Integer>>();
-        this.lockMgr = txnProvider.getLockManager();
-        this.requestFileName = new String(requestFileName);
-        this.resultListIndex = 0;
-        this.defaultWaitTime = 10;
-    }
-
-    @Override
-    public void run() {
-        Thread.currentThread().setName("Thread-0");
-        HashMap<String, Thread> threadMap = new HashMap<String, Thread>();
-        Thread t = null;
-        LockRequest lockRequest = null;
-        boolean isSuccess = true;
-
-        try {
-            readRequest();
-        } catch (IOException e) {
-            e.printStackTrace();
-            System.exit(-1);
-        } catch (ACIDException e) {
-            e.printStackTrace();
-            System.exit(-1);
-        }
-
-        //initialize workerThread
-        int size = requestList.size();
-        for (int i = 0; i < size; i++) {
-            lockRequest = requestList.get(i);
-            if (lockRequest.threadName.equals("Thread-0")) {
-                //Thread-0 is controller thread.
-                continue;
-            }
-            t = threadMap.get(lockRequest.threadName);
-            if (t == null) {
-                t = new Thread(new LockRequestWorker(txnProvider, workerReadyQueue, lockRequest.threadName),
-                        lockRequest.threadName);
-                threadMap.put(lockRequest.threadName, t);
-                t.start();
-                log("Created " + lockRequest.threadName);
-            }
-        }
-
-        //wait for all workerThreads to be ready
-        try {
-            log("waiting for all workerThreads to complete initialization ...");
-            Thread.sleep(5);
-        } catch (InterruptedException e1) {
-            e1.printStackTrace();
-        }
-        while (workerReadyQueue.size() != threadMap.size()) {
-            try {
-                log(" .");
-                Thread.sleep(5);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-        }
-
-        //make workerThread work
-        while (requestList.size() != 0) {
-            lockRequest = requestList.remove(0);
-            log("Processing: " + lockRequest.prettyPrint());
-            try {
-                if (!handleRequest(lockRequest)) {
-                    log("\n*** Test Failed ***");
-                    isSuccess = false;
-                    break;
-                } else {
-                    log("Processed: " + lockRequest.prettyPrint());
-                }
-            } catch (ACIDException e) {
-                e.printStackTrace();
-                break;
-            }
-        }
-
-        if (isSuccess) {
-            log("\n*** Test Passed ***");
-        }
-        ((LogManager) txnProvider.getLogManager()).stop(false, null);
-    }
-
-    public boolean handleRequest(LockRequest request) throws ACIDException {
-        LockRequestWorker worker = null;
-        int i = 0;
-
-        if (request.requestType == RequestType.CHECK_SEQUENCE) {
-            return validateExpectedResult(true);
-        } else if (request.requestType == RequestType.CHECK_SET) {
-            return validateExpectedResult(false);
-        } else if (request.requestType == RequestType.WAIT) {
-            try {
-                Thread.sleep((long) request.entityHashValue);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-                return false;
-            }
-        } else if (request.requestType == RequestType.END) {
-            worker = workerReadyQueue.pop(request.threadName);
-            while (worker == null) {
-                if (!IS_DEBUG_MODE) {
-                    log(request.threadName + " is not in the workerReadyQueue");
-                    return false;
-                }
-                log(Thread.currentThread().getName() + " waiting for " + request.threadName
-                        + " to be in the workerReadyQueue[" + i++ + "].");
-                try {
-                    Thread.sleep((long) 10);
-                } catch (InterruptedException e) {
-                    e.printStackTrace();
-                    return false;
-                }
-                worker = workerReadyQueue.pop(request.threadName);
-            }
-            synchronized (worker) {
-                worker.setDone(true);
-                worker.setWait(false);
-                worker.notify();
-            }
-            try {
-                Thread.sleep((long) defaultWaitTime);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-        } else {
-            worker = workerReadyQueue.pop(request.threadName);
-            while (worker == null) {
-                if (!IS_DEBUG_MODE) {
-                    log(request.threadName + " is not in the workerReadyQueue");
-                    return false;
-                }
-                log(Thread.currentThread().getName() + " waiting for " + request.threadName
-                        + " to be in the workerReadyQueue[" + i++ + "].");
-                try {
-                    Thread.sleep((long) 10);
-                } catch (InterruptedException e) {
-                    e.printStackTrace();
-                }
-                worker = workerReadyQueue.pop(request.threadName);
-            }
-
-            synchronized (worker) {
-                worker.setLockRequest(request);
-                worker.setWait(false);
-                worker.notify();
-            }
-
-            try {
-                Thread.sleep((long) defaultWaitTime);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-        }
-
-        return true;
-    }
-
-    public boolean validateExpectedResult(boolean isSequence) {
-
-        if (isSequence) {
-            return workerReadyQueue.checkSequence(expectedResultList.get(resultListIndex++));
-        } else {
-            return workerReadyQueue.checkSet(expectedResultList.get(resultListIndex++));
-        }
-
-    }
-
-    public void readRequest() throws IOException, ACIDException {
-        int i = 0;
-        LockRequest lockRequest = null;
-        TransactionContext txnContext = null;
-        HashMap<Integer, TransactionContext> jobMap = new HashMap<Integer, TransactionContext>();
-
-        int threadId;
-        String requestType;
-        int jobId;
-        int datasetId;
-        int PKHashVal;
-        int waitTime;
-        ArrayList<Integer> list = null;
-        String lockMode;
-
-        Scanner scanner = new Scanner(new FileInputStream(requestFileName));
-        while (scanner.hasNextLine()) {
-            try {
-                threadId = Integer.parseInt(scanner.next().substring(1));
-                requestType = scanner.next();
-                if (requestType.equals("CSQ") || requestType.equals("CST") || requestType.equals("END")) {
-                    log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType);
-                    lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType));
-                    if (requestType.equals("CSQ") || requestType.equals("CST")) {
-                        list = new ArrayList<Integer>();
-                        while (scanner.hasNextInt()) {
-                            threadId = scanner.nextInt();
-                            if (threadId < 0) {
-                                break;
-                            }
-                            list.add(threadId);
-                        }
-                        expectedResultList.add(list);
-                    }
-                } else if (requestType.equals("DW")) {
-                    defaultWaitTime = scanner.nextInt();
-                    log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType + "," + defaultWaitTime);
-                    continue;
-                } else if (requestType.equals("W")) {
-                    waitTime = scanner.nextInt();
-                    log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType);
-                    lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType), waitTime);
-                } else {
-                    jobId = Integer.parseInt(scanner.next().substring(1));
-                    datasetId = Integer.parseInt(scanner.next().substring(1));
-                    PKHashVal = Integer.parseInt(scanner.next().substring(1));
-                    lockMode = scanner.next();
-                    txnContext = jobMap.get(jobId);
-                    if (txnContext == null) {
-                        txnContext = new TransactionContext(new JobId(jobId), txnProvider);
-                        jobMap.put(jobId, txnContext);
-                    }
-                    log("LockRequest[" + i++ + "]:T" + threadId + "," + requestType + ",J" + jobId + ",D" + datasetId
-                            + ",E" + PKHashVal + "," + lockMode);
-                    lockRequest = new LockRequest("Thread-" + threadId, getRequestType(requestType), new DatasetId(
-                            datasetId), PKHashVal, getLockMode(lockMode), txnContext);
-                }
-
-                requestList.add(lockRequest);
-            } catch (NoSuchElementException e) {
-                scanner.close();
-                break;
-            }
-        }
-    }
-
-    public void log(String s) {
-        System.out.println(s);
-    }
-
-    private int getRequestType(String s) {
-        if (s.equals("L")) {
-            return RequestType.LOCK;
-        }
-
-        if (s.equals("TL")) {
-            return RequestType.TRY_LOCK;
-        }
-
-        if (s.equals("IL")) {
-            return RequestType.INSTANT_LOCK;
-        }
-
-        if (s.equals("ITL")) {
-            return RequestType.INSTANT_TRY_LOCK;
-        }
-
-        if (s.equals("UL")) {
-            return RequestType.UNLOCK;
-        }
-
-        if (s.equals("RL")) {
-            return RequestType.RELEASE_LOCKS;
-        }
-
-        if (s.equals("CSQ")) {
-            return RequestType.CHECK_SEQUENCE;
-        }
-
-        if (s.equals("CST")) {
-            return RequestType.CHECK_SET;
-        }
-
-        if (s.equals("END")) {
-            return RequestType.END;
-        }
-
-        if (s.equals("W")) {
-            return RequestType.WAIT;
-        }
-
-        try {
-            throw new UnsupportedOperationException("Invalid request type:" + s);
-        } catch (Exception e) {
-            e.printStackTrace();
-        } finally {
-            System.exit(0);
-        }
-
-        return -1;
-
-    }
-
-    private byte getLockMode(String s) {
-        if (s.equals("S")) {
-            return LockMode.S;
-        }
-
-        if (s.equals("X")) {
-            return LockMode.X;
-        }
-
-        try {
-            throw new UnsupportedOperationException("Invalid lock mode type:" + s);
-        } catch (Exception e) {
-            e.printStackTrace();
-        } finally {
-            System.exit(0);
-        }
-
-        return -1;
-    }
-}
-
-class LockRequestWorker implements Runnable {
-
-    String threadName;
-    ILockManager lockMgr;
-    WorkerReadyQueue workerReadyQueue;
-    LockRequest lockRequest;
-    boolean needWait;
-    boolean isAwaken;
-    boolean isDone;
-
-    public LockRequestWorker(TransactionSubsystem txnProvider, WorkerReadyQueue workerReadyQueue, String threadName) {
-        this.lockMgr = txnProvider.getLockManager();
-        this.workerReadyQueue = workerReadyQueue;
-        this.threadName = new String(threadName);
-        this.lockRequest = null;
-        needWait = true;
-        isDone = false;
-        isAwaken = false;
-    }
-
-    public boolean isAwaken() {
-        return isAwaken;
-    }
-
-    @Override
-    public void run() {
-        //initial wait
-        needWait = true;
-        isAwaken = false;
-
-        while (!isDone) {
-            while (needWait) {
-                synchronized (this) {
-                    workerReadyQueue.push(this);
-                    try {
-                        this.wait();
-                        isAwaken = true;
-                    } catch (InterruptedException e) {
-                        e.printStackTrace();
-                    }
-                }
-            }
-
-            if (isDone) {
-                break;
-            }
-
-            try {
-                sendRequest(lockRequest);
-            } catch (ACIDException e) {
-                if (lockRequest.txnContext.isTimeout()) {
-                    if (lockRequest.txnContext.getTxnState() != ITransactionManager.ABORTED) {
-                        lockRequest.txnContext.setTxnState(ITransactionManager.ABORTED);
-                        log("*** " + lockRequest.txnContext.getJobId() + " lock request causing deadlock ***");
-                        log("Abort --> Releasing all locks acquired by " + lockRequest.txnContext.getJobId());
-                        try {
-                            lockMgr.releaseLocks(lockRequest.txnContext);
-                        } catch (ACIDException e1) {
-                            e1.printStackTrace();
-                        }
-                        log("Abort --> Released all locks acquired by " + lockRequest.txnContext.getJobId());
-                    }
-                    isDone = true;
-                } else {
-                    e.printStackTrace();
-                    System.exit(-1);
-                }
-            }
-
-            try {
-                Thread.sleep(1);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-
-            needWait = true;
-            isAwaken = false;
-        }
-    }
-
-    public void sendRequest(LockRequest request) throws ACIDException {
-
-        switch (request.requestType) {
-            case RequestType.LOCK:
-                lockMgr.lock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
-                break;
-            case RequestType.INSTANT_LOCK:
-                lockMgr.instantLock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
-                break;
-            case RequestType.TRY_LOCK:
-                request.isTryLockFailed = !lockMgr.tryLock(request.datasetIdObj, request.entityHashValue,
-                        request.lockMode, request.txnContext);
-                break;
-            case RequestType.INSTANT_TRY_LOCK:
-                lockMgr.instantTryLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
-                        request.txnContext);
-                break;
-            case RequestType.UNLOCK:
-                lockMgr.unlock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
-                break;
-            case RequestType.RELEASE_LOCKS:
-                lockMgr.releaseLocks(request.txnContext);
-                break;
-            default:
-                throw new UnsupportedOperationException("Unsupported lock method");
-        }
-    }
-
-    public void setLockRequest(LockRequest request) {
-        this.lockRequest = request;
-    }
-
-    public void setWait(boolean wait) {
-        needWait = wait;
-    }
-
-    public void setDone(boolean done) {
-        isDone = done;
-    }
-
-    public String getThreadName() {
-        return threadName;
-    }
-
-    public void log(String s) {
-        System.out.println(s);
-    }
-
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("{ t : \"").append(threadName).append("\", r : ");
-        if (lockRequest == null) {
-            sb.append("null");
-        } else {
-            sb.append("\"").append(lockRequest.toString()).append("\"");
-        }
-        sb.append(" }");
-        return sb.toString();
-    }
-}
-
-class WorkerReadyQueue {
-    ArrayList<LockRequestWorker> workerReadyQueue;
-
-    public WorkerReadyQueue() {
-        workerReadyQueue = new ArrayList<LockRequestWorker>();
-    }
-
-    public synchronized void push(LockRequestWorker worker) {
-        workerReadyQueue.add(worker);
-    }
-
-    public synchronized LockRequestWorker pop(String threadName) {
-        int i;
-        LockRequestWorker worker = null;
-        int size = workerReadyQueue.size();
-        for (i = 0; i < size; i++) {
-            worker = workerReadyQueue.get(i);
-            if (worker.getThreadName().equals(threadName)) {
-                workerReadyQueue.remove(i);
-                break;
-            }
-        }
-
-        if (i == size) {
-            return null;
-        } else {
-            return worker;
-        }
-    }
-
-    public synchronized int size() {
-        return workerReadyQueue.size();
-    }
-
-    public boolean checkSet(ArrayList<Integer> threadIdList) {
-        int i;
-        int j;
-        StringBuilder s = new StringBuilder();
-        LockRequestWorker worker = null;
-        int resultListSize = 0;
-        int queueSize = workerReadyQueue.size();
-        int listSize = threadIdList.size();
-
-        s.append("ExpectedList(Set):\t");
-        for (i = 0; i < listSize; i++) {
-            s.append(threadIdList.get(i)).append(" ");
-        }
-        s.append("\n");
-
-        while (queueSize < listSize) {
-            //wait until workers finish its task
-            try {
-                Thread.sleep(1);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-            log(Thread.currentThread().getName() + " waiting for worker to finish its task...");
-            queueSize = workerReadyQueue.size();
-        }
-
-        if (listSize != queueSize) {
-            log("listSize:" + listSize + ", queueSize:" + queueSize);
-            return false;
-        }
-
-        s.append("ResultList(Set):\t");
-        for (i = 0; i < listSize; i++) {
-            for (j = 0; j < queueSize; j++) {
-                worker = workerReadyQueue.get(j);
-                if (worker.getThreadName().equals("Thread-" + threadIdList.get(i))) {
-                    s.append(threadIdList.get(i)).append(" ");
-                    resultListSize++;
-                    break;
-                }
-            }
-        }
-
-        log(s.toString());
-        if (listSize != resultListSize) {
-            return false;
-        }
-
-        return true;
-    }
-
-    public boolean checkSequence(ArrayList<Integer> threadIdList) {
-        int i;
-        StringBuilder s = new StringBuilder();
-        LockRequestWorker worker = null;
-        int queueSize = workerReadyQueue.size();
-        int listSize = threadIdList.size();
-
-        s.append("ExpectedList(Sequence):\t");
-        for (i = 0; i < listSize; i++) {
-            s.append(threadIdList.get(i)).append(" ");
-        }
-        s.append("\n");
-
-        while (queueSize < listSize) {
-            //wait until workers finish its task
-            try {
-                Thread.sleep(1);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            }
-            log(Thread.currentThread().getName() + " Waiting for worker to finish its task...");
-            queueSize = workerReadyQueue.size();
-        }
-
-        if (queueSize != listSize) {
-            return false;
-        }
-
-        s.append("ResultList(Sequence):\t");
-        for (i = 0; i < listSize; i++) {
-            worker = workerReadyQueue.get(i);
-            if (!worker.getThreadName().equals("Thread-" + threadIdList.get(i))) {
-                log(s.toString());
-                return false;
-            } else {
-                s.append(threadIdList.get(i)).append(" ");
-            }
-        }
-
-        log(s.toString());
-        return true;
-    }
-
-    public void log(String s) {
-        System.out.println(s);
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
deleted file mode 100644
index eab8f46..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
+++ /dev/null
@@ -1,636 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Random;
-
-import org.apache.asterix.common.config.AsterixPropertiesAccessor;
-import org.apache.asterix.common.config.AsterixTransactionProperties;
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.ITransactionManager;
-import org.apache.asterix.common.transactions.JobId;
-import org.apache.asterix.transaction.management.service.logging.LogManager;
-import org.apache.asterix.transaction.management.service.transaction.TransactionContext;
-import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import org.apache.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import org.apache.commons.io.FileUtils;
-
-/**
- * LockManagerUnitTest: unit test of LockManager
- *
- * @author kisskys
- */
-
-public class LockManagerRandomUnitTest {
-
-    private static final int MAX_NUM_OF_UPGRADE_JOB = 2;//2
-    private static final int MAX_NUM_OF_ENTITY_LOCK_JOB = 8;//8
-    private static final int MAX_NUM_OF_DATASET_LOCK_JOB = 2;//2
-    private static final int MAX_NUM_OF_THREAD_IN_A_JOB = 2; //4
-    private static int jobId = 0;
-    private static Random rand;
-
-    public static void main(String args[]) throws ACIDException, AsterixException, IOException {
-        int i;
-        //prepare configuration file
-        File cwd = new File(System.getProperty("user.dir"));
-        File asterixdbDir = cwd.getParentFile();
-        File srcFile = new File(asterixdbDir.getAbsoluteFile(),
-                "asterix-app/src/main/resources/asterix-build-configuration.xml");
-        File destFile = new File(cwd, "target/classes/asterix-configuration.xml");
-        FileUtils.copyFile(srcFile, destFile);
-
-        TransactionSubsystem txnProvider = new TransactionSubsystem("nc1", null,
-                new AsterixTransactionProperties(new AsterixPropertiesAccessor()));
-        rand = new Random(System.currentTimeMillis());
-        for (i = 0; i < MAX_NUM_OF_ENTITY_LOCK_JOB; i++) {
-            System.out.println("Creating " + i + "th EntityLockJob..");
-            generateEntityLockThread(txnProvider);
-        }
-
-        for (i = 0; i < MAX_NUM_OF_DATASET_LOCK_JOB; i++) {
-            System.out.println("Creating " + i + "th DatasetLockJob..");
-            generateDatasetLockThread(txnProvider);
-        }
-
-        for (i = 0; i < MAX_NUM_OF_UPGRADE_JOB; i++) {
-            System.out.println("Creating " + i + "th EntityLockUpgradeJob..");
-            generateEntityLockUpgradeThread(txnProvider);
-        }
-        ((LogManager) txnProvider.getLogManager()).stop(false, null);
-    }
-
-    private static void generateEntityLockThread(TransactionSubsystem txnProvider) {
-        Thread t;
-        int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
-        if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
-            childCount = 1;
-        }
-        TransactionContext txnContext = generateTxnContext(txnProvider);
-
-        for (int i = 0; i < childCount; i++) {
-            System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockThread..");
-            t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, false, false));
-            t.start();
-        }
-    }
-
-    private static void generateDatasetLockThread(TransactionSubsystem txnProvider) {
-        Thread t;
-        //        int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
-        //        if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
-        //            childCount = 1;
-        //        }
-        int childCount = 1;
-
-        TransactionContext txnContext = generateTxnContext(txnProvider);
-
-        for (int i = 0; i < childCount; i++) {
-            System.out.println("Creating " + txnContext.getJobId() + "," + i + "th DatasetLockThread..");
-            t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, true, false, false));
-            t.start();
-        }
-    }
-
-    private static void generateEntityLockUpgradeThread(TransactionSubsystem txnProvider) {
-        int i;
-        Thread t;
-        int childCount = MAX_NUM_OF_THREAD_IN_A_JOB;
-        if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
-            childCount = 1;
-        }
-        TransactionContext txnContext = generateTxnContext(txnProvider);
-
-        for (i = 0; i < childCount - 1; i++) {
-            System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(false)..");
-            t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, false));
-            t.start();
-        }
-        System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(true)..");
-        t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, true));
-        t.start();
-    }
-
-    private static TransactionContext generateTxnContext(TransactionSubsystem txnProvider) {
-        try {
-            return new TransactionContext(new JobId(jobId++), txnProvider);
-        } catch (ACIDException e) {
-            e.printStackTrace();
-            return null;
-        }
-    }
-
-}
-
-class LockRequestProducer implements Runnable {
-
-    private static final int MAX_DATASET_NUM = 10;//10
-    private static final int MAX_ENTITY_NUM = 30;//30
-    private static final int MAX_LOCK_MODE_NUM = 2;
-    private static final long DATASET_LOCK_THREAD_SLEEP_TIME = 1000;
-    private static final int MAX_LOCK_REQUEST_TYPE_NUM = 4;
-
-    private ILockManager lockMgr;
-    private TransactionContext txnContext;
-    private Random rand;
-    private boolean isDatasetLock; //dataset or entity
-    private ArrayList<LockRequest> requestQueue;
-    private StringBuilder requestHistory;
-    private int unlockIndex;
-    private int upgradeIndex;
-    private boolean isUpgradeThread;
-    private boolean isUpgradeThreadJob;
-    private boolean isDone;
-
-    public LockRequestProducer(ILockManager lockMgr, TransactionContext txnContext, boolean isDatasetLock,
-            boolean isUpgradeThreadJob, boolean isUpgradeThread) {
-        this.lockMgr = lockMgr;
-        this.txnContext = txnContext;
-        this.isDatasetLock = isDatasetLock;
-        this.isUpgradeThreadJob = isUpgradeThreadJob;
-        this.isUpgradeThread = isUpgradeThread;
-
-        this.rand = new Random(System.currentTimeMillis());
-        requestQueue = new ArrayList<LockRequest>();
-        requestHistory = new StringBuilder();
-        unlockIndex = 0;
-        upgradeIndex = 0;
-        isDone = false;
-    }
-
-    @Override
-    public void run() {
-        try {
-            if (isDatasetLock) {
-                System.out.println("DatasetLockThread(" + Thread.currentThread().getName() + ") is running...");
-                runDatasetLockTask();
-            } else {
-                System.out.println("EntityLockThread(" + Thread.currentThread().getName() + "," + isUpgradeThreadJob
-                        + "," + isUpgradeThread + ") is running...");
-                runEntityLockTask();
-            }
-        } catch (Exception e) {
-            e.printStackTrace();
-            return;
-        } finally {
-
-            /*
-            System.out.println("" + Thread.currentThread().getName() + "\n" + requestHistory.toString() + ""
-                    + Thread.currentThread().getName() + "\n");
-            System.out.println("RequestHistoryPerJobId\n" + ((LockManager) lockMgr).getLocalRequestHistory());
-            System.out.println("");
-            System.out.println("GlobalRequestHistory\n" + ((LockManager) lockMgr).getGlobalRequestHistory());
-            System.out.println("");
-            */
-        }
-    }
-
-    private void runDatasetLockTask() {
-        try {
-            produceDatasetLockRequest();
-            if (isDone) {
-                return;
-            }
-        } catch (ACIDException e) {
-            e.printStackTrace();
-            return;
-        }
-
-        try {
-            Thread.sleep(DATASET_LOCK_THREAD_SLEEP_TIME);
-        } catch (InterruptedException e) {
-            e.printStackTrace();
-        }
-
-        try {
-            produceDatasetUnlockRequest();
-            if (isDone) {
-                return;
-            }
-        } catch (ACIDException e) {
-            e.printStackTrace();
-            return;
-        }
-    }
-
-    private void runEntityLockTask() {
-        int i;
-        byte lockMode;
-        int lockCount;
-        int upgradeCount;
-        int releaseCount;
-        boolean mayRelease = false;
-
-        lockCount = 1 + rand.nextInt(20);
-        if (isUpgradeThreadJob) {
-            if (isUpgradeThread) {
-                upgradeCount = 1; //rand.nextInt(4) + 1;
-                if (upgradeCount > lockCount) {
-                    upgradeCount = lockCount;
-                }
-            } else {
-                upgradeCount = 0;
-            }
-            lockMode = LockMode.S;
-        } else {
-            upgradeCount = 0;
-            lockMode = (byte) (this.txnContext.getJobId().getId() % 2);
-        }
-        releaseCount = rand.nextInt(5) % 3 == 0 ? 1 : 0;
-
-        //lock
-        for (i = 0; i < lockCount; i++) {
-            try {
-                produceEntityLockRequest(lockMode);
-                if (isDone) {
-                    return;
-                }
-            } catch (ACIDException e) {
-                e.printStackTrace();
-                return;
-            }
-        }
-
-        //upgrade
-        for (i = 0; i < upgradeCount; i++) {
-            try {
-                produceEntityLockUpgradeRequest();
-                if (isDone) {
-                    return;
-                }
-            } catch (ACIDException e) {
-                e.printStackTrace();
-                return;
-            }
-        }
-
-        //unlock or releaseLocks
-        if (releaseCount == 0) {
-            //unlock
-            for (i = 0; i < lockCount; i++) {
-                try {
-                    produceEntityUnlockRequest();
-                    if (isDone) {
-                        return;
-                    }
-                } catch (ACIDException e) {
-                    e.printStackTrace();
-                    return;
-                }
-            }
-        } else {
-            try {
-                synchronized (txnContext) {
-                    if (txnContext.getTxnState() != ITransactionManager.ABORTED) {
-                        txnContext.setTxnState(ITransactionManager.ABORTED);
-                        mayRelease = true;
-                    }
-                }
-                if (mayRelease) {
-                    produceEntityReleaseLocksRequest();
-                }
-            } catch (ACIDException e) {
-                e.printStackTrace();
-                return;
-            }
-        }
-    }
-
-    private void produceDatasetLockRequest() throws ACIDException {
-        int requestType = RequestType.LOCK;
-        int datasetId = rand.nextInt(MAX_DATASET_NUM);
-        int entityHashValue = -1;
-        byte lockMode = (byte) (rand.nextInt(MAX_LOCK_MODE_NUM));
-        LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId),
-                entityHashValue, lockMode, txnContext);
-        requestQueue.add(request);
-        requestHistory.append(request.prettyPrint());
-        sendRequest(request);
-    }
-
-    private void produceDatasetUnlockRequest() throws ACIDException {
-        LockRequest lockRequest = requestQueue.get(0);
-
-        int requestType = RequestType.RELEASE_LOCKS;
-        int datasetId = lockRequest.datasetIdObj.getId();
-        int entityHashValue = -1;
-        byte lockMode = LockMode.S;//lockMode is not used for unlock() call.
-        LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId),
-                entityHashValue, lockMode, txnContext);
-        requestQueue.add(request);
-        requestHistory.append(request.prettyPrint());
-        sendRequest(request);
-    }
-
-    private void produceEntityLockRequest(byte lockMode) throws ACIDException {
-        int requestType = rand.nextInt(MAX_LOCK_REQUEST_TYPE_NUM);
-        int datasetId = rand.nextInt(MAX_DATASET_NUM);
-        int entityHashValue = rand.nextInt(MAX_ENTITY_NUM);
-        LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, new DatasetId(datasetId),
-                entityHashValue, lockMode, txnContext);
-        requestQueue.add(request);
-        requestHistory.append(request.prettyPrint());
-        sendRequest(request);
-    }
-
-    private void produceEntityLockUpgradeRequest() throws ACIDException {
-        LockRequest lockRequest = null;
-        int size = requestQueue.size();
-        boolean existLockRequest = false;
-
-        while (upgradeIndex < size) {
-            lockRequest = requestQueue.get(upgradeIndex++);
-            if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
-                continue;
-            }
-            if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
-                    || lockRequest.requestType == RequestType.INSTANT_LOCK
-                    || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
-                continue;
-            }
-            if (lockRequest.lockMode == LockMode.X) {
-                continue;
-            }
-            existLockRequest = true;
-            break;
-        }
-
-        if (existLockRequest) {
-            int requestType = lockRequest.requestType;
-            int datasetId = lockRequest.datasetIdObj.getId();
-            int entityHashValue = lockRequest.entityHashValue;
-            byte lockMode = LockMode.X;
-            LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType,
-                    new DatasetId(datasetId), entityHashValue, lockMode, txnContext);
-            request.isUpgrade = true;
-            requestQueue.add(request);
-            requestHistory.append(request.prettyPrint());
-            sendRequest(request);
-        }
-    }
-
-    private void produceEntityUnlockRequest() throws ACIDException {
-        LockRequest lockRequest = null;
-        int size = requestQueue.size();
-        boolean existLockRequest = false;
-
-        while (unlockIndex < size) {
-            lockRequest = requestQueue.get(unlockIndex++);
-            if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
-                continue;
-            }
-            if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
-                    || lockRequest.requestType == RequestType.INSTANT_LOCK
-                    || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
-                continue;
-            }
-            existLockRequest = true;
-            break;
-        }
-
-        if (existLockRequest) {
-            int requestType = RequestType.UNLOCK;
-            int datasetId = lockRequest.datasetIdObj.getId();
-            int entityHashValue = lockRequest.entityHashValue;
-            byte lockMode = lockRequest.lockMode;
-            LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType,
-                    new DatasetId(datasetId), entityHashValue, lockMode, txnContext);
-            requestQueue.add(request);
-            requestHistory.append(request.prettyPrint());
-            sendRequest(request);
-        }
-    }
-
-    private void produceEntityReleaseLocksRequest() throws ACIDException {
-        LockRequest lockRequest = null;
-        int size = requestQueue.size();
-        boolean existLockRequest = false;
-
-        while (unlockIndex < size) {
-            lockRequest = requestQueue.get(unlockIndex++);
-            if (lockRequest.isUpgrade || lockRequest.isTryLockFailed) {
-                continue;
-            }
-            if (lockRequest.requestType == RequestType.UNLOCK || lockRequest.requestType == RequestType.RELEASE_LOCKS
-                    || lockRequest.requestType == RequestType.INSTANT_LOCK
-                    || lockRequest.requestType == RequestType.INSTANT_TRY_LOCK) {
-                continue;
-            }
-            existLockRequest = true;
-            break;
-        }
-
-        if (existLockRequest) {
-            int requestType = RequestType.RELEASE_LOCKS;
-            int datasetId = lockRequest.datasetIdObj.getId();
-            int entityHashValue = lockRequest.entityHashValue;
-            byte lockMode = lockRequest.lockMode;
-            LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType,
-                    new DatasetId(datasetId), entityHashValue, lockMode, txnContext);
-            requestQueue.add(request);
-            requestHistory.append(request.prettyPrint());
-            sendRequest(request);
-        }
-    }
-
-    private void sendRequest(LockRequest request) throws ACIDException {
-
-        switch (request.requestType) {
-            case RequestType.LOCK:
-                try {
-                    lockMgr.lock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
-                } catch (ACIDException e) {
-                    if (request.txnContext.isTimeout()) {
-                        if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
-                            request.txnContext.setTxnState(ITransactionManager.ABORTED);
-                            log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
-                            log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
-                            try {
-                                lockMgr.releaseLocks(request.txnContext);
-                            } catch (ACIDException e1) {
-                                e1.printStackTrace();
-                            }
-                            log("Abort --> Released all locks acquired by " + request.txnContext.getJobId());
-                        }
-                        isDone = true;
-                    } else {
-                        throw e;
-                    }
-                }
-                break;
-            case RequestType.INSTANT_LOCK:
-                try {
-                    lockMgr.instantLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
-                            request.txnContext);
-                } catch (ACIDException e) {
-                    if (request.txnContext.isTimeout()) {
-                        if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
-                            request.txnContext.setTxnState(ITransactionManager.ABORTED);
-                            log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
-                            log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
-                            try {
-                                lockMgr.releaseLocks(request.txnContext);
-                            } catch (ACIDException e1) {
-                                e1.printStackTrace();
-                            }
-                            log("Abort --> Released all locks acquired by " + request.txnContext.getJobId());
-                        }
-                        isDone = true;
-                    } else {
-                        throw e;
-                    }
-                }
-                break;
-            case RequestType.TRY_LOCK:
-                request.isTryLockFailed = !lockMgr.tryLock(request.datasetIdObj, request.entityHashValue,
-                        request.lockMode, request.txnContext);
-                break;
-            case RequestType.INSTANT_TRY_LOCK:
-                lockMgr.instantTryLock(request.datasetIdObj, request.entityHashValue, request.lockMode,
-                        request.txnContext);
-                break;
-            case RequestType.UNLOCK:
-                lockMgr.unlock(request.datasetIdObj, request.entityHashValue, request.lockMode, request.txnContext);
-                break;
-            case RequestType.RELEASE_LOCKS:
-                lockMgr.releaseLocks(request.txnContext);
-                break;
-            default:
-                throw new UnsupportedOperationException("Unsupported lock method");
-        }
-        try {
-            Thread.sleep(0);
-        } catch (InterruptedException e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
-        }
-    }
-
-    private void log(String s) {
-        System.out.println(s);
-    }
-}
-
-class LockRequest {
-    public int requestType;
-    public DatasetId datasetIdObj;
-    public int entityHashValue;
-    public byte lockMode;
-    public ITransactionContext txnContext;
-    public boolean isUpgrade;
-    public boolean isTryLockFailed;
-    public long requestTime;
-    public String threadName;
-
-    public LockRequest(String threadName, int requestType, DatasetId datasetIdObj, int entityHashValue, byte lockMode,
-            ITransactionContext txnContext) {
-        this.requestType = requestType;
-        this.datasetIdObj = datasetIdObj;
-        this.entityHashValue = entityHashValue;
-        this.lockMode = lockMode;
-        this.txnContext = txnContext;
-        this.requestTime = System.currentTimeMillis();
-        this.threadName = new String(threadName);
-        isUpgrade = false;
-        isTryLockFailed = false;//used for TryLock request not to call Unlock when the tryLock failed.
-    }
-
-    public LockRequest(String threadName, int requestType) {
-        this.requestType = requestType;
-        this.requestTime = System.currentTimeMillis();
-        this.threadName = new String(threadName);
-    }
-
-    //used for "W" request type
-    public LockRequest(String threadName, int requestType, int waitTime) {
-        this.requestType = requestType;
-        this.requestTime = System.currentTimeMillis();
-        this.threadName = new String(threadName);
-        this.entityHashValue = waitTime;
-    }
-
-    @Override
-    public String toString() {
-        return prettyPrint();
-    }
-
-    public String prettyPrint() {
-        StringBuilder s = new StringBuilder();
-        //s.append(threadName.charAt(7)).append("\t").append("\t");
-        s.append("T").append(threadName.substring(7)).append("\t");
-        switch (requestType) {
-            case RequestType.LOCK:
-                s.append("L");
-                break;
-            case RequestType.TRY_LOCK:
-                s.append("TL");
-                break;
-            case RequestType.INSTANT_LOCK:
-                s.append("IL");
-                break;
-            case RequestType.INSTANT_TRY_LOCK:
-                s.append("ITL");
-                break;
-            case RequestType.UNLOCK:
-                s.append("UL");
-                break;
-            case RequestType.RELEASE_LOCKS:
-                s.append("RL");
-                break;
-            case RequestType.CHECK_SEQUENCE:
-                s.append("CSQ");
-                return s.toString();
-            case RequestType.CHECK_SET:
-                s.append("CST");
-                return s.toString();
-            case RequestType.END:
-                s.append("END");
-                return s.toString();
-            case RequestType.WAIT:
-                s.append("W").append("\t").append(entityHashValue);
-                return s.toString();
-            default:
-                throw new UnsupportedOperationException("Unsupported method");
-        }
-        s.append("\tJ").append(txnContext.getJobId().getId()).append("\tD").append(datasetIdObj.getId()).append("\tE")
-                .append(entityHashValue).append("\t");
-        s.append(LockMode.toString(lockMode)).append("\n");
-        return s.toString();
-    }
-}
-
-class RequestType {
-    public static final int LOCK = 0;
-    public static final int TRY_LOCK = 1;
-    public static final int INSTANT_LOCK = 2;
-    public static final int INSTANT_TRY_LOCK = 3;
-    public static final int UNLOCK = 4;
-    public static final int RELEASE_LOCKS = 5;
-    public static final int CHECK_SEQUENCE = 6;
-    public static final int CHECK_SET = 7;
-    public static final int END = 8;
-    public static final int WAIT = 9;
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockRequestTracker.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockRequestTracker.java
deleted file mode 100644
index 6fb0675..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockRequestTracker.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-public class LockRequestTracker {
-    HashMap<Integer, StringBuilder> historyPerJob; //per job
-    StringBuilder historyForAllJobs;
-    StringBuilder requestHistoryForAllJobs; //request only
-
-    public LockRequestTracker() {
-        historyForAllJobs = new StringBuilder();
-        historyPerJob = new HashMap<Integer, StringBuilder>();
-        requestHistoryForAllJobs = new StringBuilder();
-    }
-
-    public void addEvent(String msg, LockRequest request) {
-        int jobId = request.txnContext.getJobId().getId();
-        StringBuilder jobHistory = historyPerJob.get(jobId);
-
-        //update jobHistory
-        if (jobHistory == null) {
-            jobHistory = new StringBuilder();
-        }
-        jobHistory.append(request.prettyPrint()).append("--> ").append(msg).append("\n");
-        historyPerJob.put(jobId, jobHistory);
-
-        //handle global request queue
-        historyForAllJobs.append(request.prettyPrint()).append("--> ").append(msg).append("\n");
-    }
-
-    public void addRequest(LockRequest request) {
-        requestHistoryForAllJobs.append(request.prettyPrint());
-    }
-
-    public String getHistoryForAllJobs() {
-        return historyForAllJobs.toString();
-    }
-
-    public String getHistoryPerJob() {
-        StringBuilder history = new StringBuilder();
-        Set<Entry<Integer, StringBuilder>> s = historyPerJob.entrySet();
-        Iterator<Entry<Integer, StringBuilder>> iter = s.iterator();
-        while (iter.hasNext()) {
-            Map.Entry<Integer, StringBuilder> entry = (Map.Entry<Integer, StringBuilder>) iter.next();
-            history.append(entry.getValue().toString());
-        }
-        return history.toString();
-    }
-
-    public String getRequestHistoryForAllJobs() {
-        return requestHistoryForAllJobs.toString();
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiter.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiter.java
deleted file mode 100644
index 82f0877..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiter.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-/**
- * LockWaiter object is used for keeping a lock waiter or a lock upgrader information on a certain resource.
- * The resource can be a dataset or an entity.
- *
- * @author kisskys
- */
-public class LockWaiter {
-    /**
-     * entityInfoSlotNum:
-     * If this LockWaiter object is used, this variable is used to indicate the corresponding EntityInfoSlotNum.
-     * Otherwise, the variable is used for nextFreeSlot Which indicates the next free waiter object.
-     */
-    private int entityInfoSlotNum;
-    private boolean wait;
-    private boolean victim;
-    private byte waiterCount;
-    private boolean firstGetUp;
-    private int nextWaiterObjId; //used for DatasetLockInfo and EntityLockInfo
-    private int nextWaitingResourceObjId; //used for JobInfo
-    private long beginWaitTime;
-    private boolean isWaiter; //is upgrader or waiter
-    private boolean isWaitingOnEntityLock; //is waiting on datasetLock or entityLock
-
-    public LockWaiter() {
-        this.victim = false;
-        this.wait = true;
-        waiterCount = 0;
-        nextWaiterObjId = -1;
-        nextWaitingResourceObjId = -1;
-    }
-
-    public void setEntityInfoSlot(int slotNum) {
-        this.entityInfoSlotNum = slotNum;
-    }
-
-    public int getEntityInfoSlot() {
-        return this.entityInfoSlotNum;
-    }
-
-    public void setNextFreeSlot(int slotNum) {
-        this.entityInfoSlotNum = slotNum;
-    }
-
-    public int getNextFreeSlot() {
-        return this.entityInfoSlotNum;
-    }
-
-    public void setWait(boolean wait) {
-        this.wait = wait;
-    }
-
-    public boolean needWait() {
-        return this.wait;
-    }
-
-    public void setVictim(boolean victim) {
-        this.victim = victim;
-    }
-
-    public boolean isVictim() {
-        return this.victim;
-    }
-
-    public void increaseWaiterCount() {
-        waiterCount++;
-    }
-
-    public void decreaseWaiterCount() {
-        waiterCount--;
-    }
-
-    public byte getWaiterCount() {
-        return waiterCount;
-    }
-
-    public void setWaiterCount(byte count) {
-        waiterCount = count;
-    }
-
-    public void setFirstGetUp(boolean isFirst) {
-        firstGetUp = isFirst;
-    }
-
-    public boolean isFirstGetUp() {
-        return firstGetUp;
-    }
-
-    public void setNextWaiterObjId(int next) {
-        nextWaiterObjId = next;
-    }
-
-    public int getNextWaiterObjId() {
-        return nextWaiterObjId;
-    }
-
-    public void setNextWaitingResourceObjId(int next) {
-        nextWaitingResourceObjId = next;
-    }
-
-    public int getNextWaitingResourceObjId() {
-        return nextWaitingResourceObjId;
-    }
-
-    public void setBeginWaitTime(long time) {
-        this.beginWaitTime = time;
-    }
-
-    public long getBeginWaitTime() {
-        return beginWaitTime;
-    }
-
-    public boolean isWaiter() {
-        return isWaiter;
-    }
-
-    public void setWaiter(boolean isWaiter) {
-        this.isWaiter = isWaiter;
-    }
-
-    public boolean isWaitingOnEntityLock() {
-        return isWaitingOnEntityLock;
-    }
-
-    public void setWaitingOnEntityLock(boolean isWaitingOnEntityLock) {
-        this.isWaitingOnEntityLock = isWaitingOnEntityLock;
-    }
-
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiterManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiterManager.java
deleted file mode 100644
index ae971fb..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/LockWaiterManager.java
+++ /dev/null
@@ -1,403 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.ArrayList;
-
-/**
- * LockWaiterManager manages LockWaiter objects array.
- * The array grows when the slots are overflowed.
- * Also, the array shrinks according to the following shrink policy
- * : Shrink when the resource under-utilization lasts for a certain threshold time.
- *
- * @author kisskys
- */
-public class LockWaiterManager {
-
-    public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
-
-    private ArrayList<ChildLockWaiterArrayManager> pArray; //parent array
-    private int allocChild; //used to allocate the next free LockWaiter object.
-    private long shrinkTimer;
-    private boolean isShrinkTimerOn;
-    private int occupiedSlots;
-
-    //    ////////////////////////////////////////////////
-    //    // begin of unit test
-    //    ////////////////////////////////////////////////
-    //
-    //    public static final int SHRINK_TIMER_THRESHOLD = 0; //for unit test
-    //
-    //    /**
-    //     * @param args
-    //     */
-    //    public static void main(String[] args) {
-    //        final int DataSize = 5000;
-    //
-    //        int i, j;
-    //        int slots = ChildLockWaiterArrayManager.NUM_OF_SLOTS;
-    //        int data[] = new int[DataSize];
-    //        LockWaiterManager lwMgr = new LockWaiterManager();
-    //
-    //        //allocate: 50
-    //        System.out.println("allocate: 50");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = lwMgr.allocate();
-    //            }
-    //
-    //            System.out.println(lwMgr.prettyPrint());
-    //        }
-    //
-    //        //deallocate from the last child to the first child
-    //        System.out.println("deallocate from the last child to the first child");
-    //        for (i = 4; i >= 0; i--) {
-    //            for (j = i * slots + slots - 1; j >= i * slots; j--) {
-    //                lwMgr.deallocate(data[j]);
-    //            }
-    //            System.out.println(lwMgr.prettyPrint());
-    //        }
-    //
-    //        //allocate: 50
-    //        System.out.println("allocate: 50");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = lwMgr.allocate();
-    //            }
-    //
-    //            System.out.println(lwMgr.prettyPrint());
-    //        }
-    //
-    //        //deallocate from the first child to last child
-    //        System.out.println("deallocate from the first child to last child");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                lwMgr.deallocate(data[j]);
-    //            }
-    //
-    //            System.out.println(lwMgr.prettyPrint());
-    //        }
-    //
-    //        //allocate: 50
-    //        System.out.println("allocate: 50");
-    //        for (i = 0; i < 5; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = lwMgr.allocate();
-    //            }
-    //
-    //            System.out.println(lwMgr.prettyPrint());
-    //        }
-    //
-    //        //deallocate from the first child to 4th child
-    //        System.out.println("deallocate from the first child to 4th child");
-    //        for (i = 0; i < 4; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                lwMgr.deallocate(data[j]);
-    //            }
-    //
-    //            System.out.println(lwMgr.prettyPrint());
-    //        }
-    //
-    //        //allocate: 40
-    //        System.out.println("allocate: 40");
-    //        for (i = 0; i < 4; i++) {
-    //            for (j = i * slots; j < i * slots + slots; j++) {
-    //                data[j] = lwMgr.allocate();
-    //            }
-    //
-    //            System.out.println(lwMgr.prettyPrint());
-    //        }
-    //    }
-    //
-    //    ////////////////////////////////////////////////
-    //    // end of unit test
-    //    ////////////////////////////////////////////////
-
-    public LockWaiterManager() {
-        pArray = new ArrayList<ChildLockWaiterArrayManager>();
-        pArray.add(new ChildLockWaiterArrayManager());
-        allocChild = 0;
-        occupiedSlots = 0;
-        isShrinkTimerOn = false;
-    }
-
-    public int allocate() {
-        if (pArray.get(allocChild).isFull()) {
-            int size = pArray.size();
-            boolean bAlloc = false;
-            ChildLockWaiterArrayManager child;
-
-            //find a deinitialized child and initialize it
-            for (int i = 0; i < size; i++) {
-                child = pArray.get(i);
-                if (child.isDeinitialized()) {
-                    child.initialize();
-                    allocChild = i;
-                    bAlloc = true;
-                    break;
-                }
-            }
-
-            //allocate new child when there is no deinitialized child
-            if (!bAlloc) {
-                pArray.add(new ChildLockWaiterArrayManager());
-                allocChild = pArray.size() - 1;
-            }
-        }
-        occupiedSlots++;
-        return pArray.get(allocChild).allocate() + allocChild * ChildLockWaiterArrayManager.NUM_OF_SLOTS;
-    }
-
-    void deallocate(int slotNum) {
-        pArray.get(slotNum / ChildLockWaiterArrayManager.NUM_OF_SLOTS).deallocate(
-                slotNum % ChildLockWaiterArrayManager.NUM_OF_SLOTS);
-        occupiedSlots--;
-
-        if (needShrink()) {
-            shrink();
-        }
-    }
-
-    /**
-     * Shrink policy:
-     * Shrink when the resource under-utilization lasts for a certain amount of time.
-     * TODO Need to figure out which of the policies is better
-     * case1.
-     * pArray status : O x x x x x O (O is initialized, x is deinitialized)
-     * In the above status, 'CURRENT' needShrink() returns 'TRUE'
-     * even if there is nothing to shrink or deallocate.
-     * It doesn't distinguish the deinitialized children from initialized children
-     * by calculating totalNumOfSlots = pArray.size() * ChildLockWaiterArrayManager.NUM_OF_SLOTS.
-     * In other words, it doesn't subtract the deinitialized children's slots.
-     * case2.
-     * pArray status : O O x x x x x
-     * However, in the above case, if we subtract the deinitialized children's slots,
-     * needShrink() will return false even if we shrink the pArray at this case.
-     *
-     * @return
-     */
-    private boolean needShrink() {
-        int size = pArray.size();
-        int usedSlots = occupiedSlots;
-        if (usedSlots == 0) {
-            usedSlots = 1;
-        }
-
-        if (size > 1 && size * ChildLockWaiterArrayManager.NUM_OF_SLOTS / usedSlots >= 3) {
-            if (isShrinkTimerOn) {
-                if (System.currentTimeMillis() - shrinkTimer >= SHRINK_TIMER_THRESHOLD) {
-                    isShrinkTimerOn = false;
-                    return true;
-                }
-            } else {
-                //turn on timer
-                isShrinkTimerOn = true;
-                shrinkTimer = System.currentTimeMillis();
-            }
-        } else {
-            //turn off timer
-            isShrinkTimerOn = false;
-        }
-
-        return false;
-    }
-
-    /**
-     * Shrink() may
-     * deinitialize(:deallocates array of LockWaiter objects in a child) Children(s) or
-     * shrink pArray according to the deinitialized children's contiguity status.
-     * It doesn't deinitialize or shrink more than half of children at a time.
-     */
-    private void shrink() {
-        int i;
-        int removeCount = 0;
-        int size = pArray.size();
-        int maxDecreaseCount = size / 2;
-        ChildLockWaiterArrayManager child;
-
-        //The first buffer never be deinitialized.
-        for (i = 1; i < size; i++) {
-            if (pArray.get(i).isEmpty()) {
-                pArray.get(i).deinitialize();
-            }
-        }
-
-        //remove the empty buffers from the end
-        for (i = size - 1; i >= 1; i--) {
-            child = pArray.get(i);
-            if (child.isDeinitialized()) {
-                pArray.remove(i);
-                if (++removeCount == maxDecreaseCount) {
-                    break;
-                }
-            } else {
-                break;
-            }
-        }
-
-        //reset allocChild to the first buffer
-        allocChild = 0;
-
-        isShrinkTimerOn = false;
-    }
-
-    public String prettyPrint() {
-        StringBuilder s = new StringBuilder("\n########### LockWaiterManager Status #############\n");
-        int size = pArray.size();
-        ChildLockWaiterArrayManager child;
-
-        for (int i = 0; i < size; i++) {
-            child = pArray.get(i);
-            if (child.isDeinitialized()) {
-                continue;
-            }
-            s.append("child[" + i + "]");
-            s.append(child.prettyPrint());
-        }
-        return s.toString();
-    }
-
-    public void coreDump(OutputStream os) {
-        StringBuilder sb = new StringBuilder("\n########### LockWaiterManager Status #############\n");
-        int size = pArray.size();
-        ChildLockWaiterArrayManager child;
-
-        sb.append("Number of Child: " + size + "\n");
-        for (int i = 0; i < size; i++) {
-            try {
-                child = pArray.get(i);
-                sb.append("child[" + i + "]");
-                sb.append(child.prettyPrint());
-
-                os.write(sb.toString().getBytes());
-            } catch (IOException e) {
-                //ignore IOException
-            }
-            sb = new StringBuilder();
-        }
-    }
-
-    public int getShrinkTimerThreshold() {
-        return SHRINK_TIMER_THRESHOLD;
-    }
-
-    public LockWaiter getLockWaiter(int slotNum) {
-        return pArray.get(slotNum / ChildLockWaiterArrayManager.NUM_OF_SLOTS).getLockWaiter(
-                slotNum % ChildLockWaiterArrayManager.NUM_OF_SLOTS);
-    }
-}
-
-class ChildLockWaiterArrayManager {
-    public static final int NUM_OF_SLOTS = 100; //number of LockWaiter objects in 'childArray'.
-    //    public static final int NUM_OF_SLOTS = 10; //for unit test
-
-    private int freeSlotNum;
-    private int occupiedSlots; //-1 represents 'deinitialized' state.
-    LockWaiter childArray[];//childArray
-
-    public ChildLockWaiterArrayManager() {
-        initialize();
-    }
-
-    public void initialize() {
-        this.childArray = new LockWaiter[NUM_OF_SLOTS];
-        this.freeSlotNum = 0;
-        this.occupiedSlots = 0;
-
-        for (int i = 0; i < NUM_OF_SLOTS - 1; i++) {
-            childArray[i] = new LockWaiter();
-            childArray[i].setNextFreeSlot(i + 1);
-        }
-        childArray[NUM_OF_SLOTS - 1] = new LockWaiter();
-        childArray[NUM_OF_SLOTS - 1].setNextFreeSlot(-1); //-1 represents EOL(end of link)
-    }
-
-    public LockWaiter getLockWaiter(int slotNum) {
-        return childArray[slotNum];
-    }
-
-    public int allocate() {
-        int currentSlot = freeSlotNum;
-        freeSlotNum = childArray[currentSlot].getNextFreeSlot();
-        childArray[currentSlot].setWait(true);
-        childArray[currentSlot].setVictim(false);
-        childArray[currentSlot].setWaiterCount((byte) 0);
-        childArray[currentSlot].setNextWaiterObjId(-1);
-        childArray[currentSlot].setNextWaitingResourceObjId(-1);
-        childArray[currentSlot].setBeginWaitTime(-1l);
-        occupiedSlots++;
-        if (LockManager.IS_DEBUG_MODE) {
-            System.out.println(Thread.currentThread().getName() + "  Alloc LockWaiterId(" + currentSlot + ")");
-        }
-        return currentSlot;
-    }
-
-    public void deallocate(int slotNum) {
-        childArray[slotNum].setNextFreeSlot(freeSlotNum);
-        freeSlotNum = slotNum;
-        occupiedSlots--;
-        if (LockManager.IS_DEBUG_MODE) {
-            System.out.println(Thread.currentThread().getName() + "  Dealloc LockWaiterId(" + slotNum + ")");
-        }
-    }
-
-    public void deinitialize() {
-        childArray = null;
-        occupiedSlots = -1;
-    }
-
-    public boolean isDeinitialized() {
-        return occupiedSlots == -1;
-    }
-
-    public boolean isFull() {
-        return occupiedSlots == NUM_OF_SLOTS;
-    }
-
-    public boolean isEmpty() {
-        return occupiedSlots == 0;
-    }
-
-    public int getNumOfOccupiedSlots() {
-        return occupiedSlots;
-    }
-
-    public int getFreeSlotNum() {
-        return freeSlotNum;
-    }
-
-    public String prettyPrint() {
-        LockWaiter waiter;
-        StringBuilder sb = new StringBuilder();
-        sb.append("\n\toccupiedSlots:" + getNumOfOccupiedSlots());
-        sb.append("\n\tfreeSlotNum:" + getFreeSlotNum() + "\n");
-        for (int j = 0; j < ChildLockWaiterArrayManager.NUM_OF_SLOTS; j++) {
-            waiter = getLockWaiter(j);
-            sb.append(j).append(": ");
-            sb.append("\t" + waiter.getEntityInfoSlot());
-            sb.append("\t" + waiter.needWait());
-            sb.append("\t" + waiter.isVictim());
-            sb.append("\n");
-        }
-        return sb.toString();
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TimeOutDetector.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TimeOutDetector.java
deleted file mode 100644
index bd20c43..0000000
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/locking/TimeOutDetector.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you 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 at
- *
- *   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 org.apache.asterix.transaction.management.service.locking;
-
-import java.util.LinkedList;
-import java.util.concurrent.Executor;
-
-import org.apache.asterix.common.exceptions.ACIDException;
-
-/**
- * @author pouria, kisskys
- *         Any transaction which has been waiting for a lock for more
- *         than the predefined time-out threshold is considered to be deadlocked
- *         (this can happen in distributed case for example) An instance of this
- *         class triggers scanning (sweeping) lock manager's transactions table
- *         periodically and detects such timed-out transactions
- */
-
-public class TimeOutDetector {
-
-    LockManager lockMgr;
-    Thread trigger;
-    LinkedList<LockWaiter> victimList;
-    int timeoutThreshold;
-    int sweepThreshold;
-
-    public TimeOutDetector(LockManager lockMgr, Executor threadExecutor) {
-        this.victimList = new LinkedList<LockWaiter>();
-        this.lockMgr = lockMgr;
-        this.trigger = new Thread(new TimeoutTrigger(this));
-        this.timeoutThreshold = lockMgr.getTransactionProperties().getTimeoutWaitThreshold();
-        this.sweepThreshold = lockMgr.getTransactionProperties().getTimeoutSweepThreshold();
-        trigger.setDaemon(true);
-        threadExecutor.execute(trigger);
-    }
-
-    public void sweep() throws ACIDException {
-        victimList.clear();
-        // Initiates the time-out sweeping process
-        // from the lockManager
-        lockMgr.sweepForTimeout();
-        notifyVictims();
-    }
-
-    public void checkAndSetVictim(LockWaiter waiterObj) {
-        if (System.currentTimeMillis() - waiterObj.getBeginWaitTime() >= timeoutThreshold) {
-            waiterObj.setVictim(true);
-            waiterObj.setWait(false);
-            victimList.add(waiterObj);
-        }
-    }
-
-    private void notifyVictims() {
-        for (LockWaiter waiterObj : victimList) {
-            synchronized (waiterObj) {
-                waiterObj.notifyAll();
-            }
-        }
-        victimList.clear();
-    }
-}
-
-class TimeoutTrigger implements Runnable {
-
-    TimeOutDetector owner;
-
-    public TimeoutTrigger(TimeOutDetector owner) {
-        this.owner = owner;
-    }
-
-    @Override
-    public void run() {
-        while (true) {
-            try {
-                Thread.sleep(owner.sweepThreshold);
-                owner.sweep(); // Trigger the timeout detector (the owner) to
-                               // initiate sweep
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-            } catch (ACIDException e) {
-                throw new IllegalStateException(e);
-            }
-        }
-    }
-}
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
index 502e9c7..c1180a4 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogBuffer.java
@@ -90,7 +90,7 @@
     @Override
     public void append(ILogRecord logRecord, long appendLSN) {
         logRecord.writeLogRecord(appendBuffer);
-        if (logRecord.getLogType() != LogType.FLUSH) {
+        if (logRecord.getLogType() != LogType.FLUSH && logRecord.getLogType() != LogType.WAIT) {
             logRecord.getTxnCtx().setLastLSN(appendLSN);
         }
         synchronized (this) {
@@ -98,7 +98,8 @@
             if (IS_DEBUG_MODE) {
                 LOGGER.info("append()| appendOffset: " + appendOffset);
             }
-            if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+            if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+                    || logRecord.getLogType() == LogType.WAIT) {
                 logRecord.isFlushed(false);
                 syncCommitQ.offer(logRecord);
             }
@@ -114,18 +115,19 @@
     public void appendWithReplication(ILogRecord logRecord, long appendLSN) {
         logRecord.writeLogRecord(appendBuffer, appendLSN);
 
-        if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH) {
-            if (logRecord.getLogType() != LogType.FLUSH) {
-                logRecord.getTxnCtx().setLastLSN(appendLSN);
-            }
+        if (logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.FLUSH
+                && logRecord.getLogType() != LogType.WAIT) {
+            logRecord.getTxnCtx().setLastLSN(appendLSN);
         }
+
         synchronized (this) {
             appendOffset += logRecord.getLogSize();
             if (IS_DEBUG_MODE) {
                 LOGGER.info("append()| appendOffset: " + appendOffset);
             }
             if (logRecord.getLogSource() == LogSource.LOCAL) {
-                if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+                if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+                        || logRecord.getLogType() == LogType.WAIT) {
                     logRecord.isFlushed(false);
                     syncCommitQ.offer(logRecord);
                 }
@@ -265,18 +267,23 @@
                             // since this operation consisted of delete and insert, we need to notify the optracker twice
                             txnCtx.notifyOptracker(false);
                         }
+                        if (TransactionSubsystem.IS_PROFILE_MODE) {
+                            txnSubsystem.incrementEntityCommitCount();
+                        }
                     } else if (logRecord.getLogType() == LogType.JOB_COMMIT
                             || logRecord.getLogType() == LogType.ABORT) {
                         reusableJobId.setId(logRecord.getJobId());
                         txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(reusableJobId, false);
                         txnCtx.notifyOptracker(true);
-                        notifyJobTerminator();
+                        notifyJobTermination();
                     } else if (logRecord.getLogType() == LogType.FLUSH) {
-                        notifyFlushTerminator();
+                        notifyFlushTermination();
+                    } else if (logRecord.getLogType() == LogType.WAIT) {
+                        notifyWaitTermination();
                     }
                 } else if (logRecord.getLogSource() == LogSource.REMOTE) {
                     if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
-                        notifyReplicationTerminator();
+                        notifyReplicationTermination();
                     }
                 }
 
@@ -285,7 +292,15 @@
         }
     }
 
-    public void notifyJobTerminator() {
+    public void notifyJobTermination() {
+        notifyToSyncCommitQWaiter();
+    }
+
+    public void notifyWaitTermination() {
+        notifyToSyncCommitQWaiter();
+    }
+
+    public void notifyToSyncCommitQWaiter() {
         ILogRecord logRecord = null;
         while (logRecord == null) {
             try {
@@ -300,7 +315,7 @@
         }
     }
 
-    public void notifyFlushTerminator() throws ACIDException {
+    public void notifyFlushTermination() throws ACIDException {
         LogRecord logRecord = null;
         try {
             logRecord = (LogRecord) flushQ.take();
@@ -321,7 +336,7 @@
         }
     }
 
-    public void notifyReplicationTerminator() {
+    public void notifyReplicationTermination() {
         LogRecord logRecord = null;
         try {
             logRecord = (LogRecord) remoteJobsQ.take();
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
index b8ccf04..776577a 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManager.java
@@ -128,8 +128,8 @@
     protected void appendToLogTail(ILogRecord logRecord) throws ACIDException {
         syncAppendToLogTail(logRecord);
 
-        if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
-                && !logRecord.isFlushed()) {
+        if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+                || logRecord.getLogType() == LogType.WAIT) && !logRecord.isFlushed()) {
             synchronized (logRecord) {
                 while (!logRecord.isFlushed()) {
                     try {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
index efd66a8..4112529 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/logging/LogManagerWithReplication.java
@@ -42,7 +42,7 @@
         }
 
         //only locally generated logs should be replicated
-        logRecord.setReplicated(logRecord.getLogSource() == LogSource.LOCAL);
+        logRecord.setReplicated(logRecord.getLogSource() == LogSource.LOCAL && logRecord.getLogType() != LogType.WAIT);
 
         //Remote flush logs do not need to be flushed separately since they may not trigger local flush
         if (logRecord.getLogType() == LogType.FLUSH && logRecord.getLogSource() == LogSource.LOCAL) {
@@ -62,8 +62,8 @@
         }
 
         if (logRecord.getLogSource() == LogSource.LOCAL) {
-            if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
-                    && !logRecord.isFlushed()) {
+            if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT
+                    || logRecord.getLogType() == LogType.WAIT) && !logRecord.isFlushed()) {
                 synchronized (logRecord) {
                     while (!logRecord.isFlushed()) {
                         try {
@@ -74,11 +74,13 @@
                     }
 
                     //wait for job Commit/Abort ACK from replicas
-                    while (!replicationManager.hasBeenReplicated(logRecord)) {
-                        try {
-                            logRecord.wait();
-                        } catch (InterruptedException e) {
-                            //ignore
+                    if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
+                        while (!replicationManager.hasBeenReplicated(logRecord)) {
+                            try {
+                                logRecord.wait();
+                            } catch (InterruptedException e) {
+                                //ignore
+                            }
                         }
                     }
                 }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
index 9a74a18..0ec6b10 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -270,6 +270,7 @@
                     abortLogCount++;
                     break;
                 case LogType.FLUSH:
+                case LogType.WAIT:
                     break;
                 default:
                     throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
@@ -289,8 +290,8 @@
         return winnerJobSet;
     }
 
-    private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader, long lowWaterMarkLSN,
-            Set<Integer> winnerJobSet) throws IOException, ACIDException {
+    private synchronized void startRecoveryRedoPhase(Set<Integer> partitions, ILogReader logReader,
+            long lowWaterMarkLSN, Set<Integer> winnerJobSet) throws IOException, ACIDException {
         int redoCount = 0;
         int jobId = -1;
 
@@ -765,6 +766,7 @@
                         throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
                     case LogType.ABORT:
                     case LogType.FLUSH:
+                    case LogType.WAIT:
                         //ignore
                         break;
                     default:
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
index 5df230b..2e72033 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
@@ -24,11 +24,6 @@
  */
 public class TransactionManagementConstants {
 
-    public static class ResourceMgrIds {
-        public static final byte BTREE_RESOURCE_MGR_ID = 1;
-        public static final byte METADATA_RESOURCE_MGR_ID = 2;
-    }
-
     public static class LogManagerConstants {
         public static final int TERMINAL_LSN = -1;
     }
@@ -36,30 +31,28 @@
     public static class LockManagerConstants {
         public static class LockMode {
             public static final byte ANY = -1;
-            public static final byte NL  =  0;
-            public static final byte IS  =  1;
-            public static final byte IX  =  2;
-            public static final byte S   =  3;
-            public static final byte X   =  4;
-
-            public static byte intentionMode(byte mode) {
-                switch (mode) {
-                    case S:  return IS;
-                    case X:  return IX;
-                    default: throw new IllegalArgumentException(
-                            "no intention lock mode for " + toString(mode));
-                }
-            }
+            public static final byte NL = 0;
+            public static final byte IS = 1;
+            public static final byte IX = 2;
+            public static final byte S = 3;
+            public static final byte X = 4;
 
             public static String toString(byte mode) {
                 switch (mode) {
-                    case ANY: return "ANY";
-                    case NL:  return "NL";
-                    case IS:  return "IS";
-                    case IX:  return "IX";
-                    case S:   return "S";
-                    case X:   return "X";
-                    default:  throw new IllegalArgumentException("no such lock mode");
+                    case ANY:
+                        return "ANY";
+                    case NL:
+                        return "NL";
+                    case IS:
+                        return "IS";
+                    case IX:
+                        return "IX";
+                    case S:
+                        return "S";
+                    case X:
+                        return "X";
+                    default:
+                        throw new IllegalArgumentException("no such lock mode");
                 }
             }
         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
index 2112097..f035029 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -117,7 +117,7 @@
             }
             throw ae;
         } finally {
-            txnSubsystem.getLockManager().releaseLocks(txnCtx); // release
+            txnSubsystem.getLockManager().releaseLocks(txnCtx);
             transactionContextRepository.remove(txnCtx.getJobId());
             txnCtx.setTxnState(ITransactionManager.COMMITTED);
         }
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index 6650ac6..f7ed355 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.transaction.management.service.transaction;
 
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.logging.Logger;
+
 import org.apache.asterix.common.config.AsterixReplicationProperties;
 import org.apache.asterix.common.config.AsterixTransactionProperties;
 import org.apache.asterix.common.config.IAsterixPropertiesProvider;
@@ -48,6 +52,12 @@
     private final CheckpointThread checkpointThread;
     private final AsterixTransactionProperties txnProperties;
 
+    //for profiling purpose
+    public static final boolean IS_PROFILE_MODE = false;//true
+    public long profilerEntityCommitLogCount = 0;
+    private EntityCommitProfiler ecp;
+    private Future<Object> fecp;
+
     public TransactionSubsystem(String id, IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider,
             AsterixTransactionProperties txnProperties) throws ACIDException {
         this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
@@ -72,12 +82,17 @@
 
         if (asterixAppRuntimeContextProvider != null) {
             this.checkpointThread = new CheckpointThread(recoveryManager,
-                    asterixAppRuntimeContextProvider.getDatasetLifecycleManager(),logManager,
+                    asterixAppRuntimeContextProvider.getDatasetLifecycleManager(), logManager,
                     this.txnProperties.getCheckpointLSNThreshold(), this.txnProperties.getCheckpointPollFrequency());
             this.checkpointThread.start();
         } else {
             this.checkpointThread = null;
         }
+
+        if (IS_PROFILE_MODE) {
+            ecp = new EntityCommitProfiler(this, this.txnProperties.getCommitProfilerReportInterval());
+            fecp = (Future<Object>) getAsterixAppRuntimeContextProvider().getThreadExecutor().submit(ecp);
+        }
     }
 
     public ILogManager getLogManager() {
@@ -108,4 +123,63 @@
         return id;
     }
 
+    public void incrementEntityCommitCount() {
+        ++profilerEntityCommitLogCount;
+    }
+
+    /**
+     * Thread for profiling entity level commit count
+     * This thread takes a report interval (in seconds) parameter and
+     * reports entity level commit count every report interval (in seconds)
+     * only if IS_PROFILE_MODE is set to true.
+     * However, the thread doesn't start reporting the count until the entityCommitCount > 0.
+     */
+    static class EntityCommitProfiler implements Callable<Boolean> {
+        private static final Logger LOGGER = Logger.getLogger(EntityCommitProfiler.class.getName());
+        private final long reportIntervalInMillisec;
+        private long lastEntityCommitCount;
+        private int reportIntervalInSeconds;
+        private TransactionSubsystem txnSubsystem;
+        private boolean firstReport = true;
+        private long startTimeStamp = 0;
+        private long reportRound = 1;
+
+        public EntityCommitProfiler(TransactionSubsystem txnSubsystem, int reportIntervalInSeconds) {
+            Thread.currentThread().setName("EntityCommitProfiler-Thread");
+            this.txnSubsystem = txnSubsystem;
+            this.reportIntervalInSeconds = reportIntervalInSeconds;
+            this.reportIntervalInMillisec = reportIntervalInSeconds * 1000;
+            lastEntityCommitCount = txnSubsystem.profilerEntityCommitLogCount;
+        }
+
+        @Override
+        public Boolean call() throws Exception {
+            while (true) {
+                Thread.sleep(reportIntervalInMillisec);
+                if (txnSubsystem.profilerEntityCommitLogCount > 0) {
+                    if (firstReport) {
+                        startTimeStamp = System.currentTimeMillis();
+                        firstReport = false;
+                    }
+                    //output the count
+                    outputCount();
+                }
+            }
+        }
+
+        private void outputCount() {
+            long currentTimeStamp = System.currentTimeMillis();
+            long currentEntityCommitCount = txnSubsystem.profilerEntityCommitLogCount;
+
+            LOGGER.severe("EntityCommitProfiler ReportRound[" + reportRound + "], AbsoluteTimeStamp[" + currentTimeStamp
+                    + "], ActualRelativeTimeStamp[" + (currentTimeStamp - startTimeStamp)
+                    + "], ExpectedRelativeTimeStamp[" + (reportIntervalInSeconds * reportRound) + "], IIPS["
+                    + ((currentEntityCommitCount - lastEntityCommitCount) / reportIntervalInSeconds) + "], IPS["
+                    + (currentEntityCommitCount / (reportRound * reportIntervalInSeconds)) + "]");
+
+            lastEntityCommitCount = currentEntityCommitCount;
+            ++reportRound;
+        }
+    }
+
 }
diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java
index 5cabd04..14e4020 100644
--- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java
+++ b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/LockManagerUnitTest.java
@@ -19,15 +19,8 @@
 
 package org.apache.asterix.transaction.management.service.locking;
 
-
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.DatasetId;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.apache.asterix.common.transactions.JobId;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.PrintStream;
 import java.util.ArrayList;
@@ -40,10 +33,15 @@
 import java.util.logging.ConsoleHandler;
 import java.util.logging.Logger;
 
-import static org.apache.asterix.transaction.management.service.locking.Request.Kind;
-import static org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import org.apache.asterix.common.transactions.DatasetId;
+import org.apache.asterix.common.transactions.ILockManager;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.apache.asterix.common.transactions.JobId;
+import org.apache.asterix.transaction.management.service.locking.Request.Kind;
+import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
 
 public class LockManagerUnitTest {
 
@@ -59,7 +57,7 @@
         Logger.getLogger(ConcurrentLockManager.class.getName()).addHandler(new ConsoleHandler());
     }
 
-    Map<Integer, ITransactionContext> jobMap;
+    Map<Integer, ITransactionContext> jobId2TxnCtxMap;
     ILockManager lockMgr;
 
     // set to e.g. System.err to get some output
@@ -72,14 +70,14 @@
 
     @Before
     public void setUp() throws Exception {
-        jobMap = new HashMap<>();
+        jobId2TxnCtxMap = new HashMap<>();
         lockMgr = new ConcurrentLockManager(LOCK_MGR_SHRINK_TIMER, LOCK_MGR_ARENAS, LOCK_MGR_TABLE_SIZE);
     }
 
     @After
     public void tearDown() throws Exception {
         lockMgr = null;
-        jobMap = null;
+        jobId2TxnCtxMap = null;
     }
 
     @Test
@@ -87,7 +85,6 @@
         List<Request> reqs = new ArrayList<>();
         reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
         reqs.add(req(Kind.UNLOCK, j(1), d(1), e(1), LockMode.S));
-        reqs.add(req(Kind.UNLOCK, j(1), d(1), e(-1), LockMode.IS));
         reportErrors(execute(reqs));
     }
 
@@ -139,55 +136,16 @@
     }
 
     @Test
-    public void testDeadlock() throws Exception {
-        List<Request> reqs = new ArrayList<>();
-        reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
-        reqs.add(req(Kind.LOCK, j(2), d(1), e(2), LockMode.X));
-        reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.X));
-        reqs.add(req(Kind.LOCK, j(1), d(1), e(2), LockMode.X));
-        reqs.add(req(Kind.RELEASE, j(1)));
-        reqs.add(req(Kind.RELEASE, j(2)));
-        expectError(execute(reqs), j(1), ACIDException.class);
-    }
-
-    @Test
+    /**
+     * lock conversion/upgrade is not supported when deadlock-free locking
+     * protocol is enabled.
+     */
     public void testUpgrade() throws Exception {
         List<Request> reqs = new ArrayList<>();
         reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
         reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
         reqs.add(req(Kind.RELEASE, j(1)));
-        reportErrors(execute(reqs));
-    }
-
-    @Test
-    public void testUpgradeDeadlock() throws Exception {
-        List<Request> reqs = new ArrayList<>();
-        reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
-        reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.S));
-        reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
-        reqs.add(req(Kind.PRINT));
-        reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.X));
-        reqs.add(req(Kind.RELEASE, j(1)));
-        reqs.add(req(Kind.RELEASE, j(2)));
-        expectError(execute(reqs), j(2), ACIDException.class);
-    }
-
-    @Test
-    /**
-     * Runs into a time-out and j(1) gets interrupted by
-     * the test. This scenario happens only in this test as there
-     * is additional synchronization between the locking threads
-     * through the coordinator.
-     */
-    public void testTimeout() throws Exception {
-        List<Request> reqs = new ArrayList<>();
-        reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.S));
-        reqs.add(req(Kind.LOCK, j(2), d(1), e(1), LockMode.S));
-        reqs.add(req(Kind.LOCK, j(1), d(1), e(1), LockMode.X));
-        reqs.add(req(Kind.RELEASE, j(1)));
-        reqs.add(req(Kind.RELEASE, j(2)));
-        // this runs into a time-out and j(1) gets interrupted
-        expectError(execute(reqs), j(1), WaitInterruptedException.class);
+        expectError(execute(reqs), j(1), IllegalStateException.class);
     }
 
     //--------------------------------------------------------------------
@@ -202,9 +160,10 @@
      * LockManager in list order, however they are fulfilled in the order
      * decided by the LockManager
      *
-     * @param reqs a list of requests that will be execute in order
+     * @param reqs
+     *            a list of requests that will be execute in order
      * @return a map of (JodId, exception) pairs that can either be handled
-     * by the test or thrown using #reportErrors
+     *         by the test or thrown using #reportErrors
      */
     private Map<String, Throwable> execute(List<Request> reqs) throws InterruptedException {
         if (err != null) {
@@ -258,7 +217,7 @@
     private Set<Locker> createLockers(List<Request> reqs, AtomicInteger timeStamp) {
         Set<Locker> lockers = new HashSet<>();
         lockers.add(new Locker(lockMgr, null, reqs, timeStamp, err));
-        for (ITransactionContext txnCtx : jobMap.values()) {
+        for (ITransactionContext txnCtx : jobId2TxnCtxMap.values()) {
             Locker locker = new Locker(lockMgr, txnCtx, reqs, timeStamp, err);
             lockers.add(locker);
         }
@@ -275,8 +234,8 @@
         return threads;
     }
 
-    private Map<String, Throwable> stopThreads(Set<Locker> lockers, Map<String, Thread> threads) throws
-            InterruptedException {
+    private Map<String, Throwable> stopThreads(Set<Locker> lockers, Map<String, Thread> threads)
+            throws InterruptedException {
         Map<String, Throwable> result = new HashMap<>();
         for (Locker locker : lockers) {
             stopThread(threads.get(locker.name));
@@ -317,7 +276,8 @@
      * throws the first Throwable found in the map.
      * This is the default way to handle the errors returned by #execute
      *
-     * @param errors a map of (JodId, exception) pairs
+     * @param errors
+     *            a map of (JodId, exception) pairs
      */
     void reportErrors(Map<String, Throwable> errors) {
         for (String name : errors.keySet()) {
@@ -333,8 +293,10 @@
     /**
      * gets the error for a specific job from the errors map
      *
-     * @param errors a map of (JodId, throwable) pairs
-     * @param txnCtx the transaction context of the job whose error is requested
+     * @param errors
+     *            a map of (JodId, throwable) pairs
+     * @param txnCtx
+     *            the transaction context of the job whose error is requested
      * @return throwable for said error
      */
     private static Throwable getError(Map<String, Throwable> errors, ITransactionContext txnCtx) {
@@ -344,16 +306,19 @@
     /**
      * asserts that the error for a specific job from the errors map is of a specific class
      *
-     * @param errors a map of (JodId, throwable) pairs
-     * @param txnCtx the transaction context of the job whose error is requested
-     * @param clazz  the exception class
+     * @param errors
+     *            a map of (JodId, throwable) pairs
+     * @param txnCtx
+     *            the transaction context of the job whose error is requested
+     * @param clazz
+     *            the exception class
      */
     private void expectError(Map<String, Throwable> errors, ITransactionContext txnCtx,
-                             Class<? extends Throwable> clazz) throws Exception {
+            Class<? extends Throwable> clazz) throws Exception {
         Throwable error = getError(errors, txnCtx);
         if (error == null) {
-            throw new AssertionError("expected " + clazz.getSimpleName() + " for " + txnCtx.getJobId() + ", got no " +
-                    "exception");
+            throw new AssertionError(
+                    "expected " + clazz.getSimpleName() + " for " + txnCtx.getJobId() + ", got no " + "exception");
         }
         if (!clazz.isInstance(error)) {
             throw new AssertionError(error);
@@ -365,8 +330,8 @@
     // Convenience methods to make test description more compact
     //--------------------------------------------------------------------
 
-    private Request req(final Kind kind, final ITransactionContext txnCtx,
-                        final DatasetId dsId, final int hashValue, final byte lockMode) {
+    private Request req(final Kind kind, final ITransactionContext txnCtx, final DatasetId dsId, final int hashValue,
+            final byte lockMode) {
         return Request.create(kind, txnCtx, dsId, hashValue, lockMode);
     }
 
@@ -387,12 +352,11 @@
     }
 
     private ITransactionContext j(int jId) {
-        if (!jobMap.containsKey(jId)) {
+        if (!jobId2TxnCtxMap.containsKey(jId)) {
             ITransactionContext mockTxnContext = mock(ITransactionContext.class);
             when(mockTxnContext.getJobId()).thenReturn(new JobId(jId));
-            jobMap.put(jId, mockTxnContext);
+            jobId2TxnCtxMap.put(jId, mockTxnContext);
         }
-        return jobMap.get(jId);
+        return jobId2TxnCtxMap.get(jId);
     }
-
 }
diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java
index 164fc07..97b4f8a 100644
--- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java
+++ b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Locker.java
@@ -18,11 +18,6 @@
 */
 package org.apache.asterix.transaction.management.service.locking;
 
-import org.apache.asterix.common.exceptions.ACIDException;
-import org.apache.asterix.common.transactions.ILockManager;
-import org.apache.asterix.common.transactions.ITransactionContext;
-import org.junit.Assert;
-
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -31,6 +26,11 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.transactions.ILockManager;
+import org.apache.asterix.common.transactions.ITransactionContext;
+import org.junit.Assert;
+
 /**
  * Executes a sequence of lock requests against an ILockManager.
  * Lockers are run by different threads in the LockManagerUnitTest.
@@ -55,18 +55,22 @@
     private PrintStream err;
 
     /**
-     * @param lockMgr the ILockManager to send requests to
-     * @param txnCtx the ITransactionContext that identifies the transaction that this Locker represents
-     * @param allRequests an ordered list of lock requests for multiple transactions, this Locker will only execute
-     *                    requests for the transaction identified by txnCtx
-     * @param time a global timestamp that is used to synchronize different lockers to ensure that requests are started
-     *             in the order given in allRequests
-     * @param err a stream to write log/error information to
-     *
+     * @param lockMgr
+     *            the ILockManager to send requests to
+     * @param txnCtx
+     *            the ITransactionContext that identifies the transaction that this Locker represents
+     * @param allRequests
+     *            an ordered list of lock requests for multiple transactions, this Locker will only execute
+     *            requests for the transaction identified by txnCtx
+     * @param time
+     *            a global timestamp that is used to synchronize different lockers to ensure that requests are started
+     *            in the order given in allRequests
+     * @param err
+     *            a stream to write log/error information to
      * @see Request
      */
     Locker(ILockManager lockMgr, ITransactionContext txnCtx, List<Request> allRequests, AtomicInteger time,
-           PrintStream err) {
+            PrintStream err) {
         this.name = txnCtx == null ? "admin" : txnCtx.getJobId().toString();
         this.lockMgr = lockMgr;
 
@@ -110,7 +114,7 @@
     public void run() {
         log("running");
         try {
-            while (! hasErrors() && reqIter.hasNext()) {
+            while (!hasErrors() && reqIter.hasNext()) {
                 curReq = reqIter.next();
                 int localTime = globalTime.get();
                 while (localTime < curReq.time) {
@@ -123,13 +127,14 @@
                 log("will exec at t=" + localTime + " " + curReq);
                 try {
                     reqStart = currentTime();
-                    Assert.assertEquals(localTime, globalTime.getAndIncrement());
-                    log("incremented");
+                    Assert.assertEquals(localTime, globalTime.get());
                     curReq.setResult(curReq.request.execute(lockMgr) ? Requester.SUCCESS : Requester.FAIL);
                 } catch (ACIDException e) {
                     curReq.setResult(Requester.ERROR);
                     addError(e);
                 } finally {
+                    globalTime.getAndIncrement();
+                    log("incremented");
                     reqStart = -1;
                 }
                 log("time " + localTime);
diff --git a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java
index c7e0c42..70a803b 100644
--- a/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java
+++ b/asterixdb/asterix-transactions/src/test/java/org/apache/asterix/transaction/management/service/locking/Request.java
@@ -18,14 +18,14 @@
 */
 package org.apache.asterix.transaction.management.service.locking;
 
+import java.io.PrintStream;
+
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.transactions.DatasetId;
 import org.apache.asterix.common.transactions.ILockManager;
 import org.apache.asterix.common.transactions.ITransactionContext;
 import org.apache.asterix.transaction.management.service.transaction.TransactionManagementConstants;
 
-import java.io.PrintStream;
-
 /**
  * repesents a lock request for testing.
  */
@@ -56,16 +56,17 @@
         this.txnCtx = txnCtx;
     }
 
-    String asString(final Kind kind, final ITransactionContext txnCtx,
-                    final DatasetId dsId, final int hashValue, final byte lockMode) {
-        return txnCtx.getJobId().toString() + ":" + kind.name() + ":" + dsId.getId() + ":" + hashValue + ":"
+    String asString(final Kind kind, final ITransactionContext txnCtx, final DatasetId dsId, final int hashValue,
+            final byte lockMode) {
+        return txnCtx.getJobId() + ":" + kind.name() + ":" + dsId.getId() + ":" + hashValue + ":"
                 + TransactionManagementConstants.LockManagerConstants.LockMode.toString(lockMode);
     }
 
     abstract boolean execute(ILockManager lockMgr) throws ACIDException;
 
-    static Request create(final Kind kind, final ITransactionContext txnCtx,
-                          final DatasetId dsId, final int hashValue, final byte lockMode) {
+    static Request create(final Kind kind, final ITransactionContext txnCtx, final DatasetId dsId, final int hashValue,
+            final byte lockMode) {
+
         switch (kind) {
             case INSTANT_TRY_LOCK:
                 return new Request(kind, txnCtx) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
index 6fc26a7..df3a211 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-btree/src/test/java/org/apache/hyracks/storage/am/btree/test/FramewriterTest.java
@@ -442,7 +442,7 @@
     private ISearchOperationCallbackFactory[] mockSearchOpCallbackFactories() throws HyracksDataException {
         ISearchOperationCallback searchOpCallback = mockSearchOpCallback();
         ISearchOperationCallbackFactory searchOpCallbackFactory = Mockito.mock(ISearchOperationCallbackFactory.class);
-        Mockito.when(searchOpCallbackFactory.createSearchOperationCallback(Mockito.anyLong(), Mockito.any()))
+        Mockito.when(searchOpCallbackFactory.createSearchOperationCallback(Mockito.anyLong(), Mockito.any(), null))
                 .thenReturn(searchOpCallback);
         return new ISearchOperationCallbackFactory[] { searchOpCallbackFactory };
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
index 8d618c8..05114f0 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/IModificationOperationCallbackFactory.java
@@ -21,9 +21,11 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IModificationOperationCallbackFactory extends Serializable {
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException;
+            int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+            throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
index 1a629e9..c0d5128 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/api/ISearchOperationCallbackFactory.java
@@ -21,9 +21,10 @@
 import java.io.Serializable;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface ISearchOperationCallbackFactory extends Serializable {
-    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
             throws HyracksDataException;
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
index 9438632..33bad73 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexInsertUpdateDeleteOperatorNodePushable.java
@@ -78,7 +78,7 @@
             writer.open();
             modCallback = opDesc.getModificationOpCallbackFactory().createModificationOperationCallback(
                     indexHelper.getResourcePath(), indexHelper.getResourceID(), indexHelper.getResourcePartition(),
-                    index, ctx);
+                    index, ctx, this);
             indexAccessor = index.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
             ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
             if (tupleFilterFactory != null) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
index c4f3fca..650dcfa 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/IndexSearchOperatorNodePushable.java
@@ -133,7 +133,7 @@
             dos = tb.getDataOutput();
             appender = new FrameTupleAppender(new VSizeFrame(ctx), true);
             ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
-                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx);
+                    .createSearchOperationCallback(indexHelper.getResourceID(), ctx, null);
             indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE, searchCallback);
             cursor = createCursor();
             if (retainInput) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
index 170950e..3ac6065 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/dataflow/TreeIndexDiskOrderScanOperatorNodePushable.java
@@ -57,7 +57,7 @@
             ITreeIndexFrame cursorFrame = treeIndex.getLeafFrameFactory().createFrame();
             ITreeIndexCursor cursor = treeIndexHelper.createDiskOrderScanCursor(cursorFrame);
             ISearchOperationCallback searchCallback = opDesc.getSearchOpCallbackFactory()
-                    .createSearchOperationCallback(treeIndexHelper.getResourceID(), ctx);
+                    .createSearchOperationCallback(treeIndexHelper.getResourceID(), ctx, null);
             ITreeIndexAccessor indexAccessor = (ITreeIndexAccessor) treeIndex
                     .createAccessor(NoOpOperationCallback.INSTANCE, searchCallback);
             try {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
index b42e619..c2cca41 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-common/src/main/java/org/apache/hyracks/storage/am/common/impls/NoOpOperationCallbackFactory.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.storage.am.common.impls;
 
 import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallback;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
@@ -30,17 +31,18 @@
  * Implemented as an enum to preserve singleton model while being serializable
  */
 public enum NoOpOperationCallbackFactory
-    implements ISearchOperationCallbackFactory,IModificationOperationCallbackFactory {
-        INSTANCE;
+        implements ISearchOperationCallbackFactory, IModificationOperationCallbackFactory {
+    INSTANCE;
 
     @Override
-    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx) {
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable) {
         return NoOpOperationCallback.INSTANCE;
     }
 
     @Override
     public IModificationOperationCallback createModificationOperationCallback(String resourcePath, long resourceId,
-            int resourcePartition, Object resource, IHyracksTaskContext ctx) throws HyracksDataException {
+            int resourcePartition, Object resource, IHyracksTaskContext ctx, IOperatorNodePushable operatorNodePushable)
+            throws HyracksDataException {
         return NoOpOperationCallback.INSTANCE;
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFrameWriter.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFrameWriter.java
new file mode 100644
index 0000000..47c1854
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIndexFrameWriter.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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 at
+ *
+ *   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 org.apache.hyracks.storage.am.lsm.common.api;
+
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface ILSMIndexFrameWriter extends IFrameWriter {
+    /**
+     * Push tuples in the frame from startTupleIndex(inclusive) to endTupleIndex(exclusive)
+     * forward to the next operator/consumer.
+     *
+     * @param startTupleIndex
+     *            the first tuple's index to be pushed
+     * @param endTupleIndex
+     *            the last tuple's index to be pushed
+     * @throws HyracksDataException
+     */
+    public default void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
+        throw new HyracksDataException("flushPartialFrame() is not supported in this ILSMIndexFrameWriter");
+    }
+
+    /**
+     * Push tuples in the frame forward to the next operator/consumer.
+     * The flushed tuples don't have to be all tuples in the frame.
+     *
+     * @throws HyracksDataException
+     */
+    public default void flushPartialFrame() throws HyracksDataException {
+        throw new HyracksDataException("flushPartialFrame() is not supported in this ILSMIndexFrameWriter");
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
index 25f2382..de1719c 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/dataflow/LSMIndexInsertUpdateDeleteOperatorNodePushable.java
@@ -29,8 +29,10 @@
 import org.apache.hyracks.storage.am.common.dataflow.IndexInsertUpdateDeleteOperatorNodePushable;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexFrameWriter;
 
-public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertUpdateDeleteOperatorNodePushable {
+public class LSMIndexInsertUpdateDeleteOperatorNodePushable extends IndexInsertUpdateDeleteOperatorNodePushable
+        implements ILSMIndexFrameWriter {
 
     protected FrameTupleAppender appender;
 
@@ -116,7 +118,8 @@
         }
     }
 
-    private void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
+    @Override
+    public void flushPartialFrame(int startTupleIndex, int endTupleIndex) throws HyracksDataException {
         for (int i = startTupleIndex; i < endTupleIndex; i++) {
             FrameUtils.appendToWriter(writer, appender, accessor, i);
         }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
index aad5bf47..7340fdb 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ConstantMergePolicy.java
@@ -131,7 +131,6 @@
      * This method returns whether there is an ongoing merge operation or not by checking
      * each component state of given components.
      *
-     * @param immutableComponents
      * @return true if there is an ongoing merge operation, false otherwise.
      */
     private boolean isMergeOngoing(List<ILSMComponent> immutableComponents) {