merged asterix_stabilization -r722:751 to asterix_lsm_stabilization

git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_lsm_stabilization@756 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index ae2559b..c2bb117 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -3,6 +3,7 @@
 import java.util.ArrayList;
 import java.util.List;
 
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -43,11 +44,11 @@
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
  * Contributes the runtime operator for an unnest-map representing an inverted-index search.
@@ -164,12 +165,14 @@
                 searchModifierType, simThresh, secondaryIndex);
         IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(
                 searchModifierType, searchKeyType, secondaryIndex);
-        InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(jobSpec,
-                queryField, appContext.getStorageManagerInterface(), fileSplitProviders.first,
-                fileSplitProviders.second, appContext.getIndexRegistryProvider(), tokenTypeTraits,
-                tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
-                new BTreeDataflowHelperFactory(), queryTokenizerFactory, searchModifierFactory, outputRecDesc,
-                retainInput, NoOpOperationCallbackProvider.INSTANCE);
+        LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+                jobSpec, queryField, appContext.getStorageManagerInterface(), fileSplitProviders.first,
+                appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
+                invListsTypeTraits, invListsComparatorFactories, new LSMInvertedIndexDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
+                queryTokenizerFactory, searchModifierFactory, outputRecDesc, retainInput,
+                NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp,
                 secondarySplitsAndConstraint.second);
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 0f28239..09ef774 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -31,6 +31,7 @@
 import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceTransactionCommitByAssignOpRule;
 import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
 import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
 import edu.uci.ics.asterix.optimizer.rules.PullPositionalVariableFromUnnestRule;
@@ -195,6 +196,8 @@
     public final static List<IAlgebraicRewriteRule> buildPhysicalRewritesAllLevelsRuleCollection() {
         List<IAlgebraicRewriteRule> physicalRewritesAllLevels = new LinkedList<IAlgebraicRewriteRule>();
         physicalRewritesAllLevels.add(new PullSelectOutOfEqJoin());
+        //Turned off the following rule for now not to change OptimizerTest results.
+        //physicalRewritesAllLevels.add(new IntroduceTransactionCommitByAssignOpRule());        
         physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
@@ -203,6 +206,7 @@
         physicalRewritesAllLevels.add(new PullPositionalVariableFromUnnestRule());
         physicalRewritesAllLevels.add(new PushProjectDownRule());
         physicalRewritesAllLevels.add(new InsertProjectBeforeUnionRule());
+
         return physicalRewritesAllLevels;
     }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
index e700971..4b2a2ab 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -264,7 +264,7 @@
         // The translator will compile metadata internally. Run this compilation
         // under the same transaction id as the "outer" compilation.
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(mp.getTxnId(),
+        AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(mp.getJobId(),
                 metadata.getMetadataTransactionContext(), counter, null);
 
         LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
new file mode 100644
index 0000000..d635942
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceTransactionCommitByAssignOpRule.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceTransactionCommitByAssignOpRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+        SelectOperator selectOperator = (SelectOperator) op;
+
+        Mutable<ILogicalOperator> childOfSelect = selectOperator.getInputs().get(0);
+
+        //[Direction] SelectOp(cond1)<--ChildOps... ==> SelectOp(booleanValue of cond1)<--NewAssignOp(cond1)<--ChildOps... 
+        //#. Create an assign-operator with a new local variable and the condition of the select-operator. 
+        //#. Set the input(child operator) of the new assign-operator to input(child operator) of the select-operator.
+        //	 (Later, the newly created assign-operator will apply the condition on behalf of the select-operator, 
+        //    and set the variable of the assign-operator to a boolean value according to the condition evaluation.)
+        //#. Give the select-operator the result boolean value created by the newly created child assign-operator.
+
+        //create an assignOp with a variable and the condition of the select-operator. 
+        LogicalVariable v = context.newVar();
+        AssignOperator assignOperator = new AssignOperator(v, new MutableObject<ILogicalExpression>(selectOperator
+                .getCondition().getValue()));
+
+        //set the input of the new assign-operator to the input of the select-operator.
+        assignOperator.getInputs().add(childOfSelect);
+        
+        //set the result value of the assign-operator to the condition of the select-operator
+        selectOperator.getCondition().setValue(new VariableReferenceExpression(v));//scalarFunctionCallExpression);
+        selectOperator.getInputs().set(0, new MutableObject<ILogicalOperator>(assignOperator));
+
+        context.computeAndSetTypeEnvironmentForOperator(assignOperator);
+
+        //Once this rule is fired, don't apply again.
+        context.addToDontApplySet(this, selectOperator);
+        return true;
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index dbd92c0..a0f4759 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -58,11 +58,11 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.ConjunctiveSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.EditDistanceSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.searchmodifiers.JaccardSearchModifierFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.ConjunctiveSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.EditDistanceSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.search.JaccardSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
  * Class for helping rewrite rules to choose and apply inverted indexes.
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index e5712df..2e0a09b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -97,6 +97,7 @@
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
@@ -155,1521 +156,1268 @@
  * source for the current subtree.
  */
 
-public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator
-		implements
-		IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator implements
+        IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
 
-	private final MetadataTransactionContext mdTxnCtx;
-	private final long txnId;
-	private TranslationContext context;
-	private String outputDatasetName;
-	private Statement.Kind dmlKind;
-	private static AtomicLong outputFileID = new AtomicLong(0);
-	private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
+    private final MetadataTransactionContext mdTxnCtx;
+    private final JobId jobId;
+    private TranslationContext context;
+    private String outputDatasetName;
+    private Statement.Kind dmlKind;
+    private static AtomicLong outputFileID = new AtomicLong(0);
+    private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
 
-	private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+    private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-	public AqlExpressionToPlanTranslator(long txnId,
-			MetadataTransactionContext mdTxnCtx, int currentVarCounter,
-			String outputDatasetName, Statement.Kind dmlKind) {
-		this.mdTxnCtx = mdTxnCtx;
-		this.txnId = txnId;
-		this.context = new TranslationContext(new Counter(currentVarCounter));
-		this.outputDatasetName = outputDatasetName;
-		this.dmlKind = dmlKind;
-	}
+    public AqlExpressionToPlanTranslator(JobId jobId, MetadataTransactionContext mdTxnCtx, int currentVarCounter,
+            String outputDatasetName, Statement.Kind dmlKind) {
+        this.mdTxnCtx = mdTxnCtx;
+        this.jobId = jobId;
+        this.context = new TranslationContext(new Counter(currentVarCounter));
+        this.outputDatasetName = outputDatasetName;
+        this.dmlKind = dmlKind;
+    }
 
-	public int getVarCounter() {
-		return context.getVarCounter();
-	}
+    public int getVarCounter() {
+        return context.getVarCounter();
+    }
 
-	public ILogicalPlanAndMetadata translate(Query expr,
-			AqlCompiledMetadataDeclarations compiledDeclarations)
-			throws AlgebricksException, AsterixException {
-		if (expr == null) {
-			return null;
-		}
-		if (compiledDeclarations == null) {
-			compiledDeclarations = compileMetadata(mdTxnCtx,
-					expr.getPrologDeclList(), true);
-		}
-		if (!compiledDeclarations.isConnectedToDataverse())
-			compiledDeclarations.connectToDataverse(compiledDeclarations
-					.getDataverseName());
-		IDataFormat format = compiledDeclarations.getFormat();
-		if (format == null) {
-			throw new AlgebricksException("Data format has not been set.");
-		}
-		format.registerRuntimeFunctions();
-		Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-				new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator()));
+    public ILogicalPlanAndMetadata translate(Query expr, AqlCompiledMetadataDeclarations compiledDeclarations)
+            throws AlgebricksException, AsterixException {
+        if (expr == null) {
+            return null;
+        }
+        if (compiledDeclarations == null) {
+            compiledDeclarations = compileMetadata(mdTxnCtx, expr.getPrologDeclList(), true);
+        }
+        if (!compiledDeclarations.isConnectedToDataverse())
+            compiledDeclarations.connectToDataverse(compiledDeclarations.getDataverseName());
+        IDataFormat format = compiledDeclarations.getFormat();
+        if (format == null) {
+            throw new AlgebricksException("Data format has not been set.");
+        }
+        format.registerRuntimeFunctions();
+        Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
+                new EmptyTupleSourceOperator()));
 
-		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-		boolean isTransactionalWrite = false;
-		ILogicalOperator topOp = p.first;
-		ProjectOperator project = (ProjectOperator) topOp;
-		LogicalVariable resVar = project.getVariables().get(0);
-		if (outputDatasetName == null) {
-			FileSplit outputFileSplit = compiledDeclarations.getOutputFile();
-			if (outputFileSplit == null) {
-				outputFileSplit = getDefaultOutputFileLocation();
-			}
-			compiledDeclarations.setOutputFile(outputFileSplit);
-			List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			writeExprList.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(resVar)));
-			FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
-			FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
-			topOp = new WriteOperator(writeExprList, sink);
-			topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
-		} else {
-			String dataVerseName = compiledDeclarations.getDataverseName();
-			Dataset dataset = compiledDeclarations
-					.findDataset(outputDatasetName);
-			if (dataset == null) {
-				throw new AlgebricksException("Cannot find dataset "
-						+ outputDatasetName);
-			}
+        boolean isTransactionalWrite = false;
+        ILogicalOperator topOp = p.first;
+        ProjectOperator project = (ProjectOperator) topOp;
+        LogicalVariable resVar = project.getVariables().get(0);
+        if (outputDatasetName == null) {
+            FileSplit outputFileSplit = compiledDeclarations.getOutputFile();
+            if (outputFileSplit == null) {
+                outputFileSplit = getDefaultOutputFileLocation();
+            }
+            compiledDeclarations.setOutputFile(outputFileSplit);
+            List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+            FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
+            FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+            topOp = new WriteOperator(writeExprList, sink);
+            topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+        } else {
+            String dataVerseName = compiledDeclarations.getDataverseName();
+            Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+            if (dataset == null) {
+                throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
+            }
 
-			AqlSourceId sourceId = new AqlSourceId(dataVerseName,
-					outputDatasetName);
-			String itemTypeName = dataset.getItemTypeName();
-			IAType itemType = compiledDeclarations.findType(itemTypeName);
-			AqlDataSource dataSource = new AqlDataSource(sourceId, dataset,
-					itemType);
-			if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-				throw new AlgebricksException(
-						"Cannot write output to an external dataset.");
-			}
-			ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-			ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
-			List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
+            AqlSourceId sourceId = new AqlSourceId(dataVerseName, outputDatasetName);
+            String itemTypeName = dataset.getItemTypeName();
+            IAType itemType = compiledDeclarations.findType(itemTypeName);
+            AqlDataSource dataSource = new AqlDataSource(sourceId, dataset, itemType);
+            if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+                throw new AlgebricksException("Cannot write output to an external dataset.");
+            }
+            ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+            ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
+            List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
 
-			List<String> partitionKeys = DatasetUtils
-					.getPartitioningKeys(dataset);
-			for (String keyFieldName : partitionKeys) {
-				IFunctionInfo finfoAccess = AsterixBuiltinFunctions
-						.getAsterixFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
-				@SuppressWarnings("unchecked")
-				ScalarFunctionCallExpression f = new ScalarFunctionCallExpression(
-						finfoAccess, new MutableObject<ILogicalExpression>(
-								new VariableReferenceExpression(
-										METADATA_DUMMY_VAR)),
-						new MutableObject<ILogicalExpression>(
-								new ConstantExpression(
-										new AsterixConstantValue(new AString(
-												keyFieldName)))));
-				f.substituteVar(METADATA_DUMMY_VAR, resVar);
-				exprs.add(new MutableObject<ILogicalExpression>(f));
-				LogicalVariable v = context.newVar();
-				vars.add(v);
-				varRefsForLoading.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(v)));
-			}
-			AssignOperator assign = new AssignOperator(vars, exprs);
-			assign.getInputs()
-					.add(new MutableObject<ILogicalOperator>(project));
+            List<String> partitionKeys = DatasetUtils.getPartitioningKeys(dataset);
+            for (String keyFieldName : partitionKeys) {
+                IFunctionInfo finfoAccess = AsterixBuiltinFunctions
+                        .getAsterixFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
+                @SuppressWarnings("unchecked")
+                ScalarFunctionCallExpression f = new ScalarFunctionCallExpression(finfoAccess,
+                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
+                        new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                                new AString(keyFieldName)))));
+                f.substituteVar(METADATA_DUMMY_VAR, resVar);
+                exprs.add(new MutableObject<ILogicalExpression>(f));
+                LogicalVariable v = context.newVar();
+                vars.add(v);
+                varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            }
+            AssignOperator assign = new AssignOperator(vars, exprs);
+            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
 
-			Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(resVar));
-			ILogicalOperator load = null;
+            Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                    resVar));
+            ILogicalOperator load = null;
 
-			switch (dmlKind) {
-			case WRITE_FROM_QUERY_RESULT: {
-				load = new WriteResultOperator(dataSource, varRef,
-						varRefsForLoading);
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				break;
-			}
-			case INSERT: {
-				ILogicalOperator insertOp = new InsertDeleteOperator(
-						dataSource, varRef, varRefsForLoading,
-						InsertDeleteOperator.Kind.INSERT);
-				insertOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				load = new SinkOperator();
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(insertOp));
-				isTransactionalWrite = true;
-				break;
-			}
-			case DELETE: {
-				ILogicalOperator deleteOp = new InsertDeleteOperator(
-						dataSource, varRef, varRefsForLoading,
-						InsertDeleteOperator.Kind.DELETE);
-				deleteOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				load = new SinkOperator();
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(deleteOp));
-				isTransactionalWrite = true;
-				break;
-			}
-			case BEGIN_FEED: {
-				ILogicalOperator insertOp = new InsertDeleteOperator(
-						dataSource, varRef, varRefsForLoading,
-						InsertDeleteOperator.Kind.INSERT);
-				insertOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				load = new SinkOperator();
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(insertOp));
-				isTransactionalWrite = false;
-				break;
-			}
-			}
-			topOp = load;
-		}
+            switch (dmlKind) {
+                case WRITE_FROM_QUERY_RESULT: {
+                    load = new WriteResultOperator(dataSource, varRef, varRefsForLoading);
+                    load.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    break;
+                }
+                case INSERT: {
+                    ILogicalOperator insertOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
+                            InsertDeleteOperator.Kind.INSERT);
+                    insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    load = new SinkOperator();
+                    load.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+                    isTransactionalWrite = true;
+                    break;
+                }
+                case DELETE: {
+                    ILogicalOperator deleteOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
+                            InsertDeleteOperator.Kind.DELETE);
+                    deleteOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    load = new SinkOperator();
+                    load.getInputs().add(new MutableObject<ILogicalOperator>(deleteOp));
+                    isTransactionalWrite = true;
+                    break;
+                }
+                case BEGIN_FEED: {
+                    ILogicalOperator insertOp = new InsertDeleteOperator(dataSource, varRef, varRefsForLoading,
+                            InsertDeleteOperator.Kind.INSERT);
+                    insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    load = new SinkOperator();
+                    load.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+                    isTransactionalWrite = false;
+                    break;
+                }
+            }
+            topOp = load;
+        }
 
-		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-		AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
-				isTransactionalWrite, compiledDeclarations);
-		ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
-				plan, metadataProvider);
-		return planAndMetadata;
-	}
+        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+        AqlMetadataProvider metadataProvider = new AqlMetadataProvider(jobId, isTransactionalWrite,
+                compiledDeclarations);
+        ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(plan, metadataProvider);
+        return planAndMetadata;
+    }
 
-	private FileSplit getDefaultOutputFileLocation() throws MetadataException {
-		if (AsterixProperties.INSTANCE.getOutputDir() == null) {
-			throw new MetadataException(
-					"Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
-		}
-		String filePath = AsterixProperties.INSTANCE.getOutputDir()
-				+ System.getProperty("file.separator") + OUTPUT_FILE_PREFIX
-				+ outputFileID.incrementAndGet();
-		return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(),
-				new FileReference(new File(filePath)));
-	}
+    private FileSplit getDefaultOutputFileLocation() throws MetadataException {
+        if (AsterixProperties.INSTANCE.getOutputDir() == null) {
+            throw new MetadataException(
+                    "Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
+        }
+        String filePath = AsterixProperties.INSTANCE.getOutputDir() + System.getProperty("file.separator")
+                + OUTPUT_FILE_PREFIX + outputFileID.incrementAndGet();
+        return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(), new FileReference(new File(filePath)));
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v = context.newVar(fc.getVarExpr());
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar(fc.getVarExpr());
 
-		Expression inExpr = fc.getInExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				inExpr, tupSource);
-		ILogicalOperator returnedOp;
+        Expression inExpr = fc.getInExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
+        ILogicalOperator returnedOp;
 
-		if (fc.getPosVarExpr() == null) {
-			returnedOp = new UnnestOperator(v,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo.first)));
-		} else {
-			LogicalVariable pVar = context.newVar(fc.getPosVarExpr());
-			returnedOp = new UnnestOperator(v,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo.first)), pVar,
-					BuiltinType.AINT32);
-		}
-		returnedOp.getInputs().add(eo.second);
+        if (fc.getPosVarExpr() == null) {
+            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)));
+        } else {
+            LogicalVariable pVar = context.newVar(fc.getPosVarExpr());
+            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)),
+                    pVar, BuiltinType.AINT32);
+        }
+        returnedOp.getInputs().add(eo.second);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v;
-		ILogicalOperator returnedOp;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v;
+        ILogicalOperator returnedOp;
 
-		switch (lc.getBindingExpr().getKind()) {
-		case VARIABLE_EXPRESSION: {
-			v = context.newVar(lc.getVarExpr());
-			LogicalVariable prev = context.getVar(((VariableExpr) lc
-					.getBindingExpr()).getVar().getId());
-			returnedOp = new AssignOperator(v,
-					new MutableObject<ILogicalExpression>(
-							new VariableReferenceExpression(prev)));
-			returnedOp.getInputs().add(tupSource);
-			break;
-		}
-		default: {
-			v = context.newVar(lc.getVarExpr());
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					lc.getBindingExpr(), tupSource);
-			returnedOp = new AssignOperator(v,
-					new MutableObject<ILogicalExpression>(eo.first));
-			returnedOp.getInputs().add(eo.second);
-			break;
-		}
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+        switch (lc.getBindingExpr().getKind()) {
+            case VARIABLE_EXPRESSION: {
+                v = context.newVar(lc.getVarExpr());
+                LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(prev)));
+                returnedOp.getInputs().add(tupSource);
+                break;
+            }
+            default: {
+                v = context.newVar(lc.getVarExpr());
+                Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
+                        tupSource);
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
+                returnedOp.getInputs().add(eo.second);
+                break;
+            }
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
-			FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> flworPlan = tupSource;
-		boolean isTop = context.isTopFlwor();
-		if (isTop) {
-			context.setTopFlwor(false);
-		}
-		for (Clause c : flwor.getClauseList()) {
-			Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
-					flworPlan);
-			flworPlan = new MutableObject<ILogicalOperator>(pC.first);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> flworPlan = tupSource;
+        boolean isTop = context.isTopFlwor();
+        if (isTop) {
+            context.setTopFlwor(false);
+        }
+        for (Clause c : flwor.getClauseList()) {
+            Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
+            flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+        }
 
-		Expression r = flwor.getReturnExpr();
-		boolean noFlworClause = flwor.noForClause();
+        Expression r = flwor.getReturnExpr();
+        boolean noFlworClause = flwor.noForClause();
 
-		if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
-			VariableExpr v = (VariableExpr) r;
-			LogicalVariable var = context.getVar(v.getVar().getId());
+        if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+            VariableExpr v = (VariableExpr) r;
+            LogicalVariable var = context.getVar(v.getVar().getId());
 
-			return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+            return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
 
-		} else {
-			Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
-					flworPlan.getValue());
-			Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
-					baseOp);
-			ILogicalOperator rOp = rRes.first;
-			ILogicalOperator resOp;
-			if (expressionNeedsNoNesting(r)) {
-				baseOp.setValue(flworPlan.getValue());
-				resOp = rOp;
-			} else {
-				SubplanOperator s = new SubplanOperator(rOp);
-				s.getInputs().add(flworPlan);
-				resOp = s;
-				baseOp.setValue(new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(s)));
-			}
-			Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
-					resOp);
-			return produceFlwrResult(noFlworClause, isTop, resOpRef,
-					rRes.second);
-		}
-	}
+        } else {
+            Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
+            Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
+            ILogicalOperator rOp = rRes.first;
+            ILogicalOperator resOp;
+            if (expressionNeedsNoNesting(r)) {
+                baseOp.setValue(flworPlan.getValue());
+                resOp = rOp;
+            } else {
+                SubplanOperator s = new SubplanOperator(rOp);
+                s.getInputs().add(flworPlan);
+                resOp = s;
+                baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+            }
+            Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
+            return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
+        }
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
-			FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				fa.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(p.first));
-		ILogicalExpression faExpr = new ConstantExpression(
-				new AsterixConstantValue(new AString(fa.getIdent().getValue())));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(faExpr));
-		AssignOperator a = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(fldAccess));
-		a.getInputs().add(p.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
+                .getValue())));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
+        a.getInputs().add(p.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
 
-	}
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
-			IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				ia.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression f;
-		int i = ia.getIndex();
-		if (i == IndexAccessor.ANY) {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		} else {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(
-							new ConstantExpression(new AsterixConstantValue(
-									new AInt32(i)))));
-		}
-		AssignOperator a = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(f));
-		a.getInputs().add(p.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(IndexAccessor ia,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression f;
+        int i = ia.getIndex();
+        if (i == IndexAccessor.ANY) {
+            f = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        } else {
+            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+            f.getArguments().add(
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(
+                            new AsterixConstantValue(new AInt32(i)))));
+        }
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+        a.getInputs().add(p.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(
-			CallExpr fcall, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		LogicalVariable v = context.newVar();
-		AsterixFunction fid = fcall.getIdent();
-		List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> topOp = tupSource;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar();
+        AsterixFunction fid = fcall.getIdent();
+        List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		for (Expression expr : fcall.getExprList()) {
-			switch (expr.getKind()) {
-			case VARIABLE_EXPRESSION: {
-				LogicalVariable var = context.getVar(((VariableExpr) expr)
-						.getVar().getId());
-				args.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(var)));
-				break;
-			}
-			case LITERAL_EXPRESSION: {
-				LiteralExpr val = (LiteralExpr) expr;
-				args.add(new MutableObject<ILogicalExpression>(
-						new ConstantExpression(
-								new AsterixConstantValue(ConstantHelper
-										.objectFromLiteral(val.getValue())))));
-				break;
-			}
-			default: {
-				Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-						expr, topOp);
-				AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
-						.getValue();
-				args.add(new MutableObject<ILogicalExpression>(eo.first));
-				if (o1 != null
-						&& !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
-								o1, topOp))) {
-					topOp = eo.second;
-				}
-				break;
-			}
-			}
-		}
+        for (Expression expr : fcall.getExprList()) {
+            switch (expr.getKind()) {
+                case VARIABLE_EXPRESSION: {
+                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
+                    args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    break;
+                }
+                case LITERAL_EXPRESSION: {
+                    LiteralExpr val = (LiteralExpr) expr;
+                    args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                            ConstantHelper.objectFromLiteral(val.getValue())))));
+                    break;
+                }
+                default: {
+                    Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+                    AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
+                    args.add(new MutableObject<ILogicalExpression>(eo.first));
+                    if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
+                        topOp = eo.second;
+                    }
+                    break;
+                }
+            }
+        }
 
-		FunctionIdentifier fi = new FunctionIdentifier(
-				AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
-		AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
-		FunctionIdentifier builtinAquafi = afi == null ? null : afi
-				.getFunctionIdentifier();
+        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
+        AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+        FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
 
-		if (builtinAquafi != null) {
-			fi = builtinAquafi;
-		} else {
-			fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-					fid.getFunctionName());
-			FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
-					.getBuiltinFunctionIdentifier(fi);
-			if (builtinAsterixFi != null) {
-				fi = builtinAsterixFi;
-			}
-		}
-		AbstractFunctionCallExpression f;
-		if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
-			f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
-					args);
-		} else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
-			UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-			ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
-					.returnsUniqueValues(fi));
-			f = ufce;
-		} else {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-		}
-		// Put hints into function call expr.
-		if (fcall.hasHints()) {
-			for (IExpressionAnnotation hint : fcall.getHints()) {
-				f.getAnnotations().put(hint, hint);
-			}
-		}
-		AssignOperator op = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(f));
-		if (topOp != null) {
-			op.getInputs().add(topOp);
-		}
+        if (builtinAquafi != null) {
+            fi = builtinAquafi;
+        } else {
+            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, fid.getFunctionName());
+            FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
+            if (builtinAsterixFi != null) {
+                fi = builtinAsterixFi;
+            }
+        }
+        AbstractFunctionCallExpression f;
+        if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+            f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
+        } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+            UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(fi), args);
+            ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
+            f = ufce;
+        } else {
+            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
+        }
+        // Put hints into function call expr.
+        if (fcall.hasHints()) {
+            for (IExpressionAnnotation hint : fcall.getHints()) {
+                f.getAnnotations().put(hint, hint);
+            }
+        }
+        AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+        if (topOp != null) {
+            op.getInputs().add(topOp);
+        }
 
-		return new Pair<ILogicalOperator, LogicalVariable>(op, v);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
-			FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
-		// TODO Auto-generated method stub
-		throw new NotImplementedException();
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+            Mutable<ILogicalOperator> tupSource) {
+        // TODO Auto-generated method stub
+        throw new NotImplementedException();
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(
-			GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		GroupByOperator gOp = new GroupByOperator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addGbyExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addDecorExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		gOp.getInputs().add(topOp);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        GroupByOperator gOp = new GroupByOperator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addGbyExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addDecorExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        gOp.getInputs().add(topOp);
 
-		for (VariableExpr var : gc.getWithVarList()) {
-			LogicalVariable aggVar = context.newVar();
-			LogicalVariable oldVar = context.getVar(var);
-			List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			flArgs.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(oldVar)));
-			AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
-					.makeAggregateFunctionExpression(
-							AsterixBuiltinFunctions.LISTIFY, flArgs);
-			AggregateOperator agg = new AggregateOperator(
-					mkSingletonArrayList(aggVar),
-					(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-							fListify)));
+        for (VariableExpr var : gc.getWithVarList()) {
+            LogicalVariable aggVar = context.newVar();
+            LogicalVariable oldVar = context.getVar(var);
+            List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
+            flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
+            AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                    AsterixBuiltinFunctions.LISTIFY, flArgs);
+            AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
+                    (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
 
-			agg.getInputs().add(
-					new MutableObject<ILogicalOperator>(
-							new NestedTupleSourceOperator(
-									new MutableObject<ILogicalOperator>(gOp))));
-			ILogicalPlan plan = new ALogicalPlanImpl(
-					new MutableObject<ILogicalOperator>(agg));
-			gOp.getNestedPlans().add(plan);
-			// Hide the variable that was part of the "with", replacing it with
-			// the one bound by the aggregation op.
-			context.setVar(var, aggVar);
-		}
+            agg.getInputs().add(
+                    new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                            new MutableObject<ILogicalOperator>(gOp))));
+            ILogicalPlan plan = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(agg));
+            gOp.getNestedPlans().add(plan);
+            // Hide the variable that was part of the "with", replacing it with
+            // the one bound by the aggregation op.
+            context.setVar(var, aggVar);
+        }
 
-		gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
-				gc.hasHashGroupByHint());
-		return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
-	}
+        gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+        return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		// In the most general case, IfThenElse is translated in the following
-		// way.
-		//
-		// We assign the result of the condition to one variable varCond.
-		// We create one subplan which contains the plan for the "then" branch,
-		// on top of which there is a selection whose condition is varCond.
-		// Similarly, we create one subplan for the "else" branch, in which the
-		// selection is not(varCond).
-		// Finally, we concatenate the results. (??)
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        // In the most general case, IfThenElse is translated in the following
+        // way.
+        //
+        // We assign the result of the condition to one variable varCond.
+        // We create one subplan which contains the plan for the "then" branch,
+        // on top of which there is a selection whose condition is varCond.
+        // Similarly, we create one subplan for the "else" branch, in which the
+        // selection is not(varCond).
+        // Finally, we concatenate the results. (??)
 
-		Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
-				.accept(this, tupSource);
-		ILogicalOperator opCond = pCond.first;
-		LogicalVariable varCond = pCond.second;
+        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+        ILogicalOperator opCond = pCond.first;
+        LogicalVariable varCond = pCond.second;
 
-		SubplanOperator sp = new SubplanOperator();
-		Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
-				new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(sp)));
+        SubplanOperator sp = new SubplanOperator();
+        Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                new MutableObject<ILogicalOperator>(sp)));
 
-		Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
-				.accept(this, nestedSource);
-		SelectOperator sel1 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
+        Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
+        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                varCond)));
+        sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
-		Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr()
-				.accept(this, nestedSource);
-		AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		SelectOperator sel2 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(notVarCond));
-		sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+        Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
+        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(varCond)));
+        SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
+        sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
 
-		ILogicalPlan p1 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel1));
-		sp.getNestedPlans().add(p1);
-		ILogicalPlan p2 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel2));
-		sp.getNestedPlans().add(p2);
+        ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
+        sp.getNestedPlans().add(p1);
+        ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
+        sp.getNestedPlans().add(p2);
 
-		Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
-				opCond);
-		sp.getInputs().add(opCondRef);
+        Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
+        sp.getInputs().add(opCondRef);
 
-		LogicalVariable resV = context.newVar();
-		AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(pThen.second)),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(pElse.second)));
-		AssignOperator a = new AssignOperator(resV,
-				new MutableObject<ILogicalExpression>(concatNonNull));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+        LogicalVariable resV = context.newVar();
+        AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pThen.second)),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pElse.second)));
+        AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
-			LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
-		LogicalVariable var = context.newVar();
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(new ConstantExpression(
-						new AsterixConstantValue(ConstantHelper
-								.objectFromLiteral(l.getValue())))));
-		if (tupSource != null) {
-			a.getInputs().add(tupSource);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+        LogicalVariable var = context.newVar();
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
+                new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+        if (tupSource != null) {
+            a.getInputs().add(tupSource);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-		if (nOps > 0
-				&& (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
-			return visitAndOrOperator(op, tupSource);
-		}
+        if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+            return visitAndOrOperator(op, tupSource);
+        }
 
-		ArrayList<Expression> exprs = op.getExprList();
+        ArrayList<Expression> exprs = op.getExprList();
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		ILogicalExpression currExpr = null;
-		for (int i = 0; i <= nOps; i++) {
+        ILogicalExpression currExpr = null;
+        for (int i = 0; i <= nOps; i++) {
 
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			ILogicalExpression e = p.first;
-			// now look at the operator
-			if (i < nOps) {
-				if (OperatorExpr.opIsComparison(ops.get(i))) {
-					AbstractFunctionCallExpression c = createComparisonExpression(ops
-							.get(i));
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            ILogicalExpression e = p.first;
+            // now look at the operator
+            if (i < nOps) {
+                if (OperatorExpr.opIsComparison(ops.get(i))) {
+                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
 
-					// chain the operators
-					if (i == 0) {
-						c.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = c;
-						if (op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.LEFT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						c.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = c;
-						if (i == 1 && op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.RIGHT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					}
-				} else {
-					AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
-							.get(i));
+                    // chain the operators
+                    if (i == 0) {
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = c;
+                        if (op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.LEFT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = c;
+                        if (i == 1 && op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.RIGHT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    }
+                } else {
+                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
 
-					if (i == 0) {
-						f.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = f;
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						f.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = f;
-					}
-				}
-			} else { // don't forget the last expression...
-				((AbstractFunctionCallExpression) currExpr).getArguments().add(
-						new MutableObject<ILogicalExpression>(e));
-				if (i == 1 && op.isBroadcastOperand(i)) {
-					BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-					bcast.setObject(BroadcastSide.RIGHT);
-					((AbstractFunctionCallExpression) currExpr)
-							.getAnnotations()
-							.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-									bcast);
-				}
-			}
-		}
+                    if (i == 0) {
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = f;
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = f;
+                    }
+                }
+            } else { // don't forget the last expression...
+                ((AbstractFunctionCallExpression) currExpr).getArguments()
+                        .add(new MutableObject<ILogicalExpression>(e));
+                if (i == 1 && op.isBroadcastOperand(i)) {
+                    BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                    bcast.setObject(BroadcastSide.RIGHT);
+                    ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
+                            BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                }
+            }
+        }
 
-		// Add hints as annotations.
-		if (op.hasHints() && currExpr instanceof AbstractFunctionCallExpression) {
-			AbstractFunctionCallExpression currFuncExpr = (AbstractFunctionCallExpression) currExpr;
-			for (IExpressionAnnotation hint : op.getHints()) {
-				currFuncExpr.getAnnotations().put(hint, hint);
-			}
-		}
-		
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(currExpr));
+        // Add hints as annotations.
+        if (op.hasHints() && currExpr instanceof AbstractFunctionCallExpression) {
+            AbstractFunctionCallExpression currFuncExpr = (AbstractFunctionCallExpression) currExpr;
+            for (IExpressionAnnotation hint : op.getHints()) {
+                currFuncExpr.getAnnotations().put(hint, hint);
+            }
+        }
 
-		a.getInputs().add(topOp);
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
-	}
+        a.getInputs().add(topOp);
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
-			OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+    }
 
-		OrderOperator ord = new OrderOperator();
-		Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression e : oc.getOrderbyList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					e, topOp);
-			OrderModifier m = modifIter.next();
-			OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER
-					: OrderOperator.DESC_ORDER;
-			ord.getOrderExpressions().add(
-					new Pair<IOrder, Mutable<ILogicalExpression>>(comp,
-							new MutableObject<ILogicalExpression>(p.first)));
-			topOp = p.second;
-		}
-		ord.getInputs().add(topOp);
-		if (oc.getNumTuples() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
-					oc.getNumTuples());
-		}
-		if (oc.getNumFrames() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
-					oc.getNumFrames());
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
-			QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> topOp = tupSource;
+        OrderOperator ord = new OrderOperator();
+        Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression e : oc.getOrderbyList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(e, topOp);
+            OrderModifier m = modifIter.next();
+            OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
+            ord.getOrderExpressions()
+                    .add(new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(
+                            p.first)));
+            topOp = p.second;
+        }
+        ord.getInputs().add(topOp);
+        if (oc.getNumTuples() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
+        }
+        if (oc.getNumFrames() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+    }
 
-		ILogicalOperator firstOp = null;
-		Mutable<ILogicalOperator> lastOp = null;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		for (QuantifiedPair qt : qe.getQuantifiedList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					qt.getExpr(), topOp);
-			topOp = eo1.second;
-			LogicalVariable uVar = context.newVar(qt.getVarExpr());
-			ILogicalOperator u = new UnnestOperator(uVar,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo1.first)));
+        ILogicalOperator firstOp = null;
+        Mutable<ILogicalOperator> lastOp = null;
 
-			if (firstOp == null) {
-				firstOp = u;
-			}
-			if (lastOp != null) {
-				u.getInputs().add(lastOp);
-			}
-			lastOp = new MutableObject<ILogicalOperator>(u);
-		}
+        for (QuantifiedPair qt : qe.getQuantifiedList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
+            topOp = eo1.second;
+            LogicalVariable uVar = context.newVar(qt.getVarExpr());
+            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
+                    makeUnnestExpression(eo1.first)));
 
-		// We make all the unnest correspond. to quantif. vars. sit on top
-		// in the hope of enabling joins & other optimiz.
-		firstOp.getInputs().add(topOp);
-		topOp = lastOp;
+            if (firstOp == null) {
+                firstOp = u;
+            }
+            if (lastOp != null) {
+                u.getInputs().add(lastOp);
+            }
+            lastOp = new MutableObject<ILogicalOperator>(u);
+        }
 
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-				qe.getSatisfiesExpr(), topOp);
+        // We make all the unnest correspond. to quantif. vars. sit on top
+        // in the hope of enabling joins & other optimiz.
+        firstOp.getInputs().add(topOp);
+        topOp = lastOp;
 
-		AggregateFunctionCallExpression fAgg;
-		SelectOperator s;
-		if (qe.getQuantifier() == Quantifier.SOME) {
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					eo2.first));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.NON_EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		} else { // EVERY
-			List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					new ScalarFunctionCallExpression(FunctionUtils
-							.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-							satExprList)));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		}
-		LogicalVariable qeVar = context.newVar();
-		AggregateOperator a = new AggregateOperator(
-				mkSingletonArrayList(qeVar),
-				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-						fAgg)));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(s));
-		return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
-	}
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
+
+        AggregateFunctionCallExpression fAgg;
+        SelectOperator s;
+        if (qe.getQuantifier() == Quantifier.SOME) {
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        } else { // EVERY
+            List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        }
+        LogicalVariable qeVar = context.newVar();
+        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+        return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		return q.getBody().accept(this, tupSource);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        return q.getBody().accept(this, tupSource);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
-			RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (FieldBinding fb : rc.getFbList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					fb.getLeftExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo1.first));
-			topOp = eo1.second;
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-					fb.getRightExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo2.first));
-			topOp = eo2.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (FieldBinding fb : rc.getFbList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
+            topOp = eo1.second;
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
+            topOp = eo2.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
-			ListConstructor lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
-				: AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression expr : lc.getExprList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					expr, topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			topOp = eo.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+                : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression expr : lc.getExprList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            topOp = eo.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Expression expr = u.getExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				expr, tupSource);
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a;
-		if (u.getSign() == Sign.POSITIVE) {
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
-					eo.first));
-		} else {
-			AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
-			m.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
-		}
-		a.getInputs().add(eo.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Expression expr = u.getExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a;
+        if (u.getSign() == Sign.POSITIVE) {
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
+        } else {
+            AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+            m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+        }
+        a.getInputs().add(eo.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
-			VariableExpr v, Mutable<ILogicalOperator> tupSource) {
-		// Should we ever get to this method?
-		LogicalVariable var = context.newVar();
-		LogicalVariable oldV = context.getVar(v.getVar().getId());
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(oldV)));
-		a.getInputs().add(tupSource);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+        // Should we ever get to this method?
+        LogicalVariable var = context.newVar();
+        LogicalVariable oldV = context.getVar(v.getVar().getId());
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+                new VariableReferenceExpression(oldV)));
+        a.getInputs().add(tupSource);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
-			WhereClause w, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				w.getWhereExpr(), tupSource);
-		SelectOperator s = new SelectOperator(
-				new MutableObject<ILogicalExpression>(p.first));
-		s.getInputs().add(p.second);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
+        SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
+        s.getInputs().add(p.second);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(s, null);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
-			LimitClause lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getLimitExpr(), tupSource);
-		LimitOperator opLim;
-		Expression offset = lc.getOffset();
-		if (offset != null) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
-					offset, p1.second);
-			opLim = new LimitOperator(p1.first, p2.first);
-			opLim.getInputs().add(p2.second);
-		} else {
-			opLim = new LimitOperator(p1.first);
-			opLim.getInputs().add(p1.second);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        LimitOperator opLim;
+        Expression offset = lc.getOffset();
+        if (offset != null) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
+            opLim = new LimitOperator(p1.first, p2.first);
+            opLim.getInputs().add(p2.second);
+        } else {
+            opLim = new LimitOperator(p1.first);
+            opLim.getInputs().add(p1.second);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getDieExpr(), tupSource);
-		DieOperator opDie = new DieOperator(p1.first);
-		opDie.getInputs().add(p1.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
+        DieOperator opDie = new DieOperator(p1.first);
+        opDie.getInputs().add(p1.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
-			DistinctClause dc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> input = null;
-		for (Expression expr : dc.getDistinctByExpr()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					expr, tupSource);
-			exprList.add(new MutableObject<ILogicalExpression>(p.first));
-			input = p.second;
-		}
-		DistinctOperator opDistinct = new DistinctOperator(exprList);
-		opDistinct.getInputs().add(input);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> input = null;
+        for (Expression expr : dc.getDistinctByExpr()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
+            exprList.add(new MutableObject<ILogicalExpression>(p.first));
+            input = p.second;
+        }
+        DistinctOperator opDistinct = new DistinctOperator(exprList);
+        opDistinct.getInputs().add(input);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
-			UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> ts = tupSource;
-		ILogicalOperator lastOp = null;
-		LogicalVariable lastVar = null;
-		boolean first = true;
-		for (Expression e : unionExpr.getExprs()) {
-			if (first) {
-				first = false;
-			} else {
-				ts = new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator());
-			}
-			Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
-			if (lastOp == null) {
-				lastOp = p1.first;
-				lastVar = p1.second;
-			} else {
-				LogicalVariable unnestVar1 = context.newVar();
-				UnnestOperator unnest1 = new UnnestOperator(
-						unnestVar1,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										lastVar))));
-				unnest1.getInputs().add(
-						new MutableObject<ILogicalOperator>(lastOp));
-				LogicalVariable unnestVar2 = context.newVar();
-				UnnestOperator unnest2 = new UnnestOperator(
-						unnestVar2,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										p1.second))));
-				unnest2.getInputs().add(
-						new MutableObject<ILogicalOperator>(p1.first));
-				List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
-						1);
-				LogicalVariable resultVar = context.newVar();
-				Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
-						unnestVar1, unnestVar2, resultVar);
-				varMap.add(triple);
-				UnionAllOperator unionOp = new UnionAllOperator(varMap);
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest1));
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest2));
-				lastVar = resultVar;
-				lastOp = unionOp;
-			}
-		}
-		LogicalVariable aggVar = context.newVar();
-		ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
-				1);
-		aggregVars.add(aggVar);
-		List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		afcExprs.add(new MutableObject<ILogicalExpression>(
-				new VariableReferenceExpression(lastVar)));
-		AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY, afcExprs);
-		ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
-		AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
-		agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-		return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> ts = tupSource;
+        ILogicalOperator lastOp = null;
+        LogicalVariable lastVar = null;
+        boolean first = true;
+        for (Expression e : unionExpr.getExprs()) {
+            if (first) {
+                first = false;
+            } else {
+                ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+            }
+            Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+            if (lastOp == null) {
+                lastOp = p1.first;
+                lastVar = p1.second;
+            } else {
+                LogicalVariable unnestVar1 = context.newVar();
+                UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(lastVar))));
+                unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+                LogicalVariable unnestVar2 = context.newVar();
+                UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(p1.second))));
+                unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
+                List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+                        1);
+                LogicalVariable resultVar = context.newVar();
+                Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+                        unnestVar1, unnestVar2, resultVar);
+                varMap.add(triple);
+                UnionAllOperator unionOp = new UnionAllOperator(varMap);
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
+                lastVar = resultVar;
+                lastOp = unionOp;
+            }
+        }
+        LogicalVariable aggVar = context.newVar();
+        ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
+        aggregVars.add(aggVar);
+        List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
+        AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, afcExprs);
+        ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+        AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+        agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+        return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+    }
 
-	private AbstractFunctionCallExpression createComparisonExpression(
-			OperatorType t) {
-		FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-		IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
-		return new ScalarFunctionCallExpression(finfo);
-	}
+    private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
+        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+        return new ScalarFunctionCallExpression(finfo);
+    }
 
-	private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-		switch (t) {
-		case EQ: {
-			return AlgebricksBuiltinFunctions.EQ;
-		}
-		case NEQ: {
-			return AlgebricksBuiltinFunctions.NEQ;
-		}
-		case GT: {
-			return AlgebricksBuiltinFunctions.GT;
-		}
-		case GE: {
-			return AlgebricksBuiltinFunctions.GE;
-		}
-		case LT: {
-			return AlgebricksBuiltinFunctions.LT;
-		}
-		case LE: {
-			return AlgebricksBuiltinFunctions.LE;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-	}
+    private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+        switch (t) {
+            case EQ: {
+                return AlgebricksBuiltinFunctions.EQ;
+            }
+            case NEQ: {
+                return AlgebricksBuiltinFunctions.NEQ;
+            }
+            case GT: {
+                return AlgebricksBuiltinFunctions.GT;
+            }
+            case GE: {
+                return AlgebricksBuiltinFunctions.GE;
+            }
+            case LT: {
+                return AlgebricksBuiltinFunctions.LT;
+            }
+            case LE: {
+                return AlgebricksBuiltinFunctions.LE;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
 
-	private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
-			OperatorType t) throws AsterixException {
+    private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
+            throws AsterixException {
 
-		FunctionIdentifier fid = null;
-		switch (t) {
-		case PLUS: {
-			fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
-			break;
-		}
-		case MINUS: {
-			fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
-			break;
-		}
-		case MUL: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
-			break;
-		}
-		case DIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
-			break;
-		}
-		case MOD: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MOD;
-			break;
-		}
-		case IDIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
-			break;
-		}
-		case CARET: {
-			fid = AsterixBuiltinFunctions.CARET;
-			break;
-		}
-		case AND: {
-			fid = AlgebricksBuiltinFunctions.AND;
-			break;
-		}
-		case OR: {
-			fid = AlgebricksBuiltinFunctions.OR;
-			break;
-		}
-		case FUZZY_EQ: {
-			fid = AsterixBuiltinFunctions.FUZZY_EQ;
-			break;
-		}
+        FunctionIdentifier fid = null;
+        switch (t) {
+            case PLUS: {
+                fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+                break;
+            }
+            case MINUS: {
+                fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+                break;
+            }
+            case MUL: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+                break;
+            }
+            case DIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+                break;
+            }
+            case MOD: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+                break;
+            }
+            case IDIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+                break;
+            }
+            case CARET: {
+                fid = AsterixBuiltinFunctions.CARET;
+                break;
+            }
+            case AND: {
+                fid = AlgebricksBuiltinFunctions.AND;
+                break;
+            }
+            case OR: {
+                fid = AlgebricksBuiltinFunctions.OR;
+                break;
+            }
+            case FUZZY_EQ: {
+                fid = AsterixBuiltinFunctions.FUZZY_EQ;
+                break;
+            }
 
-		default: {
-			throw new NotImplementedException("Operator " + t
-					+ " is not yet implemented");
-		}
-		}
-		return new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-	}
+            default: {
+                throw new NotImplementedException("Operator " + t + " is not yet implemented");
+            }
+        }
+        return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+    }
 
-	private static boolean hasOnlyChild(ILogicalOperator parent,
-			Mutable<ILogicalOperator> childCandidate) {
-		List<Mutable<ILogicalOperator>> inp = parent.getInputs();
-		if (inp == null || inp.size() != 1) {
-			return false;
-		}
-		return inp.get(0) == childCandidate;
-	}
+    private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
+        List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+        if (inp == null || inp.size() != 1) {
+            return false;
+        }
+        return inp.get(0) == childCandidate;
+    }
 
-	private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
-			Expression expr, Mutable<ILogicalOperator> topOp)
-			throws AsterixException {
-		switch (expr.getKind()) {
-		case VARIABLE_EXPRESSION: {
-			VariableReferenceExpression ve = new VariableReferenceExpression(
-					context.getVar(((VariableExpr) expr).getVar().getId()));
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
-					topOp);
-		}
-		case LITERAL_EXPRESSION: {
-			LiteralExpr val = (LiteralExpr) expr;
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-					new ConstantExpression(new AsterixConstantValue(
-							ConstantHelper.objectFromLiteral(val.getValue()))),
-					topOp);
-		}
-		default: {
-			// Mutable<ILogicalOperator> src = new
-			// Mutable<ILogicalOperator>();
-			// Mutable<ILogicalOperator> src = topOp;
-			if (expressionNeedsNoNesting(expr)) {
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						topOp);
-				ILogicalExpression exp = ((AssignOperator) p.first)
-						.getExpressions().get(0).getValue();
-				return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-						exp, p.first.getInputs().get(0));
-			} else {
-				Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+    private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
+            Mutable<ILogicalOperator> topOp) throws AsterixException {
+        switch (expr.getKind()) {
+            case VARIABLE_EXPRESSION: {
+                VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
+                        .getVar().getId()));
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
+            }
+            case LITERAL_EXPRESSION: {
+                LiteralExpr val = (LiteralExpr) expr;
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
+                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
+            }
+            default: {
+                // Mutable<ILogicalOperator> src = new
+                // Mutable<ILogicalOperator>();
+                // Mutable<ILogicalOperator> src = topOp;
+                if (expressionNeedsNoNesting(expr)) {
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
+                    ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
+                    return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
+                } else {
+                    Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
 
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						src);
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
 
-				if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-					// src.setOperator(topOp.getOperator());
-					Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
-							p.first);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second), top2);
-				} else {
-					SubplanOperator s = new SubplanOperator();
-					s.getInputs().add(topOp);
-					src.setValue(new NestedTupleSourceOperator(
-							new MutableObject<ILogicalOperator>(s)));
-					Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
-							p.first);
-					s.setRootOp(planRoot);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second),
-							new MutableObject<ILogicalOperator>(s));
-				}
-			}
-		}
-		}
+                    if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                        // src.setOperator(topOp.getOperator());
+                        Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), top2);
+                    } else {
+                        SubplanOperator s = new SubplanOperator();
+                        s.getInputs().add(topOp);
+                        src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+                        Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
+                        s.setRootOp(planRoot);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), new MutableObject<ILogicalOperator>(s));
+                    }
+                }
+            }
+        }
 
-	}
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
-			boolean noForClause, boolean isTop,
-			Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
-		if (isTop) {
-			ProjectOperator pr = new ProjectOperator(resVar);
-			pr.getInputs().add(resOpRef);
-			return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+    private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
+            Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+        if (isTop) {
+            ProjectOperator pr = new ProjectOperator(resVar);
+            pr.getInputs().add(resOpRef);
+            return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
 
-		} else if (noForClause) {
-			return new Pair<ILogicalOperator, LogicalVariable>(
-					resOpRef.getValue(), resVar);
-		} else {
-			return aggListify(resVar, resOpRef, false);
-		}
-	}
+        } else if (noForClause) {
+            return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
+        } else {
+            return aggListify(resVar, resOpRef, false);
+        }
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> aggListify(
-			LogicalVariable var, Mutable<ILogicalOperator> opRef,
-			boolean bProject) {
-		AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY,
-						new ArrayList<Mutable<ILogicalExpression>>());
-		funAgg.getArguments().add(
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(var)));
-		LogicalVariable varListified = context.newVar();
-		AggregateOperator agg = new AggregateOperator(
-				mkSingletonArrayList(varListified),
-				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-						funAgg)));
-		agg.getInputs().add(opRef);
-		ILogicalOperator res;
-		if (bProject) {
-			ProjectOperator pr = new ProjectOperator(varListified);
-			pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
-			res = pr;
-		} else {
-			res = agg;
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
-	}
+    private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
+            boolean bProject) {
+        AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
+        funAgg.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+        LogicalVariable varListified = context.newVar();
+        AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
+        agg.getInputs().add(opRef);
+        ILogicalOperator res;
+        if (bProject) {
+            ProjectOperator pr = new ProjectOperator(varListified);
+            pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+            res = pr;
+        } else {
+            res = agg;
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+    private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-		ArrayList<Expression> exprs = op.getExprList();
+        ArrayList<Expression> exprs = op.getExprList();
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		OperatorType opLogical = ops.get(0);
-		AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+        OperatorType opLogical = ops.get(0);
+        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
 
-		for (int i = 0; i <= nOps; i++) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			// now look at the operator
-			if (i < nOps) {
-				if (ops.get(i) != opLogical) {
-					throw new TranslationException("Unexpected operator "
-							+ ops.get(i) + " in an OperatorExpr starting with "
-							+ opLogical);
-				}
-			}
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		}
+        for (int i = 0; i <= nOps; i++) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            // now look at the operator
+            if (i < nOps) {
+                if (ops.get(i) != opLogical) {
+                    throw new TranslationException("Unexpected operator " + ops.get(i)
+                            + " in an OperatorExpr starting with " + opLogical);
+                }
+            }
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        }
 
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(f));
-		a.getInputs().add(topOp);
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
+        a.getInputs().add(topOp);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
 
-	}
+    }
 
-	private static boolean expressionNeedsNoNesting(Expression expr) {
-		Kind k = expr.getKind();
-		return k == Kind.LITERAL_EXPRESSION
-				|| k == Kind.LIST_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
-				|| k == Kind.OP_EXPRESSION
-				|| k == Kind.FIELD_ACCESSOR_EXPRESSION
-				|| k == Kind.INDEX_ACCESSOR_EXPRESSION
-				|| k == Kind.UNARY_EXPRESSION;
-	}
+    private static boolean expressionNeedsNoNesting(Expression expr) {
+        Kind k = expr.getKind();
+        return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+                || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
+                || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
+                || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
+    }
 
-	private <T> ArrayList<T> mkSingletonArrayList(T item) {
-		ArrayList<T> array = new ArrayList<T>(1);
-		array.add(item);
-		return array;
-	}
+    private <T> ArrayList<T> mkSingletonArrayList(T item) {
+        ArrayList<T> array = new ArrayList<T>(1);
+        array.add(item);
+        return array;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
-			RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
-			TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
-			NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
-			LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
-			DropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
-			ControlFeedStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
-			CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
-			OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
-			UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
-		switch (expr.getExpressionTag()) {
-		case VARIABLE: {
-			return new UnnestingFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-					new MutableObject<ILogicalExpression>(expr));
-		}
-		case FUNCTION_CALL: {
-			AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-			if (fce.getKind() == FunctionKind.UNNEST) {
-				return expr;
-			} else {
-				return new UnnestingFunctionCallExpression(
-						FunctionUtils
-								.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-						new MutableObject<ILogicalExpression>(expr));
-			}
-		}
-		default: {
-			return expr;
-		}
-		}
-	}
+    private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+        switch (expr.getExpressionTag()) {
+            case VARIABLE: {
+                return new UnnestingFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                        new MutableObject<ILogicalExpression>(expr));
+            }
+            case FUNCTION_CALL: {
+                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+                if (fce.getKind() == FunctionKind.UNNEST) {
+                    return expr;
+                } else {
+                    return new UnnestingFunctionCallExpression(
+                            FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                            new MutableObject<ILogicalExpression>(expr));
+                }
+            }
+            default: {
+                return expr;
+            }
+        }
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
-			InsertStatement insert, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
-			DeleteStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
-			UpdateStatement update, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
-			UpdateClause del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
-			DataverseDecl dv, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
-			DatasetDecl dd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
-			SetStatement ss, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
-			WriteStatement ws, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
-			CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
-			IndexDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
-			NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
-			DataverseDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
-			TypeDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visit(
-			CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
-			FunctionDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
-			BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index 4fc1fc8..aa26d8a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -97,6 +97,7 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
@@ -153,1576 +154,1320 @@
  * source for the current subtree.
  */
 
-public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator
-		implements
-		IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator implements
+        IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
 
-	private static final Logger LOGGER = Logger
-			.getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
+    private static final Logger LOGGER = Logger.getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
 
-	private class MetaScopeLogicalVariable {
-		private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
+    private class MetaScopeLogicalVariable {
+        private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
 
-		public VariableReferenceExpression getVariableReferenceExpression(
-				Identifier id) throws AsterixException {
-			LogicalVariable var = map.get(id);
-			LOGGER.fine("get:" + id + ":" + var);
-			if (var == null) {
-				throw new AsterixException("Identifier " + id
-						+ " not found in AQL+ meta-scope.");
-			}
-			return new VariableReferenceExpression(var);
-		}
+        public VariableReferenceExpression getVariableReferenceExpression(Identifier id) throws AsterixException {
+            LogicalVariable var = map.get(id);
+            LOGGER.fine("get:" + id + ":" + var);
+            if (var == null) {
+                throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
+            }
+            return new VariableReferenceExpression(var);
+        }
 
-		public void put(Identifier id, LogicalVariable var) {
-			LOGGER.fine("put:" + id + ":" + var);
-			map.put(id, var);
-		}
-	}
+        public void put(Identifier id, LogicalVariable var) {
+            LOGGER.fine("put:" + id + ":" + var);
+            map.put(id, var);
+        }
+    }
 
-	private class MetaScopeILogicalOperator {
-		private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
+    private class MetaScopeILogicalOperator {
+        private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
 
-		public ILogicalOperator get(Identifier id) throws AsterixException {
-			ILogicalOperator op = map.get(id);
-			if (op == null) {
-				throw new AsterixException("Identifier " + id
-						+ " not found in AQL+ meta-scope.");
-			}
-			return op;
-		}
+        public ILogicalOperator get(Identifier id) throws AsterixException {
+            ILogicalOperator op = map.get(id);
+            if (op == null) {
+                throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
+            }
+            return op;
+        }
 
-		public void put(Identifier id, ILogicalOperator op) {
-			LOGGER.fine("put:" + id + ":" + op);
-			map.put(id, op);
-		}
-	}
+        public void put(Identifier id, ILogicalOperator op) {
+            LOGGER.fine("put:" + id + ":" + op);
+            map.put(id, op);
+        }
+    }
 
-	private final long txnId;
-	private final MetadataTransactionContext mdTxnCtx;
-	private TranslationContext context;
-	private String outputDatasetName;
-	private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
-	private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
-	private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+    private final JobId jobId;
+    private final MetadataTransactionContext mdTxnCtx;
+    private TranslationContext context;
+    private String outputDatasetName;
+    private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
+    private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
+    private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-	public AqlPlusExpressionToPlanTranslator(long txnId,
-			MetadataTransactionContext mdTxnCtx, Counter currentVarCounter,
-			String outputDatasetName) {
-		this.txnId = txnId;
-		this.mdTxnCtx = mdTxnCtx;
-		this.context = new TranslationContext(currentVarCounter);
-		this.outputDatasetName = outputDatasetName;
-		this.context.setTopFlwor(false);
-	}
+    public AqlPlusExpressionToPlanTranslator(JobId jobId, MetadataTransactionContext mdTxnCtx,
+            Counter currentVarCounter, String outputDatasetName) {
+        this.jobId = jobId;
+        this.mdTxnCtx = mdTxnCtx;
+        this.context = new TranslationContext(currentVarCounter);
+        this.outputDatasetName = outputDatasetName;
+        this.context.setTopFlwor(false);
+    }
 
-	public int getVarCounter() {
-		return context.getVarCounter();
-	}
+    public int getVarCounter() {
+        return context.getVarCounter();
+    }
 
-	public ILogicalPlanAndMetadata translate(Query expr)
-			throws AlgebricksException, AsterixException {
-		return translate(expr, null);
-	}
+    public ILogicalPlanAndMetadata translate(Query expr) throws AlgebricksException, AsterixException {
+        return translate(expr, null);
+    }
 
-	public ILogicalPlanAndMetadata translate(Query expr,
-			AqlCompiledMetadataDeclarations compiledDeclarations)
-			throws AlgebricksException, AsterixException {
-		if (expr == null) {
-			return null;
-		}
-		if (compiledDeclarations == null) {
-			compiledDeclarations = compileMetadata(mdTxnCtx,
-					expr.getPrologDeclList(), true);
-		}
-		if (!compiledDeclarations.isConnectedToDataverse())
-			compiledDeclarations.connectToDataverse(compiledDeclarations
-					.getDataverseName());
-		IDataFormat format = compiledDeclarations.getFormat();
-		if (format == null) {
-			throw new AlgebricksException("Data format has not been set.");
-		}
-		format.registerRuntimeFunctions();
-		Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-				new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator()));
+    public ILogicalPlanAndMetadata translate(Query expr, AqlCompiledMetadataDeclarations compiledDeclarations)
+            throws AlgebricksException, AsterixException {
+        if (expr == null) {
+            return null;
+        }
+        if (compiledDeclarations == null) {
+            compiledDeclarations = compileMetadata(mdTxnCtx, expr.getPrologDeclList(), true);
+        }
+        if (!compiledDeclarations.isConnectedToDataverse())
+            compiledDeclarations.connectToDataverse(compiledDeclarations.getDataverseName());
+        IDataFormat format = compiledDeclarations.getFormat();
+        if (format == null) {
+            throw new AlgebricksException("Data format has not been set.");
+        }
+        format.registerRuntimeFunctions();
+        Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
+                new EmptyTupleSourceOperator()));
 
-		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-		boolean isTransactionalWrite = false;
-		ILogicalOperator topOp = p.first;
-		ProjectOperator project = (ProjectOperator) topOp;
-		LogicalVariable resVar = project.getVariables().get(0);
-		if (outputDatasetName == null) {
-			List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			writeExprList.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(resVar)));
-			FileSplitSinkId fssi = new FileSplitSinkId(
-					compiledDeclarations.getOutputFile());
-			FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
-			topOp = new WriteOperator(writeExprList, sink);
-			topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
-		} else {
-			Dataset dataset = compiledDeclarations
-					.findDataset(outputDatasetName);
-			if (dataset == null) {
-				throw new AlgebricksException("Cannot find dataset "
-						+ outputDatasetName);
-			}
-			if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-				throw new AlgebricksException(
-						"Cannot write output to an external dataset.");
-			}
-			ARecordType itemType = (ARecordType) compiledDeclarations
-					.findType(dataset.getItemTypeName());
-			List<String> partitioningKeys = DatasetUtils
-					.getPartitioningKeys(dataset);
-			ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
-			ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
-			List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
-			for (String partitioningKey : partitioningKeys) {
-				Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner = format
-						.partitioningEvaluatorFactory(itemType, partitioningKey);
-				AbstractFunctionCallExpression f = partitioner.second
-						.cloneExpression();
-				f.substituteVar(METADATA_DUMMY_VAR, resVar);
-				exprs.add(new MutableObject<ILogicalExpression>(f));
-				LogicalVariable v = context.newVar();
-				vars.add(v);
-				varRefsForLoading.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(v)));
-			}
-			AssignOperator assign = new AssignOperator(vars, exprs);
-			assign.getInputs()
-					.add(new MutableObject<ILogicalOperator>(project));
-		}
+        boolean isTransactionalWrite = false;
+        ILogicalOperator topOp = p.first;
+        ProjectOperator project = (ProjectOperator) topOp;
+        LogicalVariable resVar = project.getVariables().get(0);
+        if (outputDatasetName == null) {
+            List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+            FileSplitSinkId fssi = new FileSplitSinkId(compiledDeclarations.getOutputFile());
+            FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+            topOp = new WriteOperator(writeExprList, sink);
+            topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+        } else {
+            Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+            if (dataset == null) {
+                throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
+            }
+            if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+                throw new AlgebricksException("Cannot write output to an external dataset.");
+            }
+            ARecordType itemType = (ARecordType) compiledDeclarations.findType(dataset.getItemTypeName());
+            List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+            ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
+            List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
+            for (String partitioningKey : partitioningKeys) {
+                Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner = format
+                        .partitioningEvaluatorFactory(itemType, partitioningKey);
+                AbstractFunctionCallExpression f = partitioner.second.cloneExpression();
+                f.substituteVar(METADATA_DUMMY_VAR, resVar);
+                exprs.add(new MutableObject<ILogicalExpression>(f));
+                LogicalVariable v = context.newVar();
+                vars.add(v);
+                varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            }
+            AssignOperator assign = new AssignOperator(vars, exprs);
+            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+        }
 
-		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-		AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
-				isTransactionalWrite, compiledDeclarations);
-		ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
-				plan, metadataProvider);
-		return planAndMetadata;
-	}
+        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+        AqlMetadataProvider metadataProvider = new AqlMetadataProvider(jobId, isTransactionalWrite,
+                compiledDeclarations);
+        ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(plan, metadataProvider);
+        return planAndMetadata;
+    }
 
-	public ILogicalPlan translate(List<Clause> clauses)
-			throws AlgebricksException, AsterixException {
+    public ILogicalPlan translate(List<Clause> clauses) throws AlgebricksException, AsterixException {
 
-		if (clauses == null) {
-			return null;
-		}
+        if (clauses == null) {
+            return null;
+        }
 
-		Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(
-				new EmptyTupleSourceOperator());
-		Pair<ILogicalOperator, LogicalVariable> p = null;
-		for (Clause c : clauses) {
-			p = c.accept(this, opRef);
-			opRef = new MutableObject<ILogicalOperator>(p.first);
-		}
+        Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+        Pair<ILogicalOperator, LogicalVariable> p = null;
+        for (Clause c : clauses) {
+            p = c.accept(this, opRef);
+            opRef = new MutableObject<ILogicalOperator>(p.first);
+        }
 
-		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-		ILogicalOperator topOp = p.first;
+        ILogicalOperator topOp = p.first;
 
-		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-		return plan;
-	}
+        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+        return plan;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v = context.newVar(fc.getVarExpr());
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar(fc.getVarExpr());
 
-		Expression inExpr = fc.getInExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				inExpr, tupSource);
-		ILogicalOperator returnedOp;
+        Expression inExpr = fc.getInExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
+        ILogicalOperator returnedOp;
 
-		if (fc.getPosVarExpr() == null) {
-			returnedOp = new UnnestOperator(v,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo.first)));
-		} else {
-			LogicalVariable pVar = context.newVar(fc.getPosVarExpr());
-			returnedOp = new UnnestOperator(v,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo.first)), pVar,
-					BuiltinType.AINT32);
-		}
-		returnedOp.getInputs().add(eo.second);
+        if (fc.getPosVarExpr() == null) {
+            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)));
+        } else {
+            LogicalVariable pVar = context.newVar(fc.getPosVarExpr());
+            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)),
+                    pVar, BuiltinType.AINT32);
+        }
+        returnedOp.getInputs().add(eo.second);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v;
-		ILogicalOperator returnedOp;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v;
+        ILogicalOperator returnedOp;
 
-		switch (lc.getBindingExpr().getKind()) {
-		case VARIABLE_EXPRESSION: {
-			v = context.newVar(lc.getVarExpr());
-			LogicalVariable prev = context.getVar(((VariableExpr) lc
-					.getBindingExpr()).getVar().getId());
-			returnedOp = new AssignOperator(v,
-					new MutableObject<ILogicalExpression>(
-							new VariableReferenceExpression(prev)));
-			returnedOp.getInputs().add(tupSource);
-			break;
-		}
-		default: {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					lc.getBindingExpr(), tupSource);
-			v = context.newVar(lc.getVarExpr());
-			returnedOp = new AssignOperator(v,
-					new MutableObject<ILogicalExpression>(eo.first));
-			returnedOp.getInputs().add(eo.second);
-			break;
-		}
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+        switch (lc.getBindingExpr().getKind()) {
+            case VARIABLE_EXPRESSION: {
+                v = context.newVar(lc.getVarExpr());
+                LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(prev)));
+                returnedOp.getInputs().add(tupSource);
+                break;
+            }
+            default: {
+                Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
+                        tupSource);
+                v = context.newVar(lc.getVarExpr());
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
+                returnedOp.getInputs().add(eo.second);
+                break;
+            }
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
-			FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> flworPlan = tupSource;
-		boolean isTop = context.isTopFlwor();
-		if (isTop) {
-			context.setTopFlwor(false);
-		}
-		for (Clause c : flwor.getClauseList()) {
-			Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
-					flworPlan);
-			flworPlan = new MutableObject<ILogicalOperator>(pC.first);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> flworPlan = tupSource;
+        boolean isTop = context.isTopFlwor();
+        if (isTop) {
+            context.setTopFlwor(false);
+        }
+        for (Clause c : flwor.getClauseList()) {
+            Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
+            flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+        }
 
-		Expression r = flwor.getReturnExpr();
-		boolean noFlworClause = flwor.noForClause();
+        Expression r = flwor.getReturnExpr();
+        boolean noFlworClause = flwor.noForClause();
 
-		if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
-			VariableExpr v = (VariableExpr) r;
-			LogicalVariable var = context.getVar(v.getVar().getId());
+        if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+            VariableExpr v = (VariableExpr) r;
+            LogicalVariable var = context.getVar(v.getVar().getId());
 
-			return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+            return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
 
-		} else {
-			Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
-					flworPlan.getValue());
-			Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
-					baseOp);
-			ILogicalOperator rOp = rRes.first;
-			ILogicalOperator resOp;
-			if (expressionNeedsNoNesting(r)) {
-				baseOp.setValue(flworPlan.getValue());
-				resOp = rOp;
-			} else {
-				SubplanOperator s = new SubplanOperator(rOp);
-				s.getInputs().add(flworPlan);
-				resOp = s;
-				baseOp.setValue(new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(s)));
-			}
-			Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
-					resOp);
-			return produceFlwrResult(noFlworClause, isTop, resOpRef,
-					rRes.second);
-		}
-	}
+        } else {
+            Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
+            Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
+            ILogicalOperator rOp = rRes.first;
+            ILogicalOperator resOp;
+            if (expressionNeedsNoNesting(r)) {
+                baseOp.setValue(flworPlan.getValue());
+                resOp = rOp;
+            } else {
+                SubplanOperator s = new SubplanOperator(rOp);
+                s.getInputs().add(flworPlan);
+                resOp = s;
+                baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+            }
+            Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
+            return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
+        }
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
-			FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				fa.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(p.first));
-		ILogicalExpression faExpr = new ConstantExpression(
-				new AsterixConstantValue(new AString(fa.getIdent().getValue())));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(faExpr));
-		AssignOperator a = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(fldAccess));
-		a.getInputs().add(p.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
+                .getValue())));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
+        a.getInputs().add(p.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
 
-	}
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
-			IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				ia.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression f;
-		int i = ia.getIndex();
-		if (i == IndexAccessor.ANY) {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		} else {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(
-							new ConstantExpression(new AsterixConstantValue(
-									new AInt32(i)))));
-		}
-		AssignOperator a = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(f));
-		a.getInputs().add(p.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(IndexAccessor ia,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression f;
+        int i = ia.getIndex();
+        if (i == IndexAccessor.ANY) {
+            f = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        } else {
+            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+            f.getArguments().add(
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(
+                            new AsterixConstantValue(new AInt32(i)))));
+        }
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+        a.getInputs().add(p.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(
-			CallExpr fcall, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		LogicalVariable v = context.newVar();
-		AsterixFunction fid = fcall.getIdent();
-		List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> topOp = tupSource;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar();
+        AsterixFunction fid = fcall.getIdent();
+        List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		for (Expression expr : fcall.getExprList()) {
-			switch (expr.getKind()) {
-			case VARIABLE_EXPRESSION: {
-				LogicalVariable var = context.getVar(((VariableExpr) expr)
-						.getVar().getId());
-				args.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(var)));
-				break;
-			}
-			case LITERAL_EXPRESSION: {
-				LiteralExpr val = (LiteralExpr) expr;
-				args.add(new MutableObject<ILogicalExpression>(
-						new ConstantExpression(
-								new AsterixConstantValue(ConstantHelper
-										.objectFromLiteral(val.getValue())))));
-				break;
-			}
-			default: {
-				Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-						expr, topOp);
-				AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
-						.getValue();
-				args.add(new MutableObject<ILogicalExpression>(eo.first));
-				if (o1 != null
-						&& !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
-								o1, topOp))) {
-					topOp = eo.second;
-				}
-				break;
-			}
-			}
-		}
+        for (Expression expr : fcall.getExprList()) {
+            switch (expr.getKind()) {
+                case VARIABLE_EXPRESSION: {
+                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
+                    args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    break;
+                }
+                case LITERAL_EXPRESSION: {
+                    LiteralExpr val = (LiteralExpr) expr;
+                    args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                            ConstantHelper.objectFromLiteral(val.getValue())))));
+                    break;
+                }
+                default: {
+                    Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+                    AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
+                    args.add(new MutableObject<ILogicalExpression>(eo.first));
+                    if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
+                        topOp = eo.second;
+                    }
+                    break;
+                }
+            }
+        }
 
-		FunctionIdentifier fi = new FunctionIdentifier(
-				AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
-		AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
-		FunctionIdentifier builtinAquafi = afi == null ? null : afi
-				.getFunctionIdentifier();
+        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
+        AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+        FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
 
-		if (builtinAquafi != null) {
-			fi = builtinAquafi;
-		} else {
-			fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-					fid.getFunctionName());
-			FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
-					.getBuiltinFunctionIdentifier(fi);
-			if (builtinAsterixFi != null) {
-				fi = builtinAsterixFi;
-			}
-		}
-		AbstractFunctionCallExpression f;
-		if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
-			f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
-					args);
-		} else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
-			UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-			ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
-					.returnsUniqueValues(fi));
-			f = ufce;
-		} else {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-		}
-		AssignOperator op = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(f));
-		if (topOp != null) {
-			op.getInputs().add(topOp);
-		}
+        if (builtinAquafi != null) {
+            fi = builtinAquafi;
+        } else {
+            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, fid.getFunctionName());
+            FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
+            if (builtinAsterixFi != null) {
+                fi = builtinAsterixFi;
+            }
+        }
+        AbstractFunctionCallExpression f;
+        if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+            f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
+        } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+            UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(fi), args);
+            ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
+            f = ufce;
+        } else {
+            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
+        }
+        AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+        if (topOp != null) {
+            op.getInputs().add(topOp);
+        }
 
-		return new Pair<ILogicalOperator, LogicalVariable>(op, v);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
-			FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
-		// TODO Auto-generated method stub
-		throw new NotImplementedException();
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+            Mutable<ILogicalOperator> tupSource) {
+        // TODO Auto-generated method stub
+        throw new NotImplementedException();
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(
-			GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		GroupByOperator gOp = new GroupByOperator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addGbyExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addDecorExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		gOp.getInputs().add(topOp);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        GroupByOperator gOp = new GroupByOperator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addGbyExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addDecorExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        gOp.getInputs().add(topOp);
 
-		for (VariableExpr var : gc.getWithVarList()) {
-			LogicalVariable aggVar = context.newVar();
-			LogicalVariable oldVar = context.getVar(var);
-			List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			flArgs.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(oldVar)));
-			AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
-					.makeAggregateFunctionExpression(
-							AsterixBuiltinFunctions.LISTIFY, flArgs);
-			AggregateOperator agg = new AggregateOperator(
-					mkSingletonArrayList(aggVar),
-					(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-							fListify)));
-			agg.getInputs().add(
-					new MutableObject<ILogicalOperator>(
-							new NestedTupleSourceOperator(
-									new MutableObject<ILogicalOperator>(gOp))));
-			ILogicalPlan plan = new ALogicalPlanImpl(
-					new MutableObject<ILogicalOperator>(agg));
-			gOp.getNestedPlans().add(plan);
-			// Hide the variable that was part of the "with", replacing it with
-			// the one bound by the aggregation op.
-			context.setVar(var, aggVar);
-		}
+        for (VariableExpr var : gc.getWithVarList()) {
+            LogicalVariable aggVar = context.newVar();
+            LogicalVariable oldVar = context.getVar(var);
+            List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
+            flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
+            AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                    AsterixBuiltinFunctions.LISTIFY, flArgs);
+            AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
+                    (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
+            agg.getInputs().add(
+                    new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                            new MutableObject<ILogicalOperator>(gOp))));
+            ILogicalPlan plan = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(agg));
+            gOp.getNestedPlans().add(plan);
+            // Hide the variable that was part of the "with", replacing it with
+            // the one bound by the aggregation op.
+            context.setVar(var, aggVar);
+        }
 
-		gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
-				gc.hasHashGroupByHint());
-		return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
-	}
+        gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+        return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		// In the most general case, IfThenElse is translated in the following
-		// way.
-		//
-		// We assign the result of the condition to one variable varCond.
-		// We create one subplan which contains the plan for the "then" branch,
-		// on top of which there is a selection whose condition is varCond.
-		// Similarly, we create one subplan for the "else" branch, in which the
-		// selection is not(varCond).
-		// Finally, we concatenate the results. (??)
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        // In the most general case, IfThenElse is translated in the following
+        // way.
+        //
+        // We assign the result of the condition to one variable varCond.
+        // We create one subplan which contains the plan for the "then" branch,
+        // on top of which there is a selection whose condition is varCond.
+        // Similarly, we create one subplan for the "else" branch, in which the
+        // selection is not(varCond).
+        // Finally, we concatenate the results. (??)
 
-		Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
-				.accept(this, tupSource);
-		ILogicalOperator opCond = pCond.first;
-		LogicalVariable varCond = pCond.second;
+        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+        ILogicalOperator opCond = pCond.first;
+        LogicalVariable varCond = pCond.second;
 
-		SubplanOperator sp = new SubplanOperator();
-		Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
-				new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(sp)));
+        SubplanOperator sp = new SubplanOperator();
+        Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                new MutableObject<ILogicalOperator>(sp)));
 
-		Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
-				.accept(this, nestedSource);
-		SelectOperator sel1 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
+        Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
+        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                varCond)));
+        sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
-		Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr()
-				.accept(this, nestedSource);
-		AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		SelectOperator sel2 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(notVarCond));
-		sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+        Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
+        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(varCond)));
+        SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
+        sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
 
-		ILogicalPlan p1 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel1));
-		sp.getNestedPlans().add(p1);
-		ILogicalPlan p2 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel2));
-		sp.getNestedPlans().add(p2);
+        ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
+        sp.getNestedPlans().add(p1);
+        ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
+        sp.getNestedPlans().add(p2);
 
-		Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
-				opCond);
-		sp.getInputs().add(opCondRef);
+        Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
+        sp.getInputs().add(opCondRef);
 
-		LogicalVariable resV = context.newVar();
-		AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(pThen.second)),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(pElse.second)));
-		AssignOperator a = new AssignOperator(resV,
-				new MutableObject<ILogicalExpression>(concatNonNull));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+        LogicalVariable resV = context.newVar();
+        AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pThen.second)),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pElse.second)));
+        AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
-			LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
-		LogicalVariable var = context.newVar();
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(new ConstantExpression(
-						new AsterixConstantValue(ConstantHelper
-								.objectFromLiteral(l.getValue())))));
-		if (tupSource != null) {
-			a.getInputs().add(tupSource);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+        LogicalVariable var = context.newVar();
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
+                new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+        if (tupSource != null) {
+            a.getInputs().add(tupSource);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-		if (nOps > 0
-				&& (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
-			return visitAndOrOperator(op, tupSource);
-		}
+        if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+            return visitAndOrOperator(op, tupSource);
+        }
 
-		ArrayList<Expression> exprs = op.getExprList();
+        ArrayList<Expression> exprs = op.getExprList();
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		ILogicalExpression currExpr = null;
-		for (int i = 0; i <= nOps; i++) {
+        ILogicalExpression currExpr = null;
+        for (int i = 0; i <= nOps; i++) {
 
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			ILogicalExpression e = p.first;
-			// now look at the operator
-			if (i < nOps) {
-				if (OperatorExpr.opIsComparison(ops.get(i))) {
-					AbstractFunctionCallExpression c = createComparisonExpression(ops
-							.get(i));
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            ILogicalExpression e = p.first;
+            // now look at the operator
+            if (i < nOps) {
+                if (OperatorExpr.opIsComparison(ops.get(i))) {
+                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
 
-					// chain the operators
-					if (i == 0) {
-						c.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = c;
-						if (op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.LEFT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						c.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = c;
-						if (i == 1 && op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.RIGHT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					}
-				} else {
-					AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
-							.get(i));
+                    // chain the operators
+                    if (i == 0) {
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = c;
+                        if (op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.LEFT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = c;
+                        if (i == 1 && op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.RIGHT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    }
+                } else {
+                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
 
-					if (i == 0) {
-						f.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = f;
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						f.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = f;
-					}
-				}
-			} else { // don't forget the last expression...
-				((AbstractFunctionCallExpression) currExpr).getArguments().add(
-						new MutableObject<ILogicalExpression>(e));
-				if (i == 1 && op.isBroadcastOperand(i)) {
-					BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-					bcast.setObject(BroadcastSide.RIGHT);
-					((AbstractFunctionCallExpression) currExpr)
-							.getAnnotations()
-							.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-									bcast);
-				}
-			}
-		}
+                    if (i == 0) {
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = f;
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = f;
+                    }
+                }
+            } else { // don't forget the last expression...
+                ((AbstractFunctionCallExpression) currExpr).getArguments()
+                        .add(new MutableObject<ILogicalExpression>(e));
+                if (i == 1 && op.isBroadcastOperand(i)) {
+                    BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                    bcast.setObject(BroadcastSide.RIGHT);
+                    ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
+                            BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                }
+            }
+        }
 
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(currExpr));
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
 
-		a.getInputs().add(topOp);
+        a.getInputs().add(topOp);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
-			OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
 
-		OrderOperator ord = new OrderOperator();
-		Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression e : oc.getOrderbyList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					e, topOp);
-			OrderModifier m = modifIter.next();
-			OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER
-					: OrderOperator.DESC_ORDER;
-			ord.getOrderExpressions().add(
-					new Pair<IOrder, Mutable<ILogicalExpression>>(comp,
-							new MutableObject<ILogicalExpression>(p.first)));
-			topOp = p.second;
-		}
-		ord.getInputs().add(topOp);
-		if (oc.getNumTuples() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
-					oc.getNumTuples());
-		}
-		if (oc.getNumFrames() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
-					oc.getNumFrames());
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
-	}
+        OrderOperator ord = new OrderOperator();
+        Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression e : oc.getOrderbyList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(e, topOp);
+            OrderModifier m = modifIter.next();
+            OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
+            ord.getOrderExpressions()
+                    .add(new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(
+                            p.first)));
+            topOp = p.second;
+        }
+        ord.getInputs().add(topOp);
+        if (oc.getNumTuples() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
+        }
+        if (oc.getNumFrames() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
-			QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> topOp = tupSource;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		ILogicalOperator firstOp = null;
-		Mutable<ILogicalOperator> lastOp = null;
+        ILogicalOperator firstOp = null;
+        Mutable<ILogicalOperator> lastOp = null;
 
-		for (QuantifiedPair qt : qe.getQuantifiedList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					qt.getExpr(), topOp);
-			topOp = eo1.second;
-			LogicalVariable uVar = context.newVar(qt.getVarExpr());
-			ILogicalOperator u = new UnnestOperator(uVar,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo1.first)));
+        for (QuantifiedPair qt : qe.getQuantifiedList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
+            topOp = eo1.second;
+            LogicalVariable uVar = context.newVar(qt.getVarExpr());
+            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
+                    makeUnnestExpression(eo1.first)));
 
-			if (firstOp == null) {
-				firstOp = u;
-			}
-			if (lastOp != null) {
-				u.getInputs().add(lastOp);
-			}
-			lastOp = new MutableObject<ILogicalOperator>(u);
-		}
+            if (firstOp == null) {
+                firstOp = u;
+            }
+            if (lastOp != null) {
+                u.getInputs().add(lastOp);
+            }
+            lastOp = new MutableObject<ILogicalOperator>(u);
+        }
 
-		// We make all the unnest correspond. to quantif. vars. sit on top
-		// in the hope of enabling joins & other optimiz.
-		firstOp.getInputs().add(topOp);
-		topOp = lastOp;
+        // We make all the unnest correspond. to quantif. vars. sit on top
+        // in the hope of enabling joins & other optimiz.
+        firstOp.getInputs().add(topOp);
+        topOp = lastOp;
 
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-				qe.getSatisfiesExpr(), topOp);
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
 
-		AggregateFunctionCallExpression fAgg;
-		SelectOperator s;
-		if (qe.getQuantifier() == Quantifier.SOME) {
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					eo2.first));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.NON_EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		} else { // EVERY
-			List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					new ScalarFunctionCallExpression(FunctionUtils
-							.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-							satExprList)));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		}
-		LogicalVariable qeVar = context.newVar();
-		AggregateOperator a = new AggregateOperator(
-				mkSingletonArrayList(qeVar),
-				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-						fAgg)));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(s));
-		return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
-	}
+        AggregateFunctionCallExpression fAgg;
+        SelectOperator s;
+        if (qe.getQuantifier() == Quantifier.SOME) {
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        } else { // EVERY
+            List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        }
+        LogicalVariable qeVar = context.newVar();
+        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+        return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		return q.getBody().accept(this, tupSource);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        return q.getBody().accept(this, tupSource);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
-			RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (FieldBinding fb : rc.getFbList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					fb.getLeftExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo1.first));
-			topOp = eo1.second;
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-					fb.getRightExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo2.first));
-			topOp = eo2.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (FieldBinding fb : rc.getFbList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
+            topOp = eo1.second;
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
+            topOp = eo2.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
-			ListConstructor lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
-				: AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression expr : lc.getExprList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					expr, topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			topOp = eo.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+                : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression expr : lc.getExprList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            topOp = eo.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Expression expr = u.getExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				expr, tupSource);
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a;
-		if (u.getSign() == Sign.POSITIVE) {
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
-					eo.first));
-		} else {
-			AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
-			m.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
-		}
-		a.getInputs().add(eo.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Expression expr = u.getExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a;
+        if (u.getSign() == Sign.POSITIVE) {
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
+        } else {
+            AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+            m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+        }
+        a.getInputs().add(eo.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
-			VariableExpr v, Mutable<ILogicalOperator> tupSource) {
-		// Should we ever get to this method?
-		LogicalVariable var = context.newVar();
-		LogicalVariable oldV = context.getVar(v.getVar().getId());
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(oldV)));
-		a.getInputs().add(tupSource);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+        // Should we ever get to this method?
+        LogicalVariable var = context.newVar();
+        LogicalVariable oldV = context.getVar(v.getVar().getId());
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+                new VariableReferenceExpression(oldV)));
+        a.getInputs().add(tupSource);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
-			WhereClause w, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				w.getWhereExpr(), tupSource);
-		SelectOperator s = new SelectOperator(
-				new MutableObject<ILogicalExpression>(p.first));
-		s.getInputs().add(p.second);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
+        SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
+        s.getInputs().add(p.second);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(s, null);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
-			LimitClause lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getLimitExpr(), tupSource);
-		LimitOperator opLim;
-		Expression offset = lc.getOffset();
-		if (offset != null) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
-					offset, p1.second);
-			opLim = new LimitOperator(p1.first, p2.first);
-			opLim.getInputs().add(p2.second);
-		} else {
-			opLim = new LimitOperator(p1.first);
-			opLim.getInputs().add(p1.second);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        LimitOperator opLim;
+        Expression offset = lc.getOffset();
+        if (offset != null) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
+            opLim = new LimitOperator(p1.first, p2.first);
+            opLim.getInputs().add(p2.second);
+        } else {
+            opLim = new LimitOperator(p1.first);
+            opLim.getInputs().add(p1.second);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getDieExpr(), tupSource);
-		DieOperator opDie = new DieOperator(p1.first);
-		opDie.getInputs().add(p1.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
+        DieOperator opDie = new DieOperator(p1.first);
+        opDie.getInputs().add(p1.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
-			DistinctClause dc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> input = null;
-		for (Expression expr : dc.getDistinctByExpr()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					expr, tupSource);
-			exprList.add(new MutableObject<ILogicalExpression>(p.first));
-			input = p.second;
-		}
-		DistinctOperator opDistinct = new DistinctOperator(exprList);
-		opDistinct.getInputs().add(input);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> input = null;
+        for (Expression expr : dc.getDistinctByExpr()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
+            exprList.add(new MutableObject<ILogicalExpression>(p.first));
+            input = p.second;
+        }
+        DistinctOperator opDistinct = new DistinctOperator(exprList);
+        opDistinct.getInputs().add(input);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
-			UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> ts = tupSource;
-		ILogicalOperator lastOp = null;
-		LogicalVariable lastVar = null;
-		boolean first = true;
-		for (Expression e : unionExpr.getExprs()) {
-			if (first) {
-				first = false;
-			} else {
-				ts = new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator());
-			}
-			Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
-			if (lastOp == null) {
-				lastOp = p1.first;
-				lastVar = p1.second;
-			} else {
-				LogicalVariable unnestVar1 = context.newVar();
-				UnnestOperator unnest1 = new UnnestOperator(
-						unnestVar1,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										lastVar))));
-				unnest1.getInputs().add(
-						new MutableObject<ILogicalOperator>(lastOp));
-				LogicalVariable unnestVar2 = context.newVar();
-				UnnestOperator unnest2 = new UnnestOperator(
-						unnestVar2,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										p1.second))));
-				unnest2.getInputs().add(
-						new MutableObject<ILogicalOperator>(p1.first));
-				List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
-						1);
-				LogicalVariable resultVar = context.newVar();
-				Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
-						unnestVar1, unnestVar2, resultVar);
-				varMap.add(triple);
-				UnionAllOperator unionOp = new UnionAllOperator(varMap);
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest1));
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest2));
-				lastVar = resultVar;
-				lastOp = unionOp;
-			}
-		}
-		LogicalVariable aggVar = context.newVar();
-		ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
-				1);
-		aggregVars.add(aggVar);
-		List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		afcExprs.add(new MutableObject<ILogicalExpression>(
-				new VariableReferenceExpression(lastVar)));
-		AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY, afcExprs);
-		ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
-		AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
-		agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-		return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> ts = tupSource;
+        ILogicalOperator lastOp = null;
+        LogicalVariable lastVar = null;
+        boolean first = true;
+        for (Expression e : unionExpr.getExprs()) {
+            if (first) {
+                first = false;
+            } else {
+                ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+            }
+            Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+            if (lastOp == null) {
+                lastOp = p1.first;
+                lastVar = p1.second;
+            } else {
+                LogicalVariable unnestVar1 = context.newVar();
+                UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(lastVar))));
+                unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+                LogicalVariable unnestVar2 = context.newVar();
+                UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(p1.second))));
+                unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
+                List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+                        1);
+                LogicalVariable resultVar = context.newVar();
+                Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+                        unnestVar1, unnestVar2, resultVar);
+                varMap.add(triple);
+                UnionAllOperator unionOp = new UnionAllOperator(varMap);
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
+                lastVar = resultVar;
+                lastOp = unionOp;
+            }
+        }
+        LogicalVariable aggVar = context.newVar();
+        ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
+        aggregVars.add(aggVar);
+        List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
+        AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, afcExprs);
+        ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+        AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+        agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+        return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+    }
 
-	private AbstractFunctionCallExpression createComparisonExpression(
-			OperatorType t) {
-		FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-		IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
-		return new ScalarFunctionCallExpression(finfo);
-	}
+    private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
+        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+        return new ScalarFunctionCallExpression(finfo);
+    }
 
-	private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-		switch (t) {
-		case EQ: {
-			return AlgebricksBuiltinFunctions.EQ;
-		}
-		case NEQ: {
-			return AlgebricksBuiltinFunctions.NEQ;
-		}
-		case GT: {
-			return AlgebricksBuiltinFunctions.GT;
-		}
-		case GE: {
-			return AlgebricksBuiltinFunctions.GE;
-		}
-		case LT: {
-			return AlgebricksBuiltinFunctions.LT;
-		}
-		case LE: {
-			return AlgebricksBuiltinFunctions.LE;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-	}
+    private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+        switch (t) {
+            case EQ: {
+                return AlgebricksBuiltinFunctions.EQ;
+            }
+            case NEQ: {
+                return AlgebricksBuiltinFunctions.NEQ;
+            }
+            case GT: {
+                return AlgebricksBuiltinFunctions.GT;
+            }
+            case GE: {
+                return AlgebricksBuiltinFunctions.GE;
+            }
+            case LT: {
+                return AlgebricksBuiltinFunctions.LT;
+            }
+            case LE: {
+                return AlgebricksBuiltinFunctions.LE;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
 
-	private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
-			OperatorType t) throws AsterixException {
+    private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
+            throws AsterixException {
 
-		FunctionIdentifier fid = null;
-		switch (t) {
-		case PLUS: {
-			fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
-			break;
-		}
-		case MINUS: {
-			fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
-			break;
-		}
-		case MUL: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
-			break;
-		}
-		case DIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
-			break;
-		}
-		case MOD: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MOD;
-			break;
-		}
-		case IDIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
-			break;
-		}
-		case CARET: {
-			fid = AsterixBuiltinFunctions.CARET;
-			break;
-		}
-		case AND: {
-			fid = AlgebricksBuiltinFunctions.AND;
-			break;
-		}
-		case OR: {
-			fid = AlgebricksBuiltinFunctions.OR;
-			break;
-		}
-		case FUZZY_EQ: {
-			fid = AsterixBuiltinFunctions.FUZZY_EQ;
-			break;
-		}
+        FunctionIdentifier fid = null;
+        switch (t) {
+            case PLUS: {
+                fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+                break;
+            }
+            case MINUS: {
+                fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+                break;
+            }
+            case MUL: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+                break;
+            }
+            case DIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+                break;
+            }
+            case MOD: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+                break;
+            }
+            case IDIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+                break;
+            }
+            case CARET: {
+                fid = AsterixBuiltinFunctions.CARET;
+                break;
+            }
+            case AND: {
+                fid = AlgebricksBuiltinFunctions.AND;
+                break;
+            }
+            case OR: {
+                fid = AlgebricksBuiltinFunctions.OR;
+                break;
+            }
+            case FUZZY_EQ: {
+                fid = AsterixBuiltinFunctions.FUZZY_EQ;
+                break;
+            }
 
-		default: {
-			throw new NotImplementedException("Operator " + t
-					+ " is not yet implemented");
-		}
-		}
-		return new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-	}
+            default: {
+                throw new NotImplementedException("Operator " + t + " is not yet implemented");
+            }
+        }
+        return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+    }
 
-	private static boolean hasOnlyChild(ILogicalOperator parent,
-			Mutable<ILogicalOperator> childCandidate) {
-		List<Mutable<ILogicalOperator>> inp = parent.getInputs();
-		if (inp == null || inp.size() != 1) {
-			return false;
-		}
-		return inp.get(0) == childCandidate;
-	}
+    private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
+        List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+        if (inp == null || inp.size() != 1) {
+            return false;
+        }
+        return inp.get(0) == childCandidate;
+    }
 
-	private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
-			Expression expr, Mutable<ILogicalOperator> topOp)
-			throws AsterixException {
-		switch (expr.getKind()) {
-		case VARIABLE_EXPRESSION: {
-			VariableReferenceExpression ve = new VariableReferenceExpression(
-					context.getVar(((VariableExpr) expr).getVar().getId()));
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
-					topOp);
-		}
-		case METAVARIABLE_EXPRESSION: {
-			ILogicalExpression le = metaScopeExp
-					.getVariableReferenceExpression(((VariableExpr) expr)
-							.getVar());
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le,
-					topOp);
-		}
-		case LITERAL_EXPRESSION: {
-			LiteralExpr val = (LiteralExpr) expr;
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-					new ConstantExpression(new AsterixConstantValue(
-							ConstantHelper.objectFromLiteral(val.getValue()))),
-					topOp);
-		}
-		default: {
-			// Mutable<ILogicalExpression> src = new
-			// Mutable<ILogicalExpression>();
-			// Mutable<ILogicalExpression> src = topOp;
-			if (expressionNeedsNoNesting(expr)) {
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						topOp);
-				ILogicalExpression exp = ((AssignOperator) p.first)
-						.getExpressions().get(0).getValue();
-				return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-						exp, p.first.getInputs().get(0));
-			} else {
-				Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+    private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
+            Mutable<ILogicalOperator> topOp) throws AsterixException {
+        switch (expr.getKind()) {
+            case VARIABLE_EXPRESSION: {
+                VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
+                        .getVar().getId()));
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
+            }
+            case METAVARIABLE_EXPRESSION: {
+                ILogicalExpression le = metaScopeExp.getVariableReferenceExpression(((VariableExpr) expr).getVar());
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le, topOp);
+            }
+            case LITERAL_EXPRESSION: {
+                LiteralExpr val = (LiteralExpr) expr;
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
+                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
+            }
+            default: {
+                // Mutable<ILogicalExpression> src = new
+                // Mutable<ILogicalExpression>();
+                // Mutable<ILogicalExpression> src = topOp;
+                if (expressionNeedsNoNesting(expr)) {
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
+                    ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
+                    return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
+                } else {
+                    Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
 
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						src);
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
 
-				if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-					// src.setOperator(topOp.getOperator());
-					Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
-							p.first);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second), top2);
-				} else {
-					SubplanOperator s = new SubplanOperator();
-					s.getInputs().add(topOp);
-					src.setValue(new NestedTupleSourceOperator(
-							new MutableObject<ILogicalOperator>(s)));
-					Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
-							p.first);
-					s.setRootOp(planRoot);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second),
-							new MutableObject<ILogicalOperator>(s));
-				}
-			}
-		}
-		}
+                    if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                        // src.setOperator(topOp.getOperator());
+                        Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), top2);
+                    } else {
+                        SubplanOperator s = new SubplanOperator();
+                        s.getInputs().add(topOp);
+                        src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+                        Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
+                        s.setRootOp(planRoot);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), new MutableObject<ILogicalOperator>(s));
+                    }
+                }
+            }
+        }
 
-	}
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
-			boolean noForClause, boolean isTop,
-			Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
-		if (isTop) {
-			ProjectOperator pr = new ProjectOperator(resVar);
-			pr.getInputs().add(resOpRef);
-			return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+    private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
+            Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+        if (isTop) {
+            ProjectOperator pr = new ProjectOperator(resVar);
+            pr.getInputs().add(resOpRef);
+            return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
 
-		} else if (noForClause) {
-			return new Pair<ILogicalOperator, LogicalVariable>(
-					resOpRef.getValue(), resVar);
-		} else {
-			return aggListify(resVar, resOpRef, false);
-		}
-	}
+        } else if (noForClause) {
+            return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
+        } else {
+            return aggListify(resVar, resOpRef, false);
+        }
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> aggListify(
-			LogicalVariable var, Mutable<ILogicalOperator> opRef,
-			boolean bProject) {
-		AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY,
-						new ArrayList<Mutable<ILogicalExpression>>());
-		funAgg.getArguments().add(
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(var)));
-		LogicalVariable varListified = context.newVar();
-		AggregateOperator agg = new AggregateOperator(
-				mkSingletonArrayList(varListified),
-				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-						funAgg)));
-		agg.getInputs().add(opRef);
-		ILogicalOperator res;
-		if (bProject) {
-			ProjectOperator pr = new ProjectOperator(varListified);
-			pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
-			res = pr;
-		} else {
-			res = agg;
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
-	}
+    private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
+            boolean bProject) {
+        AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
+        funAgg.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+        LogicalVariable varListified = context.newVar();
+        AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
+        agg.getInputs().add(opRef);
+        ILogicalOperator res;
+        if (bProject) {
+            ProjectOperator pr = new ProjectOperator(varListified);
+            pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+            res = pr;
+        } else {
+            res = agg;
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+    private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-		ArrayList<Expression> exprs = op.getExprList();
+        ArrayList<Expression> exprs = op.getExprList();
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		OperatorType opLogical = ops.get(0);
-		AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+        OperatorType opLogical = ops.get(0);
+        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
 
-		for (int i = 0; i <= nOps; i++) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			// now look at the operator
-			if (i < nOps) {
-				if (ops.get(i) != opLogical) {
-					throw new TranslationException("Unexpected operator "
-							+ ops.get(i) + " in an OperatorExpr starting with "
-							+ opLogical);
-				}
-			}
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		}
+        for (int i = 0; i <= nOps; i++) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            // now look at the operator
+            if (i < nOps) {
+                if (ops.get(i) != opLogical) {
+                    throw new TranslationException("Unexpected operator " + ops.get(i)
+                            + " in an OperatorExpr starting with " + opLogical);
+                }
+            }
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        }
 
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(f));
-		a.getInputs().add(topOp);
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
+        a.getInputs().add(topOp);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
 
-	}
+    }
 
-	private static boolean expressionNeedsNoNesting(Expression expr) {
-		Kind k = expr.getKind();
-		return k == Kind.LITERAL_EXPRESSION
-				|| k == Kind.LIST_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
-				|| k == Kind.OP_EXPRESSION
-				|| k == Kind.FIELD_ACCESSOR_EXPRESSION
-				|| k == Kind.INDEX_ACCESSOR_EXPRESSION
-				|| k == Kind.UNARY_EXPRESSION;
-	}
+    private static boolean expressionNeedsNoNesting(Expression expr) {
+        Kind k = expr.getKind();
+        return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+                || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
+                || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
+                || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
+    }
 
-	private <T> ArrayList<T> mkSingletonArrayList(T item) {
-		ArrayList<T> array = new ArrayList<T>(1);
-		array.add(item);
-		return array;
-	}
+    private <T> ArrayList<T> mkSingletonArrayList(T item) {
+        ArrayList<T> array = new ArrayList<T>(1);
+        array.add(item);
+        return array;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
-			RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
-			TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
-			NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
-			LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
-			DropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
-			CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
-			OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
-			UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(
-			MetaVariableClause mc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc
-				.getVar()), null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(MetaVariableClause mc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc.getVar()), null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(
-			JoinClause jc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		// Pair<ILogicalOperator, LogicalVariable> leftSide =
-		// jc.getLeftExpr().accept(this, tupSource);
-		Mutable<ILogicalOperator> opRef = tupSource;
-		Pair<ILogicalOperator, LogicalVariable> leftSide = null;
-		for (Clause c : jc.getLeftClauses()) {
-			leftSide = c.accept(this, opRef);
-			opRef = new MutableObject<ILogicalOperator>(leftSide.first);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(JoinClause jc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        // Pair<ILogicalOperator, LogicalVariable> leftSide =
+        // jc.getLeftExpr().accept(this, tupSource);
+        Mutable<ILogicalOperator> opRef = tupSource;
+        Pair<ILogicalOperator, LogicalVariable> leftSide = null;
+        for (Clause c : jc.getLeftClauses()) {
+            leftSide = c.accept(this, opRef);
+            opRef = new MutableObject<ILogicalOperator>(leftSide.first);
+        }
 
-		// Pair<ILogicalOperator, LogicalVariable> rightSide =
-		// jc.getRightExpr().accept(this, tupSource);
-		opRef = tupSource;
-		Pair<ILogicalOperator, LogicalVariable> rightSide = null;
-		for (Clause c : jc.getRightClauses()) {
-			rightSide = c.accept(this, opRef);
-			opRef = new MutableObject<ILogicalOperator>(rightSide.first);
-		}
+        // Pair<ILogicalOperator, LogicalVariable> rightSide =
+        // jc.getRightExpr().accept(this, tupSource);
+        opRef = tupSource;
+        Pair<ILogicalOperator, LogicalVariable> rightSide = null;
+        for (Clause c : jc.getRightClauses()) {
+            rightSide = c.accept(this, opRef);
+            opRef = new MutableObject<ILogicalOperator>(rightSide.first);
+        }
 
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(
-				jc.getWhereExpr(), tupSource);
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(jc.getWhereExpr(),
+                tupSource);
 
-		AbstractBinaryJoinOperator join;
-		switch (jc.getKind()) {
-		case INNER: {
-			join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(
-					whereCond.first));
-			break;
-		}
-		case LEFT_OUTER: {
-			join = new LeftOuterJoinOperator(
-					new MutableObject<ILogicalExpression>(whereCond.first));
-			break;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-		join.getInputs().add(
-				new MutableObject<ILogicalOperator>(leftSide.first));
-		join.getInputs().add(
-				new MutableObject<ILogicalOperator>(rightSide.first));
-		return new Pair<ILogicalOperator, LogicalVariable>(join, null);
-	}
+        AbstractBinaryJoinOperator join;
+        switch (jc.getKind()) {
+            case INNER: {
+                join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
+                break;
+            }
+            case LEFT_OUTER: {
+                join = new LeftOuterJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+        join.getInputs().add(new MutableObject<ILogicalOperator>(leftSide.first));
+        join.getInputs().add(new MutableObject<ILogicalOperator>(rightSide.first));
+        return new Pair<ILogicalOperator, LogicalVariable>(join, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(
-			MetaVariableExpr me, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		LogicalVariable var = context.newVar();
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(metaScopeExp
-						.getVariableReferenceExpression(me.getVar())));
-		a.getInputs().add(tupSource);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(MetaVariableExpr me,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        LogicalVariable var = context.newVar();
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+                metaScopeExp.getVariableReferenceExpression(me.getVar())));
+        a.getInputs().add(tupSource);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
-		metaScopeOp.put(id, op);
-	}
+    public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
+        metaScopeOp.put(id, op);
+    }
 
-	public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
-		metaScopeExp.put(id, var);
-	}
+    public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
+        metaScopeExp.put(id, var);
+    }
 
-	private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
-		switch (expr.getExpressionTag()) {
-		case VARIABLE: {
-			return new UnnestingFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-					new MutableObject<ILogicalExpression>(expr));
-		}
-		case FUNCTION_CALL: {
-			AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-			if (fce.getKind() == FunctionKind.UNNEST) {
-				return expr;
-			} else {
-				return new UnnestingFunctionCallExpression(
-						FunctionUtils
-								.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-						new MutableObject<ILogicalExpression>(expr));
-			}
-		}
-		default: {
-			return expr;
-		}
-		}
-	}
+    private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+        switch (expr.getExpressionTag()) {
+            case VARIABLE: {
+                return new UnnestingFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                        new MutableObject<ILogicalExpression>(expr));
+            }
+            case FUNCTION_CALL: {
+                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+                if (fce.getKind() == FunctionKind.UNNEST) {
+                    return expr;
+                } else {
+                    return new UnnestingFunctionCallExpression(
+                            FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                            new MutableObject<ILogicalExpression>(expr));
+                }
+            }
+            default: {
+                return expr;
+            }
+        }
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
-			InsertStatement insert, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
-			DeleteStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
-			UpdateStatement update, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
-			UpdateClause del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
-			DataverseDecl dv, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
-			DatasetDecl dd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
-			SetStatement ss, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
-			WriteStatement ws, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
-			CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
-			IndexDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
-			NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
-			DataverseDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
-			TypeDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
-			ControlFeedStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visit(
-			CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
-			FunctionDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
-			BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
index 57b2ff0..60b7e6c 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
@@ -33,7 +33,7 @@
 import edu.uci.ics.asterix.optimizer.base.RuleCollections;
 import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.translator.AqlExpressionToPlanTranslator;
 import edu.uci.ics.asterix.translator.DmlTranslator;
@@ -102,6 +102,8 @@
                 RuleCollections.buildConsolidationRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
                 RuleCollections.buildOpPushDownRuleCollection()));
+        
+        //put TXnRuleCollection!
         return defaultLogicalRewrites;
     }
 
@@ -149,7 +151,7 @@
         // Begin a transaction against the metadata.
         // Lock the metadata in X mode to protect against other DDL and DML.
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.EXCLUSIVE);
+        MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
         try {
             DdlTranslator ddlt = new DdlTranslator(mdTxnCtx, query.getPrologDeclList(), out, pc, pdf);
             ddlt.translate(hcc, false);
@@ -170,7 +172,7 @@
         // Lock the metadata in S mode to protect against other DDL
         // modifications.
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.SHARED);
+        MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.S);
         try {
             DmlTranslator dmlt = new DmlTranslator(mdTxnCtx, query.getPrologDeclList());
             dmlt.translate();
@@ -305,7 +307,7 @@
             JSONException, RemoteException, ACIDException {
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         try {
-            MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.SHARED);
+            MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.S);
             Pair<AqlCompiledMetadataDeclarations, JobSpecification> result = compileQueryInternal(mdTxnCtx,
                     dataverseName, q, varCounter, outputDatasetName, metadataDecls, pc, out, pdf, dmlKind);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -390,8 +392,8 @@
             }
 
         }
-        long txnId = TransactionIDFactory.generateTransactionId();
-        AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(txnId, mdTxnCtx, rw.getVarCounter(),
+        edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory.generateJobId();
+        AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(asterixJobId, mdTxnCtx, rw.getVarCounter(),
                 outputDatasetName, dmlKind);
 
         ILogicalPlanAndMetadata planAndMetadata = t.translate(rwQ, metadataDecls);
@@ -526,7 +528,7 @@
 
         JobSpecification spec = compiler.createJob(AsterixAppContextInfoImpl.INSTANCE);
         // set the job event listener
-        spec.setJobletEventListenerFactory(new JobEventListenerFactory(txnId, isWriteTransaction));
+        spec.setJobletEventListenerFactory(new JobEventListenerFactory(asterixJobId, isWriteTransaction));
         if (pc.isPrintJob()) {
             switch (pdf) {
                 case HTML: {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
index c68817c..6f02b90 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
@@ -82,6 +82,7 @@
 import edu.uci.ics.asterix.om.types.AbstractCollectionType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
 import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -93,1090 +94,924 @@
 
 public class DdlTranslator extends AbstractAqlTranslator {
 
-	private final MetadataTransactionContext mdTxnCtx;
-	private final List<Statement> aqlStatements;
-	private final PrintWriter out;
-	private final SessionConfig pc;
-	private final DisplayFormat pdf;
-	private AqlCompiledMetadataDeclarations compiledDeclarations;
+    private final MetadataTransactionContext mdTxnCtx;
+    private final List<Statement> aqlStatements;
+    private final PrintWriter out;
+    private final SessionConfig pc;
+    private final DisplayFormat pdf;
+    private AqlCompiledMetadataDeclarations compiledDeclarations;
 
-	private static Map<String, BuiltinType> builtinTypeMap;
+    private static Map<String, BuiltinType> builtinTypeMap;
 
-	public DdlTranslator(MetadataTransactionContext mdTxnCtx,
-			List<Statement> aqlStatements, PrintWriter out, SessionConfig pc,
-			DisplayFormat pdf) {
-		this.mdTxnCtx = mdTxnCtx;
-		this.aqlStatements = aqlStatements;
-		this.out = out;
-		this.pc = pc;
-		this.pdf = pdf;
-		builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
-	}
+    public DdlTranslator(MetadataTransactionContext mdTxnCtx, List<Statement> aqlStatements, PrintWriter out,
+            SessionConfig pc, DisplayFormat pdf) {
+        this.mdTxnCtx = mdTxnCtx;
+        this.aqlStatements = aqlStatements;
+        this.out = out;
+        this.pc = pc;
+        this.pdf = pdf;
+        builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
+    }
 
-	public void translate(IHyracksClientConnection hcc,
-			boolean disconnectFromDataverse) throws AlgebricksException {
-		try {
-			compiledDeclarations = compileMetadata(mdTxnCtx, aqlStatements,
-					true);
-			compileAndExecuteDDLstatements(hcc, mdTxnCtx,
-					disconnectFromDataverse);
-		} catch (Exception e) {
-			throw new AlgebricksException(e);
-		}
-	}
+    public void translate(IHyracksClientConnection hcc, boolean disconnectFromDataverse) throws AlgebricksException {
+        try {
+            compiledDeclarations = compileMetadata(mdTxnCtx, aqlStatements, true);
+            compileAndExecuteDDLstatements(hcc, mdTxnCtx, disconnectFromDataverse);
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        }
+    }
 
-	private void compileAndExecuteDDLstatements(IHyracksClientConnection hcc,
-			MetadataTransactionContext mdTxnCtx, boolean disconnectFromDataverse)
-			throws Exception {
-		for (Statement stmt : aqlStatements) {
-			validateOperation(compiledDeclarations, stmt);
-			switch (stmt.getKind()) {
-			case DATAVERSE_DECL: {
-				checkForDataverseConnection(false);
-				DataverseDecl dvd = (DataverseDecl) stmt;
-				String dataverseName = dvd.getDataverseName().getValue();
-				compiledDeclarations.connectToDataverse(dataverseName);
-				break;
-			}
+    private void compileAndExecuteDDLstatements(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
+            boolean disconnectFromDataverse) throws Exception {
+        for (Statement stmt : aqlStatements) {
+            validateOperation(compiledDeclarations, stmt);
+            switch (stmt.getKind()) {
+                case DATAVERSE_DECL: {
+                    checkForDataverseConnection(false);
+                    DataverseDecl dvd = (DataverseDecl) stmt;
+                    String dataverseName = dvd.getDataverseName().getValue();
+                    compiledDeclarations.connectToDataverse(dataverseName);
+                    break;
+                }
 
-			case CREATE_DATAVERSE: {
-				checkForDataverseConnection(false);
-				CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
-				String dvName = stmtCreateDataverse.getDataverseName()
-						.getValue();
-				Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx,
-						dvName);
-				if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
-					throw new AlgebricksException("A dataverse with this name "
-							+ dvName + " already exists.");
-				}
-				MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(
-						dvName, stmtCreateDataverse.getFormat()));
-				break;
-			}
+                case CREATE_DATAVERSE: {
+                    checkForDataverseConnection(false);
+                    CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
+                    String dvName = stmtCreateDataverse.getDataverseName().getValue();
+                    Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
+                    if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
+                        throw new AlgebricksException("A dataverse with this name " + dvName + " already exists.");
+                    }
+                    MetadataManager.INSTANCE.addDataverse(mdTxnCtx,
+                            new Dataverse(dvName, stmtCreateDataverse.getFormat()));
+                    break;
+                }
 
-			case DATASET_DECL: {
-				checkForDataverseConnection(true);
-				DatasetDecl dd = (DatasetDecl) stmt;
-				String datasetName = dd.getName().getValue();
-				DatasetType dsType = dd.getDatasetType();
-				String itemTypeName = null;
-				IDatasetDetails datasetDetails = null;
-				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), datasetName);
-				if (ds != null) {
-					if (dd.getIfNotExists()) {
-						continue;
-					} else {
-						throw new AlgebricksException(
-								"A dataset with this name " + datasetName
-										+ " already exists.");
-					}
-				}
-				itemTypeName = dd.getItemTypeName().getValue();
-				Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), itemTypeName);
-				if (dt == null) {
-					throw new AlgebricksException(": type " + itemTypeName
-							+ " could not be found.");
-				}
-				switch (dd.getDatasetType()) {
-				case INTERNAL: {
-					IAType itemType = dt.getDatatype();
-					if (itemType.getTypeTag() != ATypeTag.RECORD) {
-						throw new AlgebricksException(
-								"Can only partition ARecord's.");
-					}
-					List<String> partitioningExprs = ((InternalDetailsDecl) dd
-							.getDatasetDetailsDecl()).getPartitioningExprs();
-					String ngName = ((InternalDetailsDecl) dd
-							.getDatasetDetailsDecl()).getNodegroupName()
-							.getValue();
-					datasetDetails = new InternalDatasetDetails(
-							InternalDatasetDetails.FileStructure.BTREE,
-							InternalDatasetDetails.PartitioningStrategy.HASH,
-							partitioningExprs, partitioningExprs, ngName);
-					break;
-				}
-				case EXTERNAL: {
-					String adapter = ((ExternalDetailsDecl) dd
-							.getDatasetDetailsDecl()).getAdapter();
-					Map<String, String> properties = ((ExternalDetailsDecl) dd
-							.getDatasetDetailsDecl()).getProperties();
-					datasetDetails = new ExternalDatasetDetails(adapter,
-							properties);
-					break;
-				}
-				case FEED: {
-					IAType itemType = dt.getDatatype();
-					if (itemType.getTypeTag() != ATypeTag.RECORD) {
-						throw new AlgebricksException(
-								"Can only partition ARecord's.");
-					}
-					List<String> partitioningExprs = ((FeedDetailsDecl) dd
-							.getDatasetDetailsDecl()).getPartitioningExprs();
-					String ngName = ((FeedDetailsDecl) dd
-							.getDatasetDetailsDecl()).getNodegroupName()
-							.getValue();
-					String adapter = ((FeedDetailsDecl) dd
-							.getDatasetDetailsDecl()).getAdapterClassname();
-					Map<String, String> properties = ((FeedDetailsDecl) dd
-							.getDatasetDetailsDecl()).getProperties();
-					String functionIdentifier = ((FeedDetailsDecl) dd
-							.getDatasetDetailsDecl()).getFunctionIdentifier();
-					datasetDetails = new FeedDatasetDetails(
-							InternalDatasetDetails.FileStructure.BTREE,
-							InternalDatasetDetails.PartitioningStrategy.HASH,
-							partitioningExprs, partitioningExprs, ngName,
-							adapter, properties, functionIdentifier,
-							FeedDatasetDetails.FeedState.INACTIVE.toString());
-					break;
-				}
-				}
-				MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(
-						compiledDeclarations.getDataverseName(), datasetName,
-						itemTypeName, datasetDetails, dsType));
-				if (dd.getDatasetType() == DatasetType.INTERNAL
-						|| dd.getDatasetType() == DatasetType.FEED) {
-					runCreateDatasetJob(hcc, datasetName);
-				}
-				break;
-			}
+                case DATASET_DECL: {
+                    checkForDataverseConnection(true);
+                    DatasetDecl dd = (DatasetDecl) stmt;
+                    String datasetName = dd.getName().getValue();
+                    DatasetType dsType = dd.getDatasetType();
+                    String itemTypeName = null;
+                    IDatasetDetails datasetDetails = null;
+                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                            datasetName);
+                    if (ds != null) {
+                        if (dd.getIfNotExists()) {
+                            continue;
+                        } else {
+                            throw new AlgebricksException("A dataset with this name " + datasetName
+                                    + " already exists.");
+                        }
+                    }
+                    itemTypeName = dd.getItemTypeName().getValue();
+                    Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+                            compiledDeclarations.getDataverseName(), itemTypeName);
+                    if (dt == null) {
+                        throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
+                    }
+                    switch (dd.getDatasetType()) {
+                        case INTERNAL: {
+                            IAType itemType = dt.getDatatype();
+                            if (itemType.getTypeTag() != ATypeTag.RECORD) {
+                                throw new AlgebricksException("Can only partition ARecord's.");
+                            }
+                            List<String> partitioningExprs = ((InternalDetailsDecl) dd.getDatasetDetailsDecl())
+                                    .getPartitioningExprs();
+                            String ngName = ((InternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName()
+                                    .getValue();
+                            datasetDetails = new InternalDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
+                                    InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs,
+                                    partitioningExprs, ngName);
+                            break;
+                        }
+                        case EXTERNAL: {
+                            String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
+                            Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl())
+                                    .getProperties();
+                            datasetDetails = new ExternalDatasetDetails(adapter, properties);
+                            break;
+                        }
+                        case FEED: {
+                            IAType itemType = dt.getDatatype();
+                            if (itemType.getTypeTag() != ATypeTag.RECORD) {
+                                throw new AlgebricksException("Can only partition ARecord's.");
+                            }
+                            List<String> partitioningExprs = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
+                                    .getPartitioningExprs();
+                            String ngName = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName()
+                                    .getValue();
+                            String adapter = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getAdapterClassname();
+                            Map<String, String> properties = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
+                                    .getProperties();
+                            String functionIdentifier = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
+                                    .getFunctionIdentifier();
+                            datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
+                                    InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs,
+                                    partitioningExprs, ngName, adapter, properties, functionIdentifier,
+                                    FeedDatasetDetails.FeedState.INACTIVE.toString());
+                            break;
+                        }
+                    }
+                    MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(compiledDeclarations.getDataverseName(),
+                            datasetName, itemTypeName, datasetDetails, dsType, DatasetIdFactory.generateDatasetId()));
+                    if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
+                        runCreateDatasetJob(hcc, datasetName);
+                    }
+                    break;
+                }
 
-			case CREATE_INDEX: {
-				checkForDataverseConnection(true);
-				CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
-				String datasetName = stmtCreateIndex.getDatasetName()
-						.getValue();
-				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), datasetName);
-				if (ds == null) {
-					throw new AlgebricksException(
-							"There is no dataset with this name " + datasetName);
-				}
-				String indexName = stmtCreateIndex.getIndexName().getValue();
-				Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), datasetName,
-						indexName);
-				if (idx != null) {
-					if (!stmtCreateIndex.getIfNotExists()) {
-						throw new AlgebricksException(
-								"An index with this name " + indexName
-										+ " already exists.");
-					} else {
-						stmtCreateIndex.setNeedToCreate(false);
-					}
-				} else {
-					MetadataManager.INSTANCE.addIndex(
-							mdTxnCtx,
-							new Index(compiledDeclarations.getDataverseName(),
-									datasetName, indexName, stmtCreateIndex
-											.getIndexType(), stmtCreateIndex
-											.getFieldExprs(), stmtCreateIndex
-											.getGramLength(), false));
-					runCreateIndexJob(hcc, stmtCreateIndex);
-				}
-				break;
-			}
-			case TYPE_DECL: {
-				checkForDataverseConnection(true);
-				TypeDecl stmtCreateType = (TypeDecl) stmt;
-				String typeName = stmtCreateType.getIdent().getValue();
-				Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), typeName);
-				if (dt != null) {
-					if (!stmtCreateType.getIfNotExists())
-						throw new AlgebricksException(
-								"A datatype with this name " + typeName
-										+ " already exists.");
-				} else {
-					if (builtinTypeMap.get(typeName) != null) {
-						throw new AlgebricksException(
-								"Cannot redefine builtin type " + typeName
-										+ ".");
-					} else {
-						Map<String, IAType> typeMap = computeTypes(mdTxnCtx,
-								(TypeDecl) stmt);
-						IAType type = typeMap.get(typeName);
-						MetadataManager.INSTANCE.addDatatype(
-								mdTxnCtx,
-								new Datatype(compiledDeclarations
-										.getDataverseName(), typeName, type,
-										false));
-					}
-				}
-				break;
-			}
-			case NODEGROUP_DECL: {
-				NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
-				String ngName = stmtCreateNodegroup.getNodegroupName()
-						.getValue();
-				NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx,
-						ngName);
-				if (ng != null) {
-					if (!stmtCreateNodegroup.getIfNotExists())
-						throw new AlgebricksException(
-								"A nodegroup with this name " + ngName
-										+ " already exists.");
-				} else {
-					List<Identifier> ncIdentifiers = stmtCreateNodegroup
-							.getNodeControllerNames();
-					List<String> ncNames = new ArrayList<String>(
-							ncIdentifiers.size());
-					for (Identifier id : ncIdentifiers) {
-						ncNames.add(id.getValue());
-					}
-					MetadataManager.INSTANCE.addNodegroup(mdTxnCtx,
-							new NodeGroup(ngName, ncNames));
-				}
-				break;
-			}
-			// drop statements
-			case DATAVERSE_DROP: {
-				DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
-				String dvName = stmtDelete.getDataverseName().getValue();
-				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
-						ARTIFACT_KIND.DATAVERSE, dvName)) {
-					throw new AsterixException(
-							"Invalid Operation cannot drop dataverse " + dvName
-									+ " (protected by system)");
-				}
+                case CREATE_INDEX: {
+                    checkForDataverseConnection(true);
+                    CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
+                    String datasetName = stmtCreateIndex.getDatasetName().getValue();
+                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                            datasetName);
+                    if (ds == null) {
+                        throw new AlgebricksException("There is no dataset with this name " + datasetName);
+                    }
+                    String indexName = stmtCreateIndex.getIndexName().getValue();
+                    Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                            datasetName, indexName);
+                    if (idx != null) {
+                        if (!stmtCreateIndex.getIfNotExists()) {
+                            throw new AlgebricksException("An index with this name " + indexName + " already exists.");
+                        } else {
+                            stmtCreateIndex.setNeedToCreate(false);
+                        }
+                    } else {
+                        MetadataManager.INSTANCE.addIndex(mdTxnCtx, new Index(compiledDeclarations.getDataverseName(),
+                                datasetName, indexName, stmtCreateIndex.getIndexType(),
+                                stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), false));
+                        runCreateIndexJob(hcc, stmtCreateIndex);
+                    }
+                    break;
+                }
+                case TYPE_DECL: {
+                    checkForDataverseConnection(true);
+                    TypeDecl stmtCreateType = (TypeDecl) stmt;
+                    String typeName = stmtCreateType.getIdent().getValue();
+                    Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+                            compiledDeclarations.getDataverseName(), typeName);
+                    if (dt != null) {
+                        if (!stmtCreateType.getIfNotExists())
+                            throw new AlgebricksException("A datatype with this name " + typeName + " already exists.");
+                    } else {
+                        if (builtinTypeMap.get(typeName) != null) {
+                            throw new AlgebricksException("Cannot redefine builtin type " + typeName + ".");
+                        } else {
+                            Map<String, IAType> typeMap = computeTypes(mdTxnCtx, (TypeDecl) stmt);
+                            IAType type = typeMap.get(typeName);
+                            MetadataManager.INSTANCE.addDatatype(mdTxnCtx,
+                                    new Datatype(compiledDeclarations.getDataverseName(), typeName, type, false));
+                        }
+                    }
+                    break;
+                }
+                case NODEGROUP_DECL: {
+                    NodegroupDecl stmtCreateNodegroup = (NodegroupDecl) stmt;
+                    String ngName = stmtCreateNodegroup.getNodegroupName().getValue();
+                    NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, ngName);
+                    if (ng != null) {
+                        if (!stmtCreateNodegroup.getIfNotExists())
+                            throw new AlgebricksException("A nodegroup with this name " + ngName + " already exists.");
+                    } else {
+                        List<Identifier> ncIdentifiers = stmtCreateNodegroup.getNodeControllerNames();
+                        List<String> ncNames = new ArrayList<String>(ncIdentifiers.size());
+                        for (Identifier id : ncIdentifiers) {
+                            ncNames.add(id.getValue());
+                        }
+                        MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, new NodeGroup(ngName, ncNames));
+                    }
+                    break;
+                }
+                // drop statements
+                case DATAVERSE_DROP: {
+                    DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
+                    String dvName = stmtDelete.getDataverseName().getValue();
+                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.DATAVERSE, dvName)) {
+                        throw new AsterixException("Invalid Operation cannot drop dataverse " + dvName
+                                + " (protected by system)");
+                    }
 
-				if (compiledDeclarations.isConnectedToDataverse())
-					compiledDeclarations.disconnectFromDataverse();
-				checkForDataverseConnection(false);
+                    if (compiledDeclarations.isConnectedToDataverse())
+                        compiledDeclarations.disconnectFromDataverse();
+                    checkForDataverseConnection(false);
 
-				Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx,
-						dvName);
-				if (dv == null) {
-					if (!stmtDelete.getIfExists()) {
-						throw new AlgebricksException(
-								"There is no dataverse with this name "
-										+ dvName + ".");
-					}
-				} else {
-					compiledDeclarations.connectToDataverse(dvName);
-					List<Dataset> datasets = MetadataManager.INSTANCE
-							.getDataverseDatasets(mdTxnCtx, dvName);
-					for (int j = 0; j < datasets.size(); j++) {
-						String datasetName = datasets.get(j).getDatasetName();
-						DatasetType dsType = datasets.get(j).getDatasetType();
-						if (dsType == DatasetType.INTERNAL
-								|| dsType == DatasetType.FEED) {
-							List<Index> indexes = MetadataManager.INSTANCE
-									.getDatasetIndexes(mdTxnCtx, dvName,
-											datasetName);
-							for (int k = 0; k < indexes.size(); k++) {
-								if (indexes.get(k).isSecondaryIndex()) {
-									compileIndexDropStatement(hcc, mdTxnCtx,
-											datasetName, indexes.get(k)
-													.getIndexName());
-								}
-							}
-						}
-						compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
-					}
-					MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
-					if (compiledDeclarations.isConnectedToDataverse())
-						compiledDeclarations.disconnectFromDataverse();
-				}
-				break;
-			}
-			case DATASET_DROP: {
-				checkForDataverseConnection(true);
-				DropStatement stmtDelete = (DropStatement) stmt;
-				String datasetName = stmtDelete.getDatasetName().getValue();
-				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
-						ARTIFACT_KIND.DATASET, datasetName)) {
-					throw new AsterixException(
-							"Invalid Operation cannot drop dataset "
-									+ datasetName + " (protected by system)");
-				}
-				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), datasetName);
-				if (ds == null) {
-					if (!stmtDelete.getIfExists())
-						throw new AlgebricksException(
-								"There is no dataset with this name "
-										+ datasetName + ".");
-				} else {
-					if (ds.getDatasetType() == DatasetType.INTERNAL
-							|| ds.getDatasetType() == DatasetType.FEED) {
-						List<Index> indexes = MetadataManager.INSTANCE
-								.getDatasetIndexes(
-										mdTxnCtx,
-										compiledDeclarations.getDataverseName(),
-										datasetName);
-						for (int j = 0; j < indexes.size(); j++) {
-							if (indexes.get(j).isPrimaryIndex()) {
-								compileIndexDropStatement(hcc, mdTxnCtx,
-										datasetName, indexes.get(j)
-												.getIndexName());
-							}
-						}
-					}
-					compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
-				}
-				break;
-			}
-			case INDEX_DROP: {
-				checkForDataverseConnection(true);
-				IndexDropStatement stmtDelete = (IndexDropStatement) stmt;
-				String datasetName = stmtDelete.getDatasetName().getValue();
-				Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), datasetName);
-				if (ds == null)
-					throw new AlgebricksException(
-							"There is no dataset with this name " + datasetName
-									+ ".");
-				if (ds.getDatasetType() == DatasetType.INTERNAL
-						|| ds.getDatasetType() == DatasetType.FEED) {
-					String indexName = stmtDelete.getIndexName().getValue();
-					Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
-							compiledDeclarations.getDataverseName(),
-							datasetName, indexName);
-					if (idx == null) {
-						if (!stmtDelete.getIfExists())
-							throw new AlgebricksException(
-									"There is no index with this name "
-											+ indexName + ".");
-					} else
-						compileIndexDropStatement(hcc, mdTxnCtx, datasetName,
-								indexName);
-				} else {
-					throw new AlgebricksException(
-							datasetName
-									+ " is an external dataset. Indexes are not maintained for external datasets.");
-				}
-				break;
-			}
-			case TYPE_DROP: {
-				checkForDataverseConnection(true);
-				TypeDropStatement stmtDelete = (TypeDropStatement) stmt;
-				String typeName = stmtDelete.getTypeName().getValue();
-				Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-						compiledDeclarations.getDataverseName(), typeName);
-				if (dt == null) {
-					if (!stmtDelete.getIfExists())
-						throw new AlgebricksException(
-								"There is no datatype with this name "
-										+ typeName + ".");
-				} else
-					MetadataManager.INSTANCE.dropDatatype(mdTxnCtx,
-							compiledDeclarations.getDataverseName(), typeName);
-				break;
-			}
-			case NODEGROUP_DROP: {
-				NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
-				String nodegroupName = stmtDelete.getNodeGroupName().getValue();
-				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
-						ARTIFACT_KIND.NODEGROUP, nodegroupName)) {
-					throw new AsterixException(
-							"Invalid Operation cannot drop nodegroup "
-									+ nodegroupName + " (protected by system)");
-				}
-				NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx,
-						nodegroupName);
-				if (ng == null) {
-					if (!stmtDelete.getIfExists())
-						throw new AlgebricksException(
-								"There is no nodegroup with this name "
-										+ nodegroupName + ".");
-				} else
-					MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx,
-							nodegroupName);
-				break;
-			}
+                    Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
+                    if (dv == null) {
+                        if (!stmtDelete.getIfExists()) {
+                            throw new AlgebricksException("There is no dataverse with this name " + dvName + ".");
+                        }
+                    } else {
+                        compiledDeclarations.connectToDataverse(dvName);
+                        List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dvName);
+                        for (int j = 0; j < datasets.size(); j++) {
+                            String datasetName = datasets.get(j).getDatasetName();
+                            DatasetType dsType = datasets.get(j).getDatasetType();
+                            if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
+                                List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dvName,
+                                        datasetName);
+                                for (int k = 0; k < indexes.size(); k++) {
+                                    if (indexes.get(k).isSecondaryIndex()) {
+                                        compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexes.get(k)
+                                                .getIndexName());
+                                    }
+                                }
+                            }
+                            compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
+                        }
+                        MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
+                        if (compiledDeclarations.isConnectedToDataverse())
+                            compiledDeclarations.disconnectFromDataverse();
+                    }
+                    break;
+                }
+                case DATASET_DROP: {
+                    checkForDataverseConnection(true);
+                    DropStatement stmtDelete = (DropStatement) stmt;
+                    String datasetName = stmtDelete.getDatasetName().getValue();
+                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.DATASET, datasetName)) {
+                        throw new AsterixException("Invalid Operation cannot drop dataset " + datasetName
+                                + " (protected by system)");
+                    }
+                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                            datasetName);
+                    if (ds == null) {
+                        if (!stmtDelete.getIfExists())
+                            throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
+                    } else {
+                        if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+                            List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
+                                    compiledDeclarations.getDataverseName(), datasetName);
+                            for (int j = 0; j < indexes.size(); j++) {
+                                if (indexes.get(j).isPrimaryIndex()) {
+                                    compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexes.get(j).getIndexName());
+                                }
+                            }
+                        }
+                        compileDatasetDropStatement(hcc, mdTxnCtx, datasetName);
+                    }
+                    break;
+                }
+                case INDEX_DROP: {
+                    checkForDataverseConnection(true);
+                    IndexDropStatement stmtDelete = (IndexDropStatement) stmt;
+                    String datasetName = stmtDelete.getDatasetName().getValue();
+                    Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                            datasetName);
+                    if (ds == null)
+                        throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
+                    if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+                        String indexName = stmtDelete.getIndexName().getValue();
+                        Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
+                                compiledDeclarations.getDataverseName(), datasetName, indexName);
+                        if (idx == null) {
+                            if (!stmtDelete.getIfExists())
+                                throw new AlgebricksException("There is no index with this name " + indexName + ".");
+                        } else
+                            compileIndexDropStatement(hcc, mdTxnCtx, datasetName, indexName);
+                    } else {
+                        throw new AlgebricksException(datasetName
+                                + " is an external dataset. Indexes are not maintained for external datasets.");
+                    }
+                    break;
+                }
+                case TYPE_DROP: {
+                    checkForDataverseConnection(true);
+                    TypeDropStatement stmtDelete = (TypeDropStatement) stmt;
+                    String typeName = stmtDelete.getTypeName().getValue();
+                    Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
+                            compiledDeclarations.getDataverseName(), typeName);
+                    if (dt == null) {
+                        if (!stmtDelete.getIfExists())
+                            throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
+                    } else
+                        MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                                typeName);
+                    break;
+                }
+                case NODEGROUP_DROP: {
+                    NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
+                    String nodegroupName = stmtDelete.getNodeGroupName().getValue();
+                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.NODEGROUP, nodegroupName)) {
+                        throw new AsterixException("Invalid Operation cannot drop nodegroup " + nodegroupName
+                                + " (protected by system)");
+                    }
+                    NodeGroup ng = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, nodegroupName);
+                    if (ng == null) {
+                        if (!stmtDelete.getIfExists())
+                            throw new AlgebricksException("There is no nodegroup with this name " + nodegroupName + ".");
+                    } else
+                        MetadataManager.INSTANCE.dropNodegroup(mdTxnCtx, nodegroupName);
+                    break;
+                }
 
-			case CREATE_FUNCTION: {
-				CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
-				Function function = new Function(
-						compiledDeclarations.getDataverseName(), cfs
-								.getFunctionIdentifier().getFunctionName(), cfs
-								.getFunctionIdentifier().getArity(),
-						cfs.getParamList(), cfs.getFunctionBody());
-				try {
-					FunctionUtils.getFunctionDecl(function);
-				} catch (Exception e) {
-					throw new AsterixException(
-							"Unable to compile function definition", e);
-				}
-				MetadataManager.INSTANCE
-						.addFunction(mdTxnCtx, new Function(
-								compiledDeclarations.getDataverseName(), cfs
-										.getFunctionIdentifier()
-										.getFunctionName(), cfs
-										.getFunctionIdentifier().getArity(),
-								cfs.getParamList(), cfs.getFunctionBody()));
-				break;
-			}
+                case CREATE_FUNCTION: {
+                    CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
+                    Function function = new Function(compiledDeclarations.getDataverseName(), cfs
+                            .getFunctionIdentifier().getFunctionName(), cfs.getFunctionIdentifier().getArity(),
+                            cfs.getParamList(), cfs.getFunctionBody());
+                    try {
+                        FunctionUtils.getFunctionDecl(function);
+                    } catch (Exception e) {
+                        throw new AsterixException("Unable to compile function definition", e);
+                    }
+                    MetadataManager.INSTANCE.addFunction(mdTxnCtx, new Function(
+                            compiledDeclarations.getDataverseName(), cfs.getFunctionIdentifier().getFunctionName(), cfs
+                                    .getFunctionIdentifier().getArity(), cfs.getParamList(), cfs.getFunctionBody()));
+                    break;
+                }
 
-			case FUNCTION_DROP: {
-				checkForDataverseConnection(true);
-				FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
-				String functionName = stmtDropFunction.getFunctionName()
-						.getValue();
-                    FunctionIdentifier fId = new FunctionIdentifier(
-						FunctionConstants.ASTERIX_NS, functionName,
-						stmtDropFunction.getArity());
-				if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(
-						ARTIFACT_KIND.FUNCTION, fId)) {
-					throw new AsterixException(
-							"Invalid Operation cannot drop function "
-									+ functionName + " (protected by system)");
-				}
-				Function function = MetadataManager.INSTANCE.getFunction(
-						mdTxnCtx, compiledDeclarations.getDataverseName(),
-						functionName, stmtDropFunction.getArity());
-				if (function == null) {
-					if (!stmtDropFunction.getIfExists())
-						throw new AlgebricksException(
-								"There is no function with this name "
-										+ functionName + ".");
-				} else {
-					MetadataManager.INSTANCE.dropFunction(mdTxnCtx,
-							compiledDeclarations.getDataverseName(),
-							functionName, stmtDropFunction.getArity());
-				}
-				break;
-			}
-			}
-		}
+                case FUNCTION_DROP: {
+                    checkForDataverseConnection(true);
+                    FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
+                    String functionName = stmtDropFunction.getFunctionName().getValue();
+                    FunctionIdentifier fId = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, functionName,
+                            stmtDropFunction.getArity());
+                    if (AsterixBuiltinArtifactMap.isSystemProtectedArtifact(ARTIFACT_KIND.FUNCTION, fId)) {
+                        throw new AsterixException("Invalid Operation cannot drop function " + functionName
+                                + " (protected by system)");
+                    }
+                    Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx,
+                            compiledDeclarations.getDataverseName(), functionName, stmtDropFunction.getArity());
+                    if (function == null) {
+                        if (!stmtDropFunction.getIfExists())
+                            throw new AlgebricksException("There is no function with this name " + functionName + ".");
+                    } else {
+                        MetadataManager.INSTANCE.dropFunction(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                                functionName, stmtDropFunction.getArity());
+                    }
+                    break;
+                }
+            }
+        }
 
-		if (disconnectFromDataverse) {
-			if (compiledDeclarations.isConnectedToDataverse()) {
-				compiledDeclarations.disconnectFromDataverse();
-			}
-		}
-	}
+        if (disconnectFromDataverse) {
+            if (compiledDeclarations.isConnectedToDataverse()) {
+                compiledDeclarations.disconnectFromDataverse();
+            }
+        }
+    }
 
-	private void checkForDataverseConnection(boolean needConnection)
-			throws AlgebricksException {
-		if (compiledDeclarations.isConnectedToDataverse() != needConnection) {
-			if (needConnection)
-				throw new AlgebricksException(
-						"You need first to connect to a dataverse.");
-			else
-				throw new AlgebricksException(
-						"You need first to disconnect from the dataverse.");
-		}
-	}
+    private void checkForDataverseConnection(boolean needConnection) throws AlgebricksException {
+        if (compiledDeclarations.isConnectedToDataverse() != needConnection) {
+            if (needConnection)
+                throw new AlgebricksException("You need first to connect to a dataverse.");
+            else
+                throw new AlgebricksException("You need first to disconnect from the dataverse.");
+        }
+    }
 
-	private void runJob(IHyracksClientConnection hcc, JobSpecification jobSpec)
-			throws Exception {
-		System.out.println(jobSpec.toString());
-		executeJobArray(hcc, new JobSpecification[] { jobSpec }, out, pdf);
-	}
+    private void runJob(IHyracksClientConnection hcc, JobSpecification jobSpec) throws Exception {
+        System.out.println(jobSpec.toString());
+        executeJobArray(hcc, new JobSpecification[] { jobSpec }, out, pdf);
+    }
 
-	public void executeJobArray(IHyracksClientConnection hcc,
-			JobSpecification[] specs, PrintWriter out, DisplayFormat pdf)
-			throws Exception {
-		for (int i = 0; i < specs.length; i++) {
-			specs[i].setMaxReattempts(0);
-			JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
-			hcc.waitForCompletion(jobId);
-		}
-	}
+    public void executeJobArray(IHyracksClientConnection hcc, JobSpecification[] specs, PrintWriter out,
+            DisplayFormat pdf) throws Exception {
+        for (int i = 0; i < specs.length; i++) {
+            specs[i].setMaxReattempts(0);
+            JobId jobId = hcc.startJob(GlobalConfig.HYRACKS_APP_NAME, specs[i]);
+            hcc.waitForCompletion(jobId);
+        }
+    }
 
-	private void runCreateDatasetJob(IHyracksClientConnection hcc,
-			String datasetName) throws AsterixException, AlgebricksException,
-			Exception {
-		runJob(hcc, DatasetOperations.createDatasetJobSpec(datasetName,
-				compiledDeclarations));
-	}
+    private void runCreateDatasetJob(IHyracksClientConnection hcc, String datasetName) throws AsterixException,
+            AlgebricksException, Exception {
+        runJob(hcc, DatasetOperations.createDatasetJobSpec(datasetName, compiledDeclarations));
+    }
 
-	private void runCreateIndexJob(IHyracksClientConnection hcc,
-			CreateIndexStatement stmtCreateIndex) throws Exception {
-		// TODO: Eventually CreateIndexStatement and
-		// CompiledCreateIndexStatement should be replaced by the corresponding
-		// metadata entity.
-		// For now we must still convert to a CompiledCreateIndexStatement here.
-		CompiledCreateIndexStatement createIndexStmt = new CompiledCreateIndexStatement(
-				stmtCreateIndex.getIndexName().getValue(), stmtCreateIndex
-						.getDatasetName().getValue(),
-				stmtCreateIndex.getFieldExprs(),
-				stmtCreateIndex.getGramLength(), stmtCreateIndex.getIndexType());
-		JobSpecification spec = IndexOperations
-				.buildSecondaryIndexCreationJobSpec(createIndexStmt,
-						compiledDeclarations);
-		if (spec == null) {
-			throw new AsterixException(
-					"Failed to create job spec for creating index '"
-							+ stmtCreateIndex.getDatasetName() + "."
-							+ stmtCreateIndex.getIndexName() + "'");
-		}
-		runJob(hcc, spec);
-	}
+    private void runCreateIndexJob(IHyracksClientConnection hcc, CreateIndexStatement stmtCreateIndex) throws Exception {
+        // TODO: Eventually CreateIndexStatement and
+        // CompiledCreateIndexStatement should be replaced by the corresponding
+        // metadata entity.
+        // For now we must still convert to a CompiledCreateIndexStatement here.
+        CompiledCreateIndexStatement createIndexStmt = new CompiledCreateIndexStatement(stmtCreateIndex.getIndexName()
+                .getValue(), stmtCreateIndex.getDatasetName().getValue(), stmtCreateIndex.getFieldExprs(),
+                stmtCreateIndex.getGramLength(), stmtCreateIndex.getIndexType());
+        JobSpecification spec = IndexOperations.buildSecondaryIndexCreationJobSpec(createIndexStmt,
+                compiledDeclarations);
+        if (spec == null) {
+            throw new AsterixException("Failed to create job spec for creating index '"
+                    + stmtCreateIndex.getDatasetName() + "." + stmtCreateIndex.getIndexName() + "'");
+        }
+        runJob(hcc, spec);
+    }
 
-	private void compileDatasetDropStatement(IHyracksClientConnection hcc,
-			MetadataTransactionContext mdTxnCtx, String datasetName)
-			throws Exception {
-		CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(
-				datasetName);
-		Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
-				compiledDeclarations.getDataverseName(), datasetName);
-		if (ds.getDatasetType() == DatasetType.INTERNAL
-				|| ds.getDatasetType() == DatasetType.FEED) {
-			JobSpecification[] jobs = DatasetOperations
-					.createDropDatasetJobSpec(cds, compiledDeclarations);
-			for (JobSpecification job : jobs)
-				runJob(hcc, job);
-		}
-		MetadataManager.INSTANCE.dropDataset(mdTxnCtx,
-				compiledDeclarations.getDataverseName(), datasetName);
-	}
+    private void compileDatasetDropStatement(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
+            String datasetName) throws Exception {
+        CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(datasetName);
+        Dataset ds = MetadataManager.INSTANCE
+                .getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
+        if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            JobSpecification[] jobs = DatasetOperations.createDropDatasetJobSpec(cds, compiledDeclarations);
+            for (JobSpecification job : jobs)
+                runJob(hcc, job);
+        }
+        MetadataManager.INSTANCE.dropDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
+    }
 
-	public AqlCompiledMetadataDeclarations getCompiledDeclarations() {
-		return compiledDeclarations;
-	}
+    public AqlCompiledMetadataDeclarations getCompiledDeclarations() {
+        return compiledDeclarations;
+    }
 
-	private void compileIndexDropStatement(IHyracksClientConnection hcc,
-			MetadataTransactionContext mdTxnCtx, String datasetName,
-			String indexName) throws Exception {
-		CompiledIndexDropStatement cds = new CompiledIndexDropStatement(
-				datasetName, indexName);
-		runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds,
-				compiledDeclarations));
-		MetadataManager.INSTANCE
-				.dropIndex(mdTxnCtx, compiledDeclarations.getDataverseName(),
-						datasetName, indexName);
-	}
+    private void compileIndexDropStatement(IHyracksClientConnection hcc, MetadataTransactionContext mdTxnCtx,
+            String datasetName, String indexName) throws Exception {
+        CompiledIndexDropStatement cds = new CompiledIndexDropStatement(datasetName, indexName);
+        runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds, compiledDeclarations));
+        MetadataManager.INSTANCE.dropIndex(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName, indexName);
+    }
 
-	private Map<String, IAType> computeTypes(
-			MetadataTransactionContext mdTxnCtx, TypeDecl tDec)
-			throws AlgebricksException, MetadataException {
-		Map<String, IAType> typeMap = new HashMap<String, IAType>();
-		Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
-		Map<String, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<String, List<AbstractCollectionType>>();
-		Map<String, List<String>> incompleteTopLevelTypeReferences = new HashMap<String, List<String>>();
+    private Map<String, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeDecl tDec)
+            throws AlgebricksException, MetadataException {
+        Map<String, IAType> typeMap = new HashMap<String, IAType>();
+        Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
+        Map<String, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<String, List<AbstractCollectionType>>();
+        Map<String, List<String>> incompleteTopLevelTypeReferences = new HashMap<String, List<String>>();
 
-		firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes,
-				incompleteTopLevelTypeReferences);
-		secondPass(mdTxnCtx, typeMap, incompleteFieldTypes,
-				incompleteItemTypes, incompleteTopLevelTypeReferences);
+        firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
+        secondPass(mdTxnCtx, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
 
-		return typeMap;
-	}
+        return typeMap;
+    }
 
-	private void secondPass(MetadataTransactionContext mdTxnCtx,
-			Map<String, IAType> typeMap,
-			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-			Map<String, List<String>> incompleteTopLevelTypeReferences)
-			throws AlgebricksException, MetadataException {
-		// solve remaining top level references
-		for (String trefName : incompleteTopLevelTypeReferences.keySet()) {
-			IAType t;// = typeMap.get(trefName);
-			Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-					compiledDeclarations.getDataverseName(), trefName);
-			if (dt == null) {
-				throw new AlgebricksException("Could not resolve type "
-						+ trefName);
-			} else
-				t = dt.getDatatype();
-			for (String tname : incompleteTopLevelTypeReferences.get(trefName)) {
-				typeMap.put(tname, t);
-			}
-		}
-		// solve remaining field type references
-		for (String trefName : incompleteFieldTypes.keySet()) {
-			IAType t;// = typeMap.get(trefName);
-			Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-					compiledDeclarations.getDataverseName(), trefName);
-			if (dt == null) {
-				throw new AlgebricksException("Could not resolve type "
-						+ trefName);
-			} else
-				t = dt.getDatatype();
-			Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes
-					.get(trefName);
-			for (ARecordType recType : fieldsToFix.keySet()) {
-				List<Integer> positions = fieldsToFix.get(recType);
-				IAType[] fldTypes = recType.getFieldTypes();
-				for (Integer pos : positions) {
-					if (fldTypes[pos] == null) {
-						fldTypes[pos] = t;
-					} else { // nullable
-						AUnionType nullableUnion = (AUnionType) fldTypes[pos];
-						nullableUnion.setTypeAtIndex(t, 1);
-					}
-				}
-			}
-		}
-		// solve remaining item type references
-		for (String trefName : incompleteItemTypes.keySet()) {
-			IAType t;// = typeMap.get(trefName);
-			Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
-					compiledDeclarations.getDataverseName(), trefName);
-			if (dt == null) {
-				throw new AlgebricksException("Could not resolve type "
-						+ trefName);
-			} else
-				t = dt.getDatatype();
-			for (AbstractCollectionType act : incompleteItemTypes.get(trefName)) {
-				act.setItemType(t);
-			}
-		}
-	}
+    private void secondPass(MetadataTransactionContext mdTxnCtx, Map<String, IAType> typeMap,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException, MetadataException {
+        // solve remaining top level references
+        for (String trefName : incompleteTopLevelTypeReferences.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                    trefName);
+            if (dt == null) {
+                throw new AlgebricksException("Could not resolve type " + trefName);
+            } else
+                t = dt.getDatatype();
+            for (String tname : incompleteTopLevelTypeReferences.get(trefName)) {
+                typeMap.put(tname, t);
+            }
+        }
+        // solve remaining field type references
+        for (String trefName : incompleteFieldTypes.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                    trefName);
+            if (dt == null) {
+                throw new AlgebricksException("Could not resolve type " + trefName);
+            } else
+                t = dt.getDatatype();
+            Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes.get(trefName);
+            for (ARecordType recType : fieldsToFix.keySet()) {
+                List<Integer> positions = fieldsToFix.get(recType);
+                IAType[] fldTypes = recType.getFieldTypes();
+                for (Integer pos : positions) {
+                    if (fldTypes[pos] == null) {
+                        fldTypes[pos] = t;
+                    } else { // nullable
+                        AUnionType nullableUnion = (AUnionType) fldTypes[pos];
+                        nullableUnion.setTypeAtIndex(t, 1);
+                    }
+                }
+            }
+        }
+        // solve remaining item type references
+        for (String trefName : incompleteItemTypes.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, compiledDeclarations.getDataverseName(),
+                    trefName);
+            if (dt == null) {
+                throw new AlgebricksException("Could not resolve type " + trefName);
+            } else
+                t = dt.getDatatype();
+            for (AbstractCollectionType act : incompleteItemTypes.get(trefName)) {
+                act.setItemType(t);
+            }
+        }
+    }
 
-	private void firstPass(TypeDecl td, Map<String, IAType> typeMap,
-			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-			Map<String, List<String>> incompleteTopLevelTypeReferences)
-			throws AlgebricksException {
+    private void firstPass(TypeDecl td, Map<String, IAType> typeMap,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException {
 
-		TypeExpression texpr = td.getTypeDef();
-		String tdname = td.getIdent().getValue();
-		if (builtinTypeMap.get(tdname) != null) {
-			throw new AlgebricksException("Cannot redefine builtin type "
-					+ tdname + " .");
-		}
-		switch (texpr.getTypeKind()) {
-		case TYPEREFERENCE: {
-			TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
-			IAType t = solveTypeReference(tre, typeMap);
-			if (t != null) {
-				typeMap.put(tdname, t);
-			} else {
-				addIncompleteTopLevelTypeReference(tdname, tre,
-						incompleteTopLevelTypeReferences);
-			}
-			break;
-		}
-		case RECORD: {
-			RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
-			ARecordType recType = computeRecordType(tdname, rtd, typeMap,
-					incompleteFieldTypes, incompleteItemTypes);
-			typeMap.put(tdname, recType);
-			break;
-		}
-		case ORDEREDLIST: {
-			OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
-			AOrderedListType olType = computeOrderedListType(tdname, oltd,
-					typeMap, incompleteItemTypes, incompleteFieldTypes);
-			typeMap.put(tdname, olType);
-			break;
-		}
-		case UNORDEREDLIST: {
-			UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
-			AUnorderedListType ulType = computeUnorderedListType(tdname, ultd,
-					typeMap, incompleteItemTypes, incompleteFieldTypes);
-			typeMap.put(tdname, ulType);
-			break;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-	}
+        TypeExpression texpr = td.getTypeDef();
+        String tdname = td.getIdent().getValue();
+        if (builtinTypeMap.get(tdname) != null) {
+            throw new AlgebricksException("Cannot redefine builtin type " + tdname + " .");
+        }
+        switch (texpr.getTypeKind()) {
+            case TYPEREFERENCE: {
+                TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+                IAType t = solveTypeReference(tre, typeMap);
+                if (t != null) {
+                    typeMap.put(tdname, t);
+                } else {
+                    addIncompleteTopLevelTypeReference(tdname, tre, incompleteTopLevelTypeReferences);
+                }
+                break;
+            }
+            case RECORD: {
+                RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
+                ARecordType recType = computeRecordType(tdname, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes);
+                typeMap.put(tdname, recType);
+                break;
+            }
+            case ORDEREDLIST: {
+                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+                AOrderedListType olType = computeOrderedListType(tdname, oltd, typeMap, incompleteItemTypes,
+                        incompleteFieldTypes);
+                typeMap.put(tdname, olType);
+                break;
+            }
+            case UNORDEREDLIST: {
+                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+                AUnorderedListType ulType = computeUnorderedListType(tdname, ultd, typeMap, incompleteItemTypes,
+                        incompleteFieldTypes);
+                typeMap.put(tdname, ulType);
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
 
-	private AOrderedListType computeOrderedListType(String typeName,
-			OrderedListTypeDefinition oltd, Map<String, IAType> typeMap,
-			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-		TypeExpression tExpr = oltd.getItemTypeExpression();
-		AOrderedListType aolt = new AOrderedListType(null, typeName);
-		setCollectionItemType(tExpr, typeMap, incompleteItemTypes,
-				incompleteFieldTypes, aolt);
-		return aolt;
-	}
+    private AOrderedListType computeOrderedListType(String typeName, OrderedListTypeDefinition oltd,
+            Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+        TypeExpression tExpr = oltd.getItemTypeExpression();
+        AOrderedListType aolt = new AOrderedListType(null, typeName);
+        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, aolt);
+        return aolt;
+    }
 
-	private AUnorderedListType computeUnorderedListType(String typeName,
-			UnorderedListTypeDefinition ultd, Map<String, IAType> typeMap,
-			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-		TypeExpression tExpr = ultd.getItemTypeExpression();
-		AUnorderedListType ault = new AUnorderedListType(null, typeName);
-		setCollectionItemType(tExpr, typeMap, incompleteItemTypes,
-				incompleteFieldTypes, ault);
-		return ault;
-	}
+    private AUnorderedListType computeUnorderedListType(String typeName, UnorderedListTypeDefinition ultd,
+            Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+        TypeExpression tExpr = ultd.getItemTypeExpression();
+        AUnorderedListType ault = new AUnorderedListType(null, typeName);
+        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, ault);
+        return ault;
+    }
 
-	private void setCollectionItemType(TypeExpression tExpr,
-			Map<String, IAType> typeMap,
-			Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-			AbstractCollectionType act) {
-		switch (tExpr.getTypeKind()) {
-		case ORDEREDLIST: {
-			OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
-			IAType t = computeOrderedListType(null, oltd, typeMap,
-					incompleteItemTypes, incompleteFieldTypes);
-			act.setItemType(t);
-			break;
-		}
-		case UNORDEREDLIST: {
-			UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
-			IAType t = computeUnorderedListType(null, ultd, typeMap,
-					incompleteItemTypes, incompleteFieldTypes);
-			act.setItemType(t);
-			break;
-		}
-		case RECORD: {
-			RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
-			IAType t = computeRecordType(null, rtd, typeMap,
-					incompleteFieldTypes, incompleteItemTypes);
-			act.setItemType(t);
-			break;
-		}
-		case TYPEREFERENCE: {
-			TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
-			IAType tref = solveTypeReference(tre, typeMap);
-			if (tref != null) {
-				act.setItemType(tref);
-			} else {
-				addIncompleteCollectionTypeReference(act, tre,
-						incompleteItemTypes);
-			}
-			break;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-	}
+    private void setCollectionItemType(TypeExpression tExpr, Map<String, IAType> typeMap,
+            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, AbstractCollectionType act) {
+        switch (tExpr.getTypeKind()) {
+            case ORDEREDLIST: {
+                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
+                IAType t = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+                act.setItemType(t);
+                break;
+            }
+            case UNORDEREDLIST: {
+                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
+                IAType t = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+                act.setItemType(t);
+                break;
+            }
+            case RECORD: {
+                RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
+                IAType t = computeRecordType(null, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes);
+                act.setItemType(t);
+                break;
+            }
+            case TYPEREFERENCE: {
+                TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
+                IAType tref = solveTypeReference(tre, typeMap);
+                if (tref != null) {
+                    act.setItemType(tref);
+                } else {
+                    addIncompleteCollectionTypeReference(act, tre, incompleteItemTypes);
+                }
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
 
-	private ARecordType computeRecordType(String typeName,
-			RecordTypeDefinition rtd, Map<String, IAType> typeMap,
-			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-			Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
-		List<String> names = rtd.getFieldNames();
-		int n = names.size();
-		String[] fldNames = new String[n];
-		IAType[] fldTypes = new IAType[n];
-		int i = 0;
-		for (String s : names) {
-			fldNames[i++] = s;
-		}
-		boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
-		ARecordType recType = new ARecordType(typeName, fldNames, fldTypes,
-				isOpen);
-		for (int j = 0; j < n; j++) {
-			TypeExpression texpr = rtd.getFieldTypes().get(j);
-			switch (texpr.getTypeKind()) {
-			case TYPEREFERENCE: {
-				TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
-				IAType tref = solveTypeReference(tre, typeMap);
-				if (tref != null) {
-					if (!rtd.getNullableFields().get(j)) { // not nullable
-						fldTypes[j] = tref;
-					} else { // nullable
-						fldTypes[j] = makeUnionWithNull(null, tref);
-					}
-				} else {
-					addIncompleteFieldTypeReference(recType, j, tre,
-							incompleteFieldTypes);
-					if (rtd.getNullableFields().get(j)) {
-						fldTypes[j] = makeUnionWithNull(null, null);
-					}
-				}
-				break;
-			}
-			case RECORD: {
-				RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
-				IAType t2 = computeRecordType(null, recTypeDef2, typeMap,
-						incompleteFieldTypes, incompleteItemTypes);
-				if (!rtd.getNullableFields().get(j)) { // not nullable
-					fldTypes[j] = t2;
-				} else { // nullable
-					fldTypes[j] = makeUnionWithNull(null, t2);
-				}
-				break;
-			}
-			case ORDEREDLIST: {
-				OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
-				IAType t2 = computeOrderedListType(null, oltd, typeMap,
-						incompleteItemTypes, incompleteFieldTypes);
-				fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(
-						null, t2) : t2;
-				break;
-			}
-			case UNORDEREDLIST: {
-				UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
-				IAType t2 = computeUnorderedListType(null, ultd, typeMap,
-						incompleteItemTypes, incompleteFieldTypes);
-				fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(
-						null, t2) : t2;
-				break;
-			}
-			default: {
-				throw new IllegalStateException();
-			}
-			}
+    private ARecordType computeRecordType(String typeName, RecordTypeDefinition rtd, Map<String, IAType> typeMap,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
+        List<String> names = rtd.getFieldNames();
+        int n = names.size();
+        String[] fldNames = new String[n];
+        IAType[] fldTypes = new IAType[n];
+        int i = 0;
+        for (String s : names) {
+            fldNames[i++] = s;
+        }
+        boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
+        ARecordType recType = new ARecordType(typeName, fldNames, fldTypes, isOpen);
+        for (int j = 0; j < n; j++) {
+            TypeExpression texpr = rtd.getFieldTypes().get(j);
+            switch (texpr.getTypeKind()) {
+                case TYPEREFERENCE: {
+                    TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+                    IAType tref = solveTypeReference(tre, typeMap);
+                    if (tref != null) {
+                        if (!rtd.getNullableFields().get(j)) { // not nullable
+                            fldTypes[j] = tref;
+                        } else { // nullable
+                            fldTypes[j] = makeUnionWithNull(null, tref);
+                        }
+                    } else {
+                        addIncompleteFieldTypeReference(recType, j, tre, incompleteFieldTypes);
+                        if (rtd.getNullableFields().get(j)) {
+                            fldTypes[j] = makeUnionWithNull(null, null);
+                        }
+                    }
+                    break;
+                }
+                case RECORD: {
+                    RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
+                    IAType t2 = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes, incompleteItemTypes);
+                    if (!rtd.getNullableFields().get(j)) { // not nullable
+                        fldTypes[j] = t2;
+                    } else { // nullable
+                        fldTypes[j] = makeUnionWithNull(null, t2);
+                    }
+                    break;
+                }
+                case ORDEREDLIST: {
+                    OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+                    IAType t2 = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
+                    break;
+                }
+                case UNORDEREDLIST: {
+                    UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+                    IAType t2 = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
+                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
+                    break;
+                }
+                default: {
+                    throw new IllegalStateException();
+                }
+            }
 
-		}
+        }
 
-		return recType;
-	}
+        return recType;
+    }
 
-	private AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
-		ArrayList<IAType> unionList = new ArrayList<IAType>(2);
-		unionList.add(BuiltinType.ANULL);
-		unionList.add(type);
-		return new AUnionType(unionList, unionTypeName);
-	}
+    private AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
+        ArrayList<IAType> unionList = new ArrayList<IAType>(2);
+        unionList.add(BuiltinType.ANULL);
+        unionList.add(type);
+        return new AUnionType(unionList, unionTypeName);
+    }
 
-	private void addIncompleteCollectionTypeReference(
-			AbstractCollectionType collType, TypeReferenceExpression tre,
-			Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
-		String typeName = tre.getIdent().getValue();
-		List<AbstractCollectionType> typeList = incompleteItemTypes
-				.get(typeName);
-		if (typeList == null) {
-			typeList = new LinkedList<AbstractCollectionType>();
-			incompleteItemTypes.put(typeName, typeList);
-		}
-		typeList.add(collType);
-	}
+    private void addIncompleteCollectionTypeReference(AbstractCollectionType collType, TypeReferenceExpression tre,
+            Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
+        String typeName = tre.getIdent().getValue();
+        List<AbstractCollectionType> typeList = incompleteItemTypes.get(typeName);
+        if (typeList == null) {
+            typeList = new LinkedList<AbstractCollectionType>();
+            incompleteItemTypes.put(typeName, typeList);
+        }
+        typeList.add(collType);
+    }
 
-	private void addIncompleteFieldTypeReference(ARecordType recType,
-			int fldPosition, TypeReferenceExpression tre,
-			Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-		String typeName = tre.getIdent().getValue();
-		Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes
-				.get(typeName);
-		if (refMap == null) {
-			refMap = new HashMap<ARecordType, List<Integer>>();
-			incompleteFieldTypes.put(typeName, refMap);
-		}
-		List<Integer> typeList = refMap.get(recType);
-		if (typeList == null) {
-			typeList = new ArrayList<Integer>();
-			refMap.put(recType, typeList);
-		}
-		typeList.add(fldPosition);
-	}
+    private void addIncompleteFieldTypeReference(ARecordType recType, int fldPosition, TypeReferenceExpression tre,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+        String typeName = tre.getIdent().getValue();
+        Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes.get(typeName);
+        if (refMap == null) {
+            refMap = new HashMap<ARecordType, List<Integer>>();
+            incompleteFieldTypes.put(typeName, refMap);
+        }
+        List<Integer> typeList = refMap.get(recType);
+        if (typeList == null) {
+            typeList = new ArrayList<Integer>();
+            refMap.put(recType, typeList);
+        }
+        typeList.add(fldPosition);
+    }
 
-	private void addIncompleteTopLevelTypeReference(String tdeclName,
-			TypeReferenceExpression tre,
-			Map<String, List<String>> incompleteTopLevelTypeReferences) {
-		String name = tre.getIdent().getValue();
-		List<String> refList = incompleteTopLevelTypeReferences.get(name);
-		if (refList == null) {
-			refList = new LinkedList<String>();
-			incompleteTopLevelTypeReferences.put(name, refList);
-		}
-		refList.add(tdeclName);
-	}
+    private void addIncompleteTopLevelTypeReference(String tdeclName, TypeReferenceExpression tre,
+            Map<String, List<String>> incompleteTopLevelTypeReferences) {
+        String name = tre.getIdent().getValue();
+        List<String> refList = incompleteTopLevelTypeReferences.get(name);
+        if (refList == null) {
+            refList = new LinkedList<String>();
+            incompleteTopLevelTypeReferences.put(name, refList);
+        }
+        refList.add(tdeclName);
+    }
 
-	private IAType solveTypeReference(TypeReferenceExpression tre,
-			Map<String, IAType> typeMap) {
-		String name = tre.getIdent().getValue();
-		IAType builtin = builtinTypeMap.get(name);
-		if (builtin != null) {
-			return builtin;
-		} else {
-			return typeMap.get(name);
-		}
-	}
+    private IAType solveTypeReference(TypeReferenceExpression tre, Map<String, IAType> typeMap) {
+        String name = tre.getIdent().getValue();
+        IAType builtin = builtinTypeMap.get(name);
+        if (builtin != null) {
+            return builtin;
+        } else {
+            return typeMap.get(name);
+        }
+    }
 
-	public static interface ICompiledStatement {
+    public static interface ICompiledStatement {
 
-		public abstract Kind getKind();
-	}
+        public abstract Kind getKind();
+    }
 
-	public static class CompiledLoadFromFileStatement implements
-			ICompiledStatement, IParseFileSplitsDecl {
-		private String datasetName;
-		private FileSplit[] splits;
-		private boolean alreadySorted;
-		private Character delimChar;
+    public static class CompiledLoadFromFileStatement implements ICompiledStatement, IParseFileSplitsDecl {
+        private String datasetName;
+        private FileSplit[] splits;
+        private boolean alreadySorted;
+        private Character delimChar;
 
-		public CompiledLoadFromFileStatement(String datasetName,
-				FileSplit[] splits, Character delimChar, boolean alreadySorted) {
-			this.datasetName = datasetName;
-			this.splits = splits;
-			this.delimChar = delimChar;
-			this.alreadySorted = alreadySorted;
-		}
+        public CompiledLoadFromFileStatement(String datasetName, FileSplit[] splits, Character delimChar,
+                boolean alreadySorted) {
+            this.datasetName = datasetName;
+            this.splits = splits;
+            this.delimChar = delimChar;
+            this.alreadySorted = alreadySorted;
+        }
 
-		public String getDatasetName() {
-			return datasetName;
-		}
+        public String getDatasetName() {
+            return datasetName;
+        }
 
-		@Override
-		public FileSplit[] getSplits() {
-			return splits;
-		}
+        @Override
+        public FileSplit[] getSplits() {
+            return splits;
+        }
 
-		@Override
-		public Character getDelimChar() {
-			return delimChar;
-		}
+        @Override
+        public Character getDelimChar() {
+            return delimChar;
+        }
 
-		public boolean alreadySorted() {
-			return alreadySorted;
-		}
+        public boolean alreadySorted() {
+            return alreadySorted;
+        }
 
-		@Override
-		public boolean isDelimitedFileFormat() {
-			return delimChar != null;
-		}
+        @Override
+        public boolean isDelimitedFileFormat() {
+            return delimChar != null;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.LOAD_FROM_FILE;
-		}
-	}
+        @Override
+        public Kind getKind() {
+            return Kind.LOAD_FROM_FILE;
+        }
+    }
 
-	public static class CompiledWriteFromQueryResultStatement implements
-			ICompiledStatement {
+    public static class CompiledWriteFromQueryResultStatement implements ICompiledStatement {
 
-		private String datasetName;
-		private Query query;
-		private int varCounter;
+        private String datasetName;
+        private Query query;
+        private int varCounter;
 
-		public CompiledWriteFromQueryResultStatement(String datasetName,
-				Query query, int varCounter) {
-			this.datasetName = datasetName;
-			this.query = query;
-			this.varCounter = varCounter;
-		}
+        public CompiledWriteFromQueryResultStatement(String datasetName, Query query, int varCounter) {
+            this.datasetName = datasetName;
+            this.query = query;
+            this.varCounter = varCounter;
+        }
 
-		public String getDatasetName() {
-			return datasetName;
-		}
+        public String getDatasetName() {
+            return datasetName;
+        }
 
-		public int getVarCounter() {
-			return varCounter;
-		}
+        public int getVarCounter() {
+            return varCounter;
+        }
 
-		public Query getQuery() {
-			return query;
-		}
+        public Query getQuery() {
+            return query;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.WRITE_FROM_QUERY_RESULT;
-		}
+        @Override
+        public Kind getKind() {
+            return Kind.WRITE_FROM_QUERY_RESULT;
+        }
 
-	}
+    }
 
-	public static class CompiledDatasetDropStatement implements
-			ICompiledStatement {
-		private String datasetName;
+    public static class CompiledDatasetDropStatement implements ICompiledStatement {
+        private String datasetName;
 
-		public CompiledDatasetDropStatement(String datasetName) {
-			this.datasetName = datasetName;
-		}
+        public CompiledDatasetDropStatement(String datasetName) {
+            this.datasetName = datasetName;
+        }
 
-		public String getDatasetName() {
-			return datasetName;
-		}
+        public String getDatasetName() {
+            return datasetName;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.DATASET_DROP;
-		}
-	}
+        @Override
+        public Kind getKind() {
+            return Kind.DATASET_DROP;
+        }
+    }
 
-	// added by yasser
-	public static class CompiledCreateDataverseStatement implements
-			ICompiledStatement {
-		private String dataverseName;
-		private String format;
+    // added by yasser
+    public static class CompiledCreateDataverseStatement implements ICompiledStatement {
+        private String dataverseName;
+        private String format;
 
-		public CompiledCreateDataverseStatement(String dataverseName,
-				String format) {
-			this.dataverseName = dataverseName;
-			this.format = format;
-		}
+        public CompiledCreateDataverseStatement(String dataverseName, String format) {
+            this.dataverseName = dataverseName;
+            this.format = format;
+        }
 
-		public String getDataverseName() {
-			return dataverseName;
-		}
+        public String getDataverseName() {
+            return dataverseName;
+        }
 
-		public String getFormat() {
-			return format;
-		}
+        public String getFormat() {
+            return format;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.CREATE_DATAVERSE;
-		}
-	}
+        @Override
+        public Kind getKind() {
+            return Kind.CREATE_DATAVERSE;
+        }
+    }
 
-	public static class CompiledNodeGroupDropStatement implements
-			ICompiledStatement {
-		private String nodeGroupName;
+    public static class CompiledNodeGroupDropStatement implements ICompiledStatement {
+        private String nodeGroupName;
 
-		public CompiledNodeGroupDropStatement(String nodeGroupName) {
-			this.nodeGroupName = nodeGroupName;
-		}
+        public CompiledNodeGroupDropStatement(String nodeGroupName) {
+            this.nodeGroupName = nodeGroupName;
+        }
 
-		public String getNodeGroupName() {
-			return nodeGroupName;
-		}
+        public String getNodeGroupName() {
+            return nodeGroupName;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.NODEGROUP_DROP;
-		}
-	}
+        @Override
+        public Kind getKind() {
+            return Kind.NODEGROUP_DROP;
+        }
+    }
 
-	public static class CompiledIndexDropStatement implements
-			ICompiledStatement {
-		private String datasetName;
-		private String indexName;
+    public static class CompiledIndexDropStatement implements ICompiledStatement {
+        private String datasetName;
+        private String indexName;
 
-		public CompiledIndexDropStatement(String datasetName, String indexName) {
-			this.datasetName = datasetName;
-			this.indexName = indexName;
-		}
+        public CompiledIndexDropStatement(String datasetName, String indexName) {
+            this.datasetName = datasetName;
+            this.indexName = indexName;
+        }
 
-		public String getDatasetName() {
-			return datasetName;
-		}
+        public String getDatasetName() {
+            return datasetName;
+        }
 
-		public String getIndexName() {
-			return indexName;
-		}
+        public String getIndexName() {
+            return indexName;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.INDEX_DROP;
-		}
-	}
+        @Override
+        public Kind getKind() {
+            return Kind.INDEX_DROP;
+        }
+    }
 
-	public static class CompiledDataverseDropStatement implements
-			ICompiledStatement {
-		private String dataverseName;
-		private boolean ifExists;
+    public static class CompiledDataverseDropStatement implements ICompiledStatement {
+        private String dataverseName;
+        private boolean ifExists;
 
-		public CompiledDataverseDropStatement(String dataverseName,
-				boolean ifExists) {
-			this.dataverseName = dataverseName;
-			this.ifExists = ifExists;
-		}
+        public CompiledDataverseDropStatement(String dataverseName, boolean ifExists) {
+            this.dataverseName = dataverseName;
+            this.ifExists = ifExists;
+        }
 
-		public String getDataverseName() {
-			return dataverseName;
-		}
+        public String getDataverseName() {
+            return dataverseName;
+        }
 
-		public boolean getIfExists() {
-			return ifExists;
-		}
+        public boolean getIfExists() {
+            return ifExists;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.DATAVERSE_DROP;
-		}
-	}
+        @Override
+        public Kind getKind() {
+            return Kind.DATAVERSE_DROP;
+        }
+    }
 
-	public static class CompiledTypeDropStatement implements ICompiledStatement {
-		private String typeName;
+    public static class CompiledTypeDropStatement implements ICompiledStatement {
+        private String typeName;
 
-		public CompiledTypeDropStatement(String nodeGroupName) {
-			this.typeName = nodeGroupName;
-		}
+        public CompiledTypeDropStatement(String nodeGroupName) {
+            this.typeName = nodeGroupName;
+        }
 
-		public String getTypeName() {
-			return typeName;
-		}
+        public String getTypeName() {
+            return typeName;
+        }
 
-		@Override
-		public Kind getKind() {
-			return Kind.TYPE_DROP;
-		}
-	}
+        @Override
+        public Kind getKind() {
+            return Kind.TYPE_DROP;
+        }
+    }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index b028e06..b12e6d8 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -23,8 +23,7 @@
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
@@ -35,7 +34,9 @@
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.asterix.translator.DmlTranslator.CompiledLoadFromFileStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -67,14 +68,17 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDropOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 
 public class DatasetOperations {
 
@@ -92,9 +96,6 @@
 
         LOGGER.info("DROP DATASETPATH: " + datasetPath);
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
         Dataset dataset = metadata.findDataset(datasetName);
         if (dataset == null) {
             throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
@@ -121,8 +122,36 @@
                     Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
                             .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
                                     index.getIndexName());
-                    TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
-                            storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
+                    IIndexDataflowHelperFactory dfhFactory;
+                    switch (index.getIndexType()) {
+                        case BTREE:
+                            dfhFactory = new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE);
+                            break;
+                        case RTREE:
+                            dfhFactory = new LSMRTreeDataflowHelperFactory(
+                                    new IPrimitiveValueProviderFactory[] { null }, RTreePolicyType.RTREE,
+                                    new IBinaryComparatorFactory[] { null }, AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE, null);
+                            break;
+                        case NGRAM_INVIX:
+                        case WORD_INVIX:
+                            dfhFactory = new LSMInvertedIndexDataflowHelperFactory(
+                                    AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE,
+                                    AsterixRuntimeComponentsProvider.INSTANCE);
+                            break;
+                        default:
+                            throw new AsterixException("Unknown index type provided.");
+                    }
+                    IndexDropOperatorDescriptor secondaryBtreeDrop = new IndexDropOperatorDescriptor(specs[i],
+                            AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                            idxSplitsAndConstraint.first, dfhFactory);
                     AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
                             idxSplitsAndConstraint.second);
                     i++;
@@ -136,8 +165,11 @@
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
-        TreeIndexDropOperatorDescriptor primaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specPrimary,
-                storageManager, indexRegistryProvider, splitsAndConstraint.first);
+        IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
 
@@ -165,11 +197,14 @@
             sb.append(stringOf(fs[i]) + " ");
         }
         LOGGER.info("CREATING File Splits: " + sb.toString());
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-        TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, splitsAndConstraint.first, typeTraits, comparatorFactories,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+        //TODO replace this transient one to the persistent one 
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
+                localResourceFactoryProvider);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
         spec.addRoot(indexCreateOp);
@@ -190,6 +225,9 @@
         }
         JobSpecification spec = new JobSpecification();
 
+        edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory
+                .generateJobId();
+
         ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         IDataFormat format = metadata.getFormat();
         ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
@@ -238,11 +276,12 @@
         }
         LOGGER.info("LOAD into File Splits: " + sb.toString());
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, splitsAndConstraint.first, typeTraits, comparatorFactories,
-                fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
+                GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
                 NoOpOperationCallbackProvider.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                 splitsAndConstraint.second);
@@ -266,6 +305,7 @@
         }
         spec.addRoot(btreeBulkLoad);
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        spec.setJobletEventListenerFactory(new JobEventListenerFactory(asterixJobId, false));
 
         return new Job(spec);
     }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index 7bd7ae6..8b2d583 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -2,8 +2,7 @@
 
 import edu.uci.ics.asterix.aql.translator.DdlTranslator.CompiledIndexDropStatement;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
@@ -15,10 +14,8 @@
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDropOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexDropOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 
 public class IndexOperations {
 
@@ -45,13 +42,14 @@
         String indexName = deleteStmt.getIndexName();
 
         JobSpecification spec = new JobSpecification();
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = datasetDecls
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
-        TreeIndexDropOperatorDescriptor btreeDrop = new TreeIndexDropOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, splitsAndConstraint.first);
+        IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
         spec.addRoot(btreeDrop);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index b9ad908..ce26312 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.asterix.file;
 
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -12,12 +11,12 @@
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 
 public class SecondaryBTreeCreator extends SecondaryIndexCreator {
 
@@ -28,10 +27,14 @@
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
+        //TODO replace this transient one to the persistent one 
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), localResourceFactoryProvider);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -63,7 +66,9 @@
 
         // Create secondary BTree bulk load op.
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys,
-                new BTreeDataflowHelperFactory(), BTree.DEFAULT_FILL_FACTOR);
+                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index 65f6f5e..6cda5ab 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -4,8 +4,7 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -50,11 +49,11 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 
 @SuppressWarnings("rawtypes")
 // TODO: We should eventually have a hierarchy of classes that can create all possible index job specs, 
@@ -225,10 +224,11 @@
         // +Infinity
         int[] highKeyFields = null;
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, new BTreeDataflowHelperFactory(), false,
-                NoOpOperationCallbackProvider.INSTANCE);
+                highKeyFields, true, true, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), false, NoOpOperationCallbackProvider.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
@@ -283,9 +283,9 @@
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, secondaryIndexName);
         TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
                 secondarySplitsAndConstraint.first, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                fieldPermutation, fillFactor, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                fieldPermutation, fillFactor, false, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondarySplitsAndConstraint.second);
         return treeIndexBulkLoadOp;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index d5e8222..1deb278 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -2,8 +2,7 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -27,13 +26,15 @@
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexBulkLoadOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.dataflow.InvertedIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.BinaryTokenizerOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexCreateOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 
 public class SecondaryInvertedIndexCreator extends SecondaryIndexCreator {
 
@@ -63,7 +64,8 @@
         if (numSecondaryKeys > 1) {
             throw new AsterixException("Cannot create composite inverted index on multiple fields.");
         }
-        // Prepare record descriptor used in the assign op, and the optional select op.
+        // Prepare record descriptor used in the assign op, and the optional
+        // select op.
         List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
         secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
         ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
@@ -87,11 +89,12 @@
         tokenComparatorFactories[0] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
         tokenTypeTraits[0] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
         // Set tokenizer factory.
-        // TODO: We might want to expose the hashing option at the AQL level, 
+        // TODO: We might want to expose the hashing option at the AQL level,
         // and add the choice to the index metadata.
         tokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(secondaryKeyType.getTypeTag(),
                 createIndexStmt.getIndexType(), createIndexStmt.getGramLength());
-        // Type traits for inverted-list elements. Inverted lists contain primary keys.
+        // Type traits for inverted-list elements. Inverted lists contain
+        // primary keys.
         invListsTypeTraits = new ITypeTraits[numPrimaryKeys];
         for (int i = 0; i < numPrimaryKeys; i++) {
             invListsTypeTraits[i] = primaryRecDesc.getTypeTraits()[i];
@@ -119,11 +122,16 @@
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
-        InvertedIndexCreateOperatorDescriptor invIndexCreateOp = new InvertedIndexCreateOperatorDescriptor(spec,
-                AsterixStorageManagerInterface.INSTANCE, fileSplitProviders.first, fileSplitProviders.second,
-                AsterixIndexRegistryProvider.INSTANCE, tokenTypeTraits, tokenComparatorFactories, invListsTypeTraits,
-                primaryComparatorFactories, tokenizerFactory, new BTreeDataflowHelperFactory(),
-                NoOpOperationCallbackProvider.INSTANCE);
+        //TODO replace the transient one to persistent one
+        ILocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+        LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
+                AsterixRuntimeComponentsProvider.INSTANCE, fileSplitProviders.first,
+                AsterixRuntimeComponentsProvider.INSTANCE, tokenTypeTraits, tokenComparatorFactories,
+                invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
+                new LSMInvertedIndexDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), NoOpOperationCallbackProvider.INSTANCE,
+                localResourceFactoryProvider);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(invIndexCreateOp);
@@ -135,7 +143,7 @@
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
 
-        // Create dummy key provider for feeding the primary index scan. 
+        // Create dummy key provider for feeding the primary index scan.
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
 
         // Create primary index scan op.
@@ -144,7 +152,8 @@
         // Assign op.
         AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
 
-        // If any of the secondary fields are nullable, then add a select op that filters nulls.
+        // If any of the secondary fields are nullable, then add a select op
+        // that filters nulls.
         AlgebricksMetaOperatorDescriptor selectOp = null;
         if (anySecondaryKeyIsNullable) {
             selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
@@ -157,7 +166,7 @@
         ExternalSortOperatorDescriptor sortOp = createSortOp(spec, tokenKeyPairComparatorFactories, tokenKeyPairRecDesc);
 
         // Create secondary inverted index bulk load op.
-        InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec);
+        LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = createInvertedIndexBulkLoadOp(spec);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
@@ -205,16 +214,18 @@
         return sortOp;
     }
 
-    private InvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
+    private LSMInvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
         int[] fieldPermutation = new int[numSecondaryKeys + numPrimaryKeys];
         for (int i = 0; i < numTokenKeyPairFields; i++) {
             fieldPermutation[i] = i;
         }
-        InvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new InvertedIndexBulkLoadOperatorDescriptor(spec,
-                fieldPermutation, AsterixStorageManagerInterface.INSTANCE, fileSplitProviders.first,
-                fileSplitProviders.second, AsterixIndexRegistryProvider.INSTANCE, tokenTypeTraits,
-                tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+        LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
+                spec, fieldPermutation, false, AsterixRuntimeComponentsProvider.INSTANCE, fileSplitProviders.first,
+                AsterixRuntimeComponentsProvider.INSTANCE, tokenTypeTraits, tokenComparatorFactories,
+                invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
+                new LSMInvertedIndexDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), NoOpOperationCallbackProvider.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return invIndexBulkLoadOp;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index 3127573..a3069b2 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -2,14 +2,15 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
@@ -31,14 +32,16 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexCreateOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 
 @SuppressWarnings("rawtypes")
 public class SecondaryRTreeCreator extends SecondaryIndexCreator {
 
     protected IPrimitiveValueProviderFactory[] valueProviderFactories;
     protected int numNestedSecondaryKeyFields;
+    protected ATypeTag keyType;
 
     protected SecondaryRTreeCreator(PhysicalOptimizationConfig physOptConf) {
         super(physOptConf);
@@ -47,10 +50,16 @@
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
+        //TODO replace this transient one to the persistent one 
+        TransientLocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                new RTreeDataflowHelperFactory(valueProviderFactories), NoOpOperationCallbackProvider.INSTANCE);
+                new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+                                secondaryComparatorFactories.length)), localResourceFactoryProvider);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -85,6 +94,7 @@
                 + numNestedSecondaryKeyFields];
         ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numNestedSecondaryKeyFields + numPrimaryKeys];
         IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
+        keyType = nestedKeyType.getTypeTag();
         for (int i = 0; i < numNestedSecondaryKeyFields; i++) {
             ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
                     .getSerializerDeserializer(nestedKeyType);
@@ -123,9 +133,14 @@
         }
 
         // Create secondary RTree bulk load op.
-        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec,
-                numNestedSecondaryKeyFields, new RTreeDataflowHelperFactory(valueProviderFactories),
-                BTree.DEFAULT_FILL_FACTOR);
+        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+                spec,
+                numNestedSecondaryKeyFields,
+                new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+                                secondaryComparatorFactories.length)), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
index c295f55..4811bb7 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
@@ -6,8 +6,7 @@
 import java.util.HashMap;
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
@@ -38,12 +37,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 
 public class TestKeywordIndexJob {
 
@@ -66,9 +62,6 @@
 
         // ---------- START GENERAL BTREE STUFF
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
         // ---------- END GENERAL BTREE STUFF
 
         List<String> nodeGroup = new ArrayList<String>();
@@ -144,9 +137,11 @@
                 new FileSplit("nc1", new FileReference(new File("/tmp/nc1/demo1112/Customers_idx_NameInvIndex"))),
                 new FileSplit("nc2", new FileReference(new File("/tmp/nc2/demo1112/Customers_idx_NameInvIndex"))) });
         BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
-                storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, lowKeyFields, highKeyFields, true, true,
-                new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories, lowKeyFields, highKeyFields,
+                true, true, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), false, NoOpOperationCallbackProvider.INSTANCE);
         String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
         for (int p = 0; p < nodeGroup.size(); p++) {
             secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
index 5d2cdc5..c04e8dd 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
@@ -6,8 +6,7 @@
 import java.util.HashMap;
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
@@ -38,12 +37,9 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.misc.PrinterOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 
 public class TestSecondaryIndexJob {
 
@@ -66,9 +62,6 @@
 
         // ---------- START GENERAL BTREE STUFF
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
         // ---------- END GENERAL BTREE STUFF
 
         List<String> nodeGroup = new ArrayList<String>();
@@ -144,9 +137,11 @@
                 new FileSplit("nc1", new FileReference(new File("/tmp/nc1/demo1112/Customers_idx_NameBtreeIndex"))),
                 new FileSplit("nc2", new FileReference(new File("/tmp/nc2/demo1112/Customers_idx_NameBtreeIndex"))) });
         BTreeSearchOperatorDescriptor secondarySearchOp = new BTreeSearchOperatorDescriptor(spec, secondaryRecDesc,
-                storageManager, indexRegistryProvider, secondarySplitProvider, secondaryTypeTraits,
-                secondaryComparatorFactories, lowKeyFields, highKeyFields, true, true,
-                new BTreeDataflowHelperFactory(), false, NoOpOperationCallbackProvider.INSTANCE);
+                AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                secondarySplitProvider, secondaryTypeTraits, secondaryComparatorFactories, lowKeyFields, highKeyFields,
+                true, true, new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), false, NoOpOperationCallbackProvider.INSTANCE);
         String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
         for (int p = 0; p < nodeGroup.size(); p++) {
             secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
diff --git a/asterix-app/src/test/resources/runtimets/ignore.txt b/asterix-app/src/test/resources/runtimets/ignore.txt
index 2b011ec..ec49149 100644
--- a/asterix-app/src/test/resources/runtimets/ignore.txt
+++ b/asterix-app/src/test/resources/runtimets/ignore.txt
@@ -33,3 +33,7 @@
 flwor
 string/startwith03.aql
 aggregate/droptype.aql
+failure/insert-rtree.aql
+failure/insert.aql
+failure/delete-rtree.aql
+failure/delete.aql
\ No newline at end of file
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 048c037..67377d5 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -40,6 +40,11 @@
 			<type>jar</type>
 			<scope>compile</scope>
 		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-common</artifactId>
+			<version>0.2.1-SNAPSHOT</version>
+		</dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
index 144a8824..dca790d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfoImpl.java
@@ -3,11 +3,9 @@
 import java.util.Map;
 import java.util.Set;
 
-import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
-import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 public class AsterixAppContextInfoImpl implements IAsterixApplicationContextInfo {
@@ -20,13 +18,8 @@
     }
 
     @Override
-    public IIndexRegistryProvider<IIndex> getIndexRegistryProvider() {
-        return AsterixIndexRegistryProvider.INSTANCE;
-    }
-
-    @Override
     public IStorageManagerInterface getStorageManagerInterface() {
-        return AsterixStorageManagerInterface.INSTANCE;
+        return AsterixRuntimeComponentsProvider.INSTANCE;
     }
 
     public static void setNodeControllerInfo(Map<String, Set<String>> nodeControllerInfo) {
@@ -37,4 +30,9 @@
         return nodeControllerMap;
     }
 
+    @Override
+    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
+        return AsterixRuntimeComponentsProvider.INSTANCE;
+    }
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
index d676cb5..52a67d3 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContext.java
@@ -8,8 +8,16 @@
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.io.IIOManager;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMFlushController;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.FlushController;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ImmediateScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ReferenceCountingOperationTracker;
 import edu.uci.ics.hyracks.storage.common.buffercache.BufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ClockPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
@@ -18,16 +26,29 @@
 import edu.uci.ics.hyracks.storage.common.buffercache.IPageReplacementStrategy;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapManager;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
+import edu.uci.ics.hyracks.storage.common.file.PersistentLocalResourceRepositoryFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
 
 public class AsterixAppRuntimeContext {
     private static final int DEFAULT_BUFFER_CACHE_PAGE_SIZE = 32768;
+    private static final int DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET = 1024 * 1024 * 1024; // 1GB
     private final INCApplicationContext ncApplicationContext;
-    
-    private IndexRegistry<IIndex> indexRegistry;
+
+    private IIndexLifecycleManager indexLifecycleManager;
     private IFileMapManager fileMapManager;
     private IBufferCache bufferCache;
     private TransactionProvider provider;
 
+    private ILSMFlushController flushController;
+    private ILSMMergePolicy mergePolicy;
+    private ILSMOperationTracker opTracker;
+    private ILSMIOOperationScheduler lsmIOScheduler;
+    private ILocalResourceRepository localResourceRepository;
+    private ResourceIdFactory resourceIdFactory;
+
     public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) {
         this.ncApplicationContext = ncApplicationContext;
     }
@@ -36,20 +57,23 @@
         int pageSize = getBufferCachePageSize();
         int numPages = getBufferCacheNumPages();
 
-        // Initialize file map manager
         fileMapManager = new AsterixFileMapManager();
-
-        // Initialize the buffer cache
         ICacheMemoryAllocator allocator = new HeapBufferAllocator();
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
         IIOManager ioMgr = ncApplicationContext.getRootContext().getIOManager();
         bufferCache = new BufferCache(ioMgr, allocator, prs, fileMapManager, pageSize, numPages, Integer.MAX_VALUE);
-
-        // Initialize the index registry
-        indexRegistry = new IndexRegistry<IIndex>();
-
-        // Initialize the transaction sub-system
+        indexLifecycleManager = new IndexLifecycleManager(DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET);
         provider = new TransactionProvider(ncApplicationContext.getNodeId());
+
+        flushController = new FlushController();
+        lsmIOScheduler = ImmediateScheduler.INSTANCE;
+        mergePolicy = new ConstantMergePolicy(lsmIOScheduler, 3);
+        opTracker = new ReferenceCountingOperationTracker();
+
+        ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
+                ioMgr);
+        localResourceRepository = persistentLocalResourceRepositoryFactory.createRepository();
+        resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
     }
 
     private int getBufferCachePageSize() {
@@ -107,12 +131,35 @@
         return fileMapManager;
     }
 
-    public IndexRegistry<IIndex> getIndexRegistry() {
-        return indexRegistry;
-    }
-
     public TransactionProvider getTransactionProvider() {
         return provider;
     }
 
+    public IIndexLifecycleManager getIndexLifecycleManager() {
+        return indexLifecycleManager;
+    }
+
+    public ILSMFlushController getFlushController() {
+        return flushController;
+    }
+
+    public ILSMMergePolicy getLSMMergePolicy() {
+        return mergePolicy;
+    }
+
+    public ILSMOperationTracker getLSMOperationTracker() {
+        return opTracker;
+    }
+
+    public ILSMIOOperationScheduler getLSMIOScheduler() {
+        return lsmIOScheduler;
+    }
+
+    public ILocalResourceRepository getLocalResourceRepository() {
+        return localResourceRepository;
+    }
+
+    public ResourceIdFactory getResourceIdFactory() {
+        return resourceIdFactory;
+    }
 }
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixIndexRegistryProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixIndexRegistryProvider.java
deleted file mode 100644
index bc59291..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixIndexRegistryProvider.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package edu.uci.ics.asterix.common.context;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-
-public class AsterixIndexRegistryProvider implements IIndexRegistryProvider<IIndex> {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final AsterixIndexRegistryProvider INSTANCE = new AsterixIndexRegistryProvider();
-
-    private AsterixIndexRegistryProvider() {
-    }
-
-    @Override
-    public IndexRegistry<IIndex> getRegistry(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getIndexRegistry();
-    }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
new file mode 100644
index 0000000..49f59a0
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
@@ -0,0 +1,78 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMFlushController;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMFlushControllerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
+import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+
+public enum AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
+        ILSMIOOperationSchedulerProvider, ILSMFlushControllerProvider, ILSMOperationTrackerProvider,
+        ILSMMergePolicyProvider {
+    INSTANCE;
+
+    @Override
+    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getLSMOperationTracker();
+    }
+
+    @Override
+    public ILSMFlushController getFlushController(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getFlushController();
+    }
+
+    @Override
+    public ILSMIOOperationScheduler getIOScheduler(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getLSMIOScheduler();
+    }
+
+    @Override
+    public ILSMMergePolicy getMergePolicy(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getLSMMergePolicy();
+    }
+
+    @Override
+    public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getBufferCache();
+    }
+
+    @Override
+    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getFileMapManager();
+    }
+
+    @Override
+    public ILocalResourceRepository getLocalResourceRepository(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getLocalResourceRepository();
+    }
+
+    @Override
+    public IIndexLifecycleManager getLifecycleManager(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getIndexLifecycleManager();
+    }
+
+    @Override
+    public ResourceIdFactory getResourceIdFactory(IHyracksTaskContext ctx) {
+        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
+                .getResourceIdFactory();
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixStorageManagerInterface.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixStorageManagerInterface.java
deleted file mode 100644
index a1b5a94..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixStorageManagerInterface.java
+++ /dev/null
@@ -1,24 +0,0 @@
-package edu.uci.ics.asterix.common.context;
-
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
-
-public class AsterixStorageManagerInterface implements IStorageManagerInterface {
-    private static final long serialVersionUID = 1L;
-
-    public static AsterixStorageManagerInterface INSTANCE = new AsterixStorageManagerInterface();
-
-    @Override
-    public IBufferCache getBufferCache(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getBufferCache();
-    }
-
-    @Override
-    public IFileMapProvider getFileMapProvider(IHyracksTaskContext ctx) {
-        return ((AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getFileMapManager();
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
index 7bb0fd6..200c2e19 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/IAsterixApplicationContextInfo.java
@@ -1,11 +1,10 @@
 package edu.uci.ics.asterix.common.dataflow;
 
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 
 public interface IAsterixApplicationContextInfo {
-    public IIndexRegistryProvider<IIndex> getIndexRegistryProvider();
+    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider();
 
     public IStorageManagerInterface getStorageManagerInterface();
 }
diff --git a/asterix-hyracks-glue/pom.xml b/asterix-hyracks-glue/pom.xml
deleted file mode 100644
index 5f991a4..0000000
--- a/asterix-hyracks-glue/pom.xml
+++ /dev/null
@@ -1,51 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>asterix</artifactId>
-    <groupId>edu.uci.ics.asterix</groupId>
-    <version>0.0.4-SNAPSHOT</version>
-  </parent>
-  <groupId>edu.uci.ics.asterix</groupId>
-  <artifactId>asterix-hyracks-glue</artifactId>
-  <version>0.0.4-SNAPSHOT</version>
-  <build>
-       <plugins>
-           <plugin>
-              <groupId>org.apache.maven.plugins</groupId>
-              <artifactId>maven-compiler-plugin</artifactId>
-              <version>2.0.2</version>
-              <configuration>
-                 <source>1.6</source>
-                 <target>1.6</target>
-              </configuration>
-           </plugin>
-       </plugins>
-  </build>     
-  <name>asterix-hyracks-glue</name>
-  <url>http://maven.apache.org</url>
-  <dependencies>
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>3.8.1</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-       <groupId>edu.uci.ics.asterix</groupId>
-       <artifactId>asterix-transactions</artifactId>
-       <version>0.0.4-SNAPSHOT</version>
-       <scope>compile</scope>
-    </dependency>
-    <dependency>
-    	<groupId>edu.uci.ics.asterix</groupId>
-    	<artifactId>asterix-common</artifactId>
-    	<version>0.0.4-SNAPSHOT</version>
-    	<type>jar</type>
-    	<scope>compile</scope>
-    </dependency>
-  </dependencies>
-  <properties>
-    <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-  </properties>
-</project>
diff --git a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
deleted file mode 100644
index f96d3da..0000000
--- a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorDescriptor.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.runtime.transaction;
-
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexRegistryProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-public class TreeIndexInsertUpdateDeleteOperatorDescriptor extends AbstractTreeIndexOperatorDescriptor {
-
-    private static final long serialVersionUID = 1L;
-
-    private final int[] fieldPermutation;
-
-    private final IndexOp op;
-
-    private final long transactionId;
-
-    /**
-     * TODO: Index operators should live in Hyracks. Right now, they are needed
-     * here in Asterix as a hack to provide transactionIDs. The Asterix verions
-     * of this operator will disappear and the operator will come from Hyracks
-     * once the LSM/Recovery/Transactions world has been introduced.
-     */
-    public TreeIndexInsertUpdateDeleteOperatorDescriptor(JobSpecification spec, RecordDescriptor recDesc,
-            IStorageManagerInterface storageManager, IIndexRegistryProvider<IIndex> indexRegistryProvider,
-            IFileSplitProvider fileSplitProvider, ITypeTraits[] typeTraits,
-            IBinaryComparatorFactory[] comparatorFactories, int[] fieldPermutation, IndexOp op,
-            IIndexDataflowHelperFactory dataflowHelperFactory, ITupleFilterFactory tupleFilterFactory,
-            IOperationCallbackProvider opCallbackProvider, long transactionId) {
-        super(spec, 1, 1, recDesc, storageManager, indexRegistryProvider, fileSplitProvider, typeTraits,
-                comparatorFactories, dataflowHelperFactory, tupleFilterFactory, false, opCallbackProvider);
-        this.fieldPermutation = fieldPermutation;
-        this.op = op;
-        this.transactionId = transactionId;
-    }
-
-    @Override
-    public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
-            IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions) {
-        TransactionContext txnContext;
-        try {
-            ITransactionManager transactionManager = ((AsterixAppRuntimeContext) ctx.getJobletContext()
-                    .getApplicationContext().getApplicationObject()).getTransactionProvider().getTransactionManager();
-            txnContext = transactionManager.getTransactionContext(transactionId);
-        } catch (ACIDException ae) {
-            throw new RuntimeException(" could not obtain context for invalid transaction id " + transactionId);
-        }
-        return new TreeIndexInsertUpdateDeleteOperatorNodePushable(txnContext, this, ctx, partition, fieldPermutation,
-                recordDescProvider, op);
-    }
-}
diff --git a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java b/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
deleted file mode 100644
index 1fd8fee..0000000
--- a/asterix-hyracks-glue/src/main/java/edu/uci/ics/asterix/runtime/transaction/TreeIndexInsertUpdateDeleteOperatorNodePushable.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.runtime.transaction;
-
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
-import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.DataUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeLogger;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.FrameTupleReference;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilter;
-import edu.uci.ics.hyracks.storage.am.common.api.ITupleFilterFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.AbstractTreeIndexOperatorDescriptor;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.PermutingFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexDataflowHelper;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-
-public class TreeIndexInsertUpdateDeleteOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
-    private final IHyracksTaskContext ctx;
-    private FrameTupleAccessor accessor;
-    private TreeIndexDataflowHelper treeIndexHelper;
-    private final IRecordDescriptorProvider recordDescProvider;
-    private final IndexOp op;
-    private final PermutingFrameTupleReference tuple = new PermutingFrameTupleReference();
-    private FrameTupleReference frameTuple;
-    private ByteBuffer writeBuffer;
-    private IIndexAccessor indexAccessor;
-    private ILockManager lockManager;
-    private final TransactionContext txnContext;
-    private TreeLogger treeLogger;
-    private ITupleFilter tupleFilter;
-    private final TransactionProvider transactionProvider;
-
-    /* TODO: Index operators should live in Hyracks. Right now, they are needed here in Asterix
-     * as a hack to provide transactionIDs. The Asterix verions of this operator will disappear 
-     * and the operator will come from Hyracks once the LSM/Recovery/Transactions world has 
-     * been introduced.
-     */
-    public TreeIndexInsertUpdateDeleteOperatorNodePushable(TransactionContext txnContext,
-            AbstractTreeIndexOperatorDescriptor opDesc, IHyracksTaskContext ctx, int partition, int[] fieldPermutation,
-            IRecordDescriptorProvider recordDescProvider, IndexOp op) {
-        this.ctx = ctx;
-        treeIndexHelper = (TreeIndexDataflowHelper) opDesc.getIndexDataflowHelperFactory().createIndexDataflowHelper(
-                opDesc, ctx, partition);
-        this.recordDescProvider = recordDescProvider;
-        this.op = op;
-        tuple.setFieldPermutation(fieldPermutation);
-        this.txnContext = txnContext;
-        AsterixAppRuntimeContext runtimeContext = (AsterixAppRuntimeContext) ctx.getJobletContext()
-                .getApplicationContext().getApplicationObject();
-        transactionProvider = runtimeContext.getTransactionProvider();
-    }
-
-    public void initializeTransactionSupport() {
-        TransactionalResourceRepository resourceRepository = transactionProvider.getTransactionalResourceRepository();
-        IResourceManager resourceMgr = resourceRepository.getTransactionalResourceMgr(TreeResourceManager.ID);
-        if (resourceMgr == null) {
-            resourceRepository.registerTransactionalResourceManager(TreeResourceManager.ID, new TreeResourceManager(
-                    transactionProvider));
-        }
-        int fileId = treeIndexHelper.getIndexFileId();
-        byte[] resourceId = DataUtil.intToByteArray(fileId);
-        transactionProvider.getTransactionalResourceRepository().registerTransactionalResource(resourceId,
-                treeIndexHelper.getIndex());
-        lockManager = transactionProvider.getLockManager();
-        treeLogger = transactionProvider.getTreeLoggerRepository().getTreeLogger(resourceId);
-    }
-
-    @Override
-    public void open() throws HyracksDataException {
-        AbstractTreeIndexOperatorDescriptor opDesc = (AbstractTreeIndexOperatorDescriptor) treeIndexHelper
-                .getOperatorDescriptor();
-        RecordDescriptor inputRecDesc = recordDescProvider.getInputRecordDescriptor(opDesc.getActivityId(), 0);
-        accessor = new FrameTupleAccessor(treeIndexHelper.getHyracksTaskContext().getFrameSize(), inputRecDesc);
-        writeBuffer = treeIndexHelper.getHyracksTaskContext().allocateFrame();
-        writer.open();
-        try {
-            treeIndexHelper.init(false);
-            ITreeIndex treeIndex = (ITreeIndex) treeIndexHelper.getIndex();
-            indexAccessor = treeIndex.createAccessor();
-            ITupleFilterFactory tupleFilterFactory = opDesc.getTupleFilterFactory();
-            if (tupleFilterFactory != null) {
-                tupleFilter = tupleFilterFactory.createTupleFilter(ctx);
-                frameTuple = new FrameTupleReference();
-            }
-            initializeTransactionSupport();
-        } catch (Exception e) {
-            // cleanup in case of failure
-            treeIndexHelper.deinit();
-            throw new HyracksDataException(e);
-        }
-    }
-
-    @Override
-    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-        final IIndex treeIndex = treeIndexHelper.getIndex();
-        accessor.reset(buffer);
-        int fileId = treeIndexHelper.getIndexFileId();
-        byte[] resourceId = DataUtil.intToByteArray(fileId);
-        int tupleCount = accessor.getTupleCount();
-        try {
-            for (int i = 0; i < tupleCount; i++) {
-                if (tupleFilter != null) {
-                    frameTuple.reset(accessor, i);
-                    if (!tupleFilter.accept(frameTuple)) {
-                        continue;
-                    }
-                }
-                tuple.reset(accessor, i);
-                switch (op) {
-                    case INSERT: {
-                        lockManager.lock(txnContext, resourceId,
-                                TransactionManagementConstants.LockManagerConstants.LockMode.EXCLUSIVE);
-                        indexAccessor.insert(tuple);
-                        treeLogger.generateLogRecord(transactionProvider, txnContext, op, tuple);
-                        break;
-                    }
-
-                    case DELETE: {
-                        lockManager.lock(txnContext, resourceId,
-                                TransactionManagementConstants.LockManagerConstants.LockMode.EXCLUSIVE);
-                        indexAccessor.delete(tuple);
-                        treeLogger.generateLogRecord(transactionProvider, txnContext, op, tuple);
-                        break;
-                    }
-
-                    default: {
-                        throw new HyracksDataException("Unsupported operation " + op
-                                + " in tree index InsertUpdateDelete operator");
-                    }
-                }
-            }
-        } catch (ACIDException ae) {
-            throw new HyracksDataException("exception in locking/logging during operation " + op + " on tree "
-                    + treeIndex, ae);
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new HyracksDataException(e);
-        }
-
-        // pass a copy of the frame to next op
-        System.arraycopy(buffer.array(), 0, writeBuffer.array(), 0, buffer.capacity());
-        FrameUtils.flushFrame(writeBuffer, writer);
-
-    }
-
-    @Override
-    public void close() throws HyracksDataException {
-        try {
-            writer.close();
-        } finally {
-            txnContext.addCloseableResource(new ICloseable() {
-                @Override
-                public void close(TransactionContext txnContext) throws ACIDException {
-                    try {
-                        treeIndexHelper.deinit();
-                    } catch (Exception e) {
-                        throw new ACIDException(txnContext, "could not de-initialize " + treeIndexHelper, e);
-                    }
-                }
-            });
-        }
-    }
-
-    @Override
-    public void fail() throws HyracksDataException {
-        try {
-            writer.fail();
-        } finally {
-            txnContext.addCloseableResource(new ICloseable() {
-                @Override
-                public void close(TransactionContext txnContext) throws ACIDException {
-                    try {
-                        treeIndexHelper.deinit();
-                    } catch (Exception e) {
-                        throw new ACIDException(txnContext, "could not de-initialize " + treeIndexHelper, e);
-                    }
-                }
-            });
-        }
-    }
-
-}
diff --git a/asterix-metadata/pom.xml b/asterix-metadata/pom.xml
index 67f16ff..d95e29a 100644
--- a/asterix-metadata/pom.xml
+++ b/asterix-metadata/pom.xml
@@ -1,4 +1,5 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
 		<artifactId>asterix</artifactId>
@@ -43,27 +44,32 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-invertedindex</artifactId>
-			<version>0.2.2-SNAPSHOT</version>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-rtree</artifactId>
-			<version>0.2.2-SNAPSHOT</version>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.asterix</groupId>
-			<artifactId>asterix-hyracks-glue</artifactId>
-			<version>0.0.4-SNAPSHOT</version>
+			<groupId>org.apache.hadoop</groupId>
+			<artifactId>hadoop-core</artifactId>
+			<version>0.20.2</version>
+			<type>jar</type>
 			<scope>compile</scope>
 		</dependency>
-                <dependency>
-                        <groupId>org.apache.hadoop</groupId>
-                        <artifactId>hadoop-core</artifactId>
-                        <version>0.20.2</version>
-                        <type>jar</type>
-                        <scope>compile</scope>
-                </dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+                		hyracks-storage-am-lsm-invertedindex
+                	</artifactId>
+			<version>0.2.2-SNAPSHOT</version>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+                		hyracks-storage-am-lsm-btree
+                	</artifactId>
+			<version>0.2.2-SNAPSHOT</version>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+                		hyracks-storage-am-lsm-rtree
+                	</artifactId>
+			<version>0.2.2-SNAPSHOT</version>
+		</dependency>
 	</dependencies>
 </project>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index bd4f9bb..dfb272c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -29,7 +29,8 @@
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 
 /**
  * Provides access to Asterix metadata via remote methods to the metadata node.
@@ -105,36 +106,36 @@
 
     @Override
     public MetadataTransactionContext beginTransaction() throws RemoteException, ACIDException {
-        long txnId = TransactionIDFactory.generateTransactionId();
-        metadataNode.beginTransaction(txnId);
-        return new MetadataTransactionContext(txnId);
+        JobId jobId = JobIdFactory.generateJobId();
+        metadataNode.beginTransaction(jobId);
+        return new MetadataTransactionContext(jobId);
     }
 
     @Override
     public void commitTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
-        metadataNode.commitTransaction(ctx.getTxnId());
+        metadataNode.commitTransaction(ctx.getJobId());
         cache.commit(ctx);
     }
 
     @Override
     public void abortTransaction(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
-        metadataNode.abortTransaction(ctx.getTxnId());
+        metadataNode.abortTransaction(ctx.getJobId());
     }
 
     @Override
-    public void lock(MetadataTransactionContext ctx, int lockMode) throws RemoteException, ACIDException {
-        metadataNode.lock(ctx.getTxnId(), lockMode);
+    public void lock(MetadataTransactionContext ctx, byte lockMode) throws RemoteException, ACIDException {
+        metadataNode.lock(ctx.getJobId(), lockMode);
     }
 
     @Override
     public void unlock(MetadataTransactionContext ctx) throws RemoteException, ACIDException {
-        metadataNode.unlock(ctx.getTxnId());
+        metadataNode.unlock(ctx.getJobId());
     }
 
     @Override
     public void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws MetadataException {
         try {
-            metadataNode.addDataverse(ctx.getTxnId(), dataverse);
+            metadataNode.addDataverse(ctx.getJobId(), dataverse);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -144,7 +145,7 @@
     @Override
     public void dropDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException {
         try {
-            metadataNode.dropDataverse(ctx.getTxnId(), dataverseName);
+            metadataNode.dropDataverse(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -172,7 +173,7 @@
             return dataverse;
         }
         try {
-            dataverse = metadataNode.getDataverse(ctx.getTxnId(), dataverseName);
+            dataverse = metadataNode.getDataverse(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -191,7 +192,7 @@
         try {
             // Assuming that the transaction can read its own writes on the
             // metadata node.
-            dataverseDatasets = metadataNode.getDataverseDatasets(ctx.getTxnId(), dataverseName);
+            dataverseDatasets = metadataNode.getDataverseDatasets(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -203,7 +204,7 @@
     @Override
     public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException {
         try {
-            metadataNode.addDataset(ctx.getTxnId(), dataset);
+            metadataNode.addDataset(ctx.getJobId(), dataset);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -214,7 +215,7 @@
     public void dropDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
             throws MetadataException {
         try {
-            metadataNode.dropDataset(ctx.getTxnId(), dataverseName, datasetName);
+            metadataNode.dropDataset(ctx.getJobId(), dataverseName, datasetName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -248,7 +249,7 @@
             return dataset;
         }
         try {
-            dataset = metadataNode.getDataset(ctx.getTxnId(), dataverseName, datasetName);
+            dataset = metadataNode.getDataset(ctx.getJobId(), dataverseName, datasetName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -265,7 +266,7 @@
             throws MetadataException {
         List<Index> datsetIndexes;
         try {
-            datsetIndexes = metadataNode.getDatasetIndexes(ctx.getTxnId(), dataverseName, datasetName);
+            datsetIndexes = metadataNode.getDatasetIndexes(ctx.getJobId(), dataverseName, datasetName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -275,7 +276,7 @@
     @Override
     public void addDatatype(MetadataTransactionContext ctx, Datatype datatype) throws MetadataException {
         try {
-            metadataNode.addDatatype(ctx.getTxnId(), datatype);
+            metadataNode.addDatatype(ctx.getJobId(), datatype);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -286,7 +287,7 @@
     public void dropDatatype(MetadataTransactionContext ctx, String dataverseName, String datatypeName)
             throws MetadataException {
         try {
-            metadataNode.dropDatatype(ctx.getTxnId(), dataverseName, datatypeName);
+            metadataNode.dropDatatype(ctx.getJobId(), dataverseName, datatypeName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -320,7 +321,7 @@
             return datatype;
         }
         try {
-            datatype = metadataNode.getDatatype(ctx.getTxnId(), dataverseName, datatypeName);
+            datatype = metadataNode.getDatatype(ctx.getJobId(), dataverseName, datatypeName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -335,7 +336,7 @@
     @Override
     public void addIndex(MetadataTransactionContext ctx, Index index) throws MetadataException {
         try {
-            metadataNode.addIndex(ctx.getTxnId(), index);
+            metadataNode.addIndex(ctx.getJobId(), index);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -345,7 +346,7 @@
     public void dropIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
             throws MetadataException {
         try {
-            metadataNode.dropIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
+            metadataNode.dropIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -355,7 +356,7 @@
     public Index getIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
             throws MetadataException {
         try {
-            return metadataNode.getIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
+            return metadataNode.getIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -364,7 +365,7 @@
     @Override
     public void addNode(MetadataTransactionContext ctx, Node node) throws MetadataException {
         try {
-            metadataNode.addNode(ctx.getTxnId(), node);
+            metadataNode.addNode(ctx.getJobId(), node);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -373,7 +374,7 @@
     @Override
     public void addNodegroup(MetadataTransactionContext ctx, NodeGroup nodeGroup) throws MetadataException {
         try {
-            metadataNode.addNodeGroup(ctx.getTxnId(), nodeGroup);
+            metadataNode.addNodeGroup(ctx.getJobId(), nodeGroup);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -383,7 +384,7 @@
     @Override
     public void dropNodegroup(MetadataTransactionContext ctx, String nodeGroupName) throws MetadataException {
         try {
-            metadataNode.dropNodegroup(ctx.getTxnId(), nodeGroupName);
+            metadataNode.dropNodegroup(ctx.getJobId(), nodeGroupName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -411,7 +412,7 @@
             return nodeGroup;
         }
         try {
-            nodeGroup = metadataNode.getNodeGroup(ctx.getTxnId(), nodeGroupName);
+            nodeGroup = metadataNode.getNodeGroup(ctx.getJobId(), nodeGroupName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -426,7 +427,7 @@
     @Override
     public void addFunction(MetadataTransactionContext mdTxnCtx, Function function) throws MetadataException {
         try {
-            metadataNode.addFunction(mdTxnCtx.getTxnId(), function);
+            metadataNode.addFunction(mdTxnCtx.getJobId(), function);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -437,7 +438,7 @@
     public void dropFunction(MetadataTransactionContext ctx, String dataverseName, String functionName, int arity)
             throws MetadataException {
         try {
-            metadataNode.dropFunction(ctx.getTxnId(), dataverseName, functionName, arity);
+            metadataNode.dropFunction(ctx.getJobId(), dataverseName, functionName, arity);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -471,7 +472,7 @@
             return function;
         }
         try {
-            function = metadataNode.getFunction(ctx.getTxnId(), dataverseName, functionName, arity);
+            function = metadataNode.getFunction(ctx.getJobId(), dataverseName, functionName, arity);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -483,4 +484,13 @@
         return function;
 
     }
+
+    @Override
+    public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException {
+        try {
+            metadataNode.initializeDatasetIdFactory(ctx.getJobId());
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+    }
 }
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index 591154a..c634d95 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -53,6 +53,9 @@
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetIdFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
@@ -63,27 +66,27 @@
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.btree.api.IBTreeLeafFrame;
 import edu.uci.ics.hyracks.storage.am.btree.exceptions.BTreeDuplicateKeyException;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTreeRangeSearchCursor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.RangePredicate;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexCursor;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexCursor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrame;
 import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.common.ophelpers.MultiComparator;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeRangeSearchCursor;
 
 public class MetadataNode implements IMetadataNode {
     private static final long serialVersionUID = 1L;
 
     // TODO: Temporary transactional resource id for metadata.
     private static final byte[] metadataResourceId = MetadataNode.class.toString().getBytes();
+    private static final DatasetId METADATA_DATASET_ID = new DatasetId(MetadataPrimaryIndexes.METADATA_DATASET_ID);
 
-    private IndexRegistry<IIndex> indexRegistry;
+    private IIndexLifecycleManager indexLifecycleManager;
     private TransactionProvider transactionProvider;
 
     public static final MetadataNode INSTANCE = new MetadataNode();
@@ -94,24 +97,24 @@
 
     public void initialize(AsterixAppRuntimeContext runtimeContext) {
         this.transactionProvider = runtimeContext.getTransactionProvider();
-        this.indexRegistry = runtimeContext.getIndexRegistry();
+        this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
     }
 
     @Override
-    public void beginTransaction(long transactionId) throws ACIDException, RemoteException {
+    public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
         transactionProvider.getTransactionManager().beginTransaction(transactionId);
     }
 
     @Override
-    public void commitTransaction(long txnId) throws RemoteException, ACIDException {
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
+    public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
+        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
         transactionProvider.getTransactionManager().commitTransaction(txnCtx);
     }
 
     @Override
-    public void abortTransaction(long txnId) throws RemoteException, ACIDException {
+    public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
         try {
-            TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
+            TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
             transactionProvider.getTransactionManager().abortTransaction(txnCtx);
         } catch (ACIDException e) {
             e.printStackTrace();
@@ -120,23 +123,23 @@
     }
 
     @Override
-    public boolean lock(long txnId, int lockMode) throws ACIDException, RemoteException {
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
-        return transactionProvider.getLockManager().lock(txnCtx, metadataResourceId, lockMode);
+    public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
+        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+        transactionProvider.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
     }
 
     @Override
-    public boolean unlock(long txnId) throws ACIDException, RemoteException {
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
-        return transactionProvider.getLockManager().unlock(txnCtx, metadataResourceId);
+    public void unlock(JobId jobId) throws ACIDException, RemoteException {
+        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+        transactionProvider.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
     }
 
     @Override
-    public void addDataverse(long txnId, Dataverse dataverse) throws MetadataException, RemoteException {
+    public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException {
         try {
             DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(dataverse);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("A dataverse with this name " + dataverse.getDataverseName()
                     + " already exists.", e);
@@ -146,26 +149,26 @@
     }
 
     @Override
-    public void addDataset(long txnId, Dataset dataset) throws MetadataException, RemoteException {
+    public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException {
         try {
             // Insert into the 'dataset' dataset.
             DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
             ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
                 // Add the primary index for the dataset.
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
                         dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true);
-                addIndex(txnId, primaryIndex);
+                addIndex(jobId, primaryIndex);
                 ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
                         dataset.getDatasetName());
-                insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
+                insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
             }
             // Add entry in datatype secondary index.
             ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
                     dataset.getDatasetName());
-            insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+            insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
                     + " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
@@ -175,11 +178,11 @@
     }
 
     @Override
-    public void addIndex(long txnId, Index index) throws MetadataException, RemoteException {
+    public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
         try {
             IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
             ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
         } catch (Exception e) {
@@ -188,11 +191,11 @@
     }
 
     @Override
-    public void addNode(long txnId, Node node) throws MetadataException, RemoteException {
+    public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
         try {
             NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
         } catch (Exception e) {
@@ -201,11 +204,11 @@
     }
 
     @Override
-    public void addNodeGroup(long txnId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
+    public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
         try {
             NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(nodeGroup);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("A nodegroup with name '" + nodeGroup.getNodeGroupName() + "' already exists.",
                     e);
@@ -215,11 +218,11 @@
     }
 
     @Override
-    public void addDatatype(long txnId, Datatype datatype) throws MetadataException, RemoteException {
+    public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException {
         try {
-            DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(txnId, this, true);
+            DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, true);
             ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(datatype);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("A datatype with name '" + datatype.getDatatypeName() + "' already exists.", e);
         } catch (Exception e) {
@@ -228,12 +231,12 @@
     }
 
     @Override
-    public void addFunction(long txnId, Function function) throws MetadataException, RemoteException {
+    public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException {
         try {
             // Insert into the 'function' dataset.
             FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(true);
             ITupleReference functionTuple = tupleReaderWriter.getTupleFromMetadataEntity(function);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, functionTuple);
 
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("A dataset with this name " + function.getFunctionName() + " and arity "
@@ -244,53 +247,55 @@
         }
     }
 
-    public void insertIntoDatatypeSecondaryIndex(long txnId, String dataverseName, String nestedTypeName,
+    public void insertIntoDatatypeSecondaryIndex(JobId jobId, String dataverseName, String nestedTypeName,
             String topTypeName) throws Exception {
         ITupleReference tuple = createTuple(dataverseName, nestedTypeName, topTypeName);
-        insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+        insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
     }
 
-    private void insertTupleIntoIndex(long txnId, IMetadataIndex index, ITupleReference tuple) throws Exception {
-        int fileId = index.getFileId();
-        BTree btree = (BTree) indexRegistry.get(fileId);
-        btree.open(fileId);
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
-        transactionProvider.getLockManager().lock(txnCtx, index.getResourceId(), LockMode.EXCLUSIVE);
+    private void insertTupleIntoIndex(JobId jobId, IMetadataIndex index, ITupleReference tuple) throws Exception {
+        long resourceID = index.getResourceID();
+        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+        indexLifecycleManager.open(resourceID);
+        IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+        transactionProvider.getLockManager().lock(index.getDatasetId(), -1, LockMode.X, txnCtx);
         // TODO: fix exceptions once new BTree exception model is in hyracks.
         indexAccessor.insert(tuple);
-        index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOp.INSERT, tuple);
+        index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOperation.INSERT, tuple);
+        indexLifecycleManager.close(resourceID);
     }
 
     @Override
-    public void dropDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException {
+    public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
         try {
             List<Dataset> dataverseDatasets;
             // As a side effect, acquires an S lock on the 'dataset' dataset
             // on behalf of txnId.
-            dataverseDatasets = getDataverseDatasets(txnId, dataverseName);
+            dataverseDatasets = getDataverseDatasets(jobId, dataverseName);
             if (dataverseDatasets != null && dataverseDatasets.size() > 0) {
                 // Drop all datasets in this dataverse.
                 for (int i = 0; i < dataverseDatasets.size(); i++) {
-                    dropDataset(txnId, dataverseName, dataverseDatasets.get(i).getDatasetName());
+                    dropDataset(jobId, dataverseName, dataverseDatasets.get(i).getDatasetName());
                 }
             }
             List<Datatype> dataverseDatatypes;
             // As a side effect, acquires an S lock on the 'datatype' dataset
             // on behalf of txnId.
-            dataverseDatatypes = getDataverseDatatypes(txnId, dataverseName);
+            dataverseDatatypes = getDataverseDatatypes(jobId, dataverseName);
             if (dataverseDatatypes != null && dataverseDatatypes.size() > 0) {
                 // Drop all types in this dataverse.
                 for (int i = 0; i < dataverseDatatypes.size(); i++) {
-                    forceDropDatatype(txnId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
+                    forceDropDatatype(jobId, dataverseName, dataverseDatatypes.get(i).getDatatypeName());
                 }
             }
             // Delete the dataverse entry from the 'dataverse' dataset.
             ITupleReference searchKey = createTuple(dataverseName);
             // As a side effect, acquires an S lock on the 'dataverse' dataset
             // on behalf of txnId.
-            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, tuple);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
         } catch (TreeIndexException e) {
@@ -301,11 +306,11 @@
     }
 
     @Override
-    public void dropDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+    public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
             RemoteException {
         Dataset dataset;
         try {
-            dataset = getDataset(txnId, dataverseName, datasetName);
+            dataset = getDataset(jobId, dataverseName, datasetName);
         } catch (Exception e) {
             throw new MetadataException(e);
         }
@@ -317,30 +322,30 @@
             ITupleReference searchKey = createTuple(dataverseName, datasetName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'dataset' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             // Delete entry from secondary index 'group'.
             if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
                 // Searches the index for the tuple to be deleted. Acquires an S
                 // lock on the GROUPNAME_ON_DATASET_INDEX index.
-                ITupleReference groupNameTuple = getTupleToBeDeleted(txnId,
+                ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
                         MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
-                deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
+                deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
             }
             // Delete entry from secondary index 'type'.
             ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getItemTypeName(), datasetName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the DATATYPENAME_ON_DATASET_INDEX index.
-            ITupleReference dataTypeTuple = getTupleToBeDeleted(txnId,
+            ITupleReference dataTypeTuple = getTupleToBeDeleted(jobId,
                     MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
-            deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+            deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
             // Delete entry(s) from the 'indexes' dataset.
             if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
-                List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
+                List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
                 for (Index index : datasetIndexes) {
-                    dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
+                    dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
                 }
             }
             // TODO: Change this to be a BTree specific exception, e.g.,
@@ -353,14 +358,14 @@
     }
 
     @Override
-    public void dropIndex(long txnId, String dataverseName, String datasetName, String indexName)
+    public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'index' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
         } catch (TreeIndexException e) {
@@ -372,10 +377,10 @@
     }
 
     @Override
-    public void dropNodegroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException {
+    public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
         List<String> datasetNames;
         try {
-            datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(txnId, nodeGroupName);
+            datasetNames = getDatasetNamesPartitionedOnThisNodeGroup(jobId, nodeGroupName);
         } catch (Exception e) {
             throw new MetadataException(e);
         }
@@ -391,8 +396,8 @@
             ITupleReference searchKey = createTuple(nodeGroupName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'nodegroup' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, tuple);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
         } catch (TreeIndexException e) {
@@ -403,13 +408,13 @@
     }
 
     @Override
-    public void dropDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+    public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
             RemoteException {
         List<String> datasetNames;
         List<String> usedDatatypes;
         try {
-            datasetNames = getDatasetNamesDeclaredByThisDatatype(txnId, dataverseName, datatypeName);
-            usedDatatypes = getDatatypeNamesUsingThisDatatype(txnId, dataverseName, datatypeName);
+            datasetNames = getDatasetNamesDeclaredByThisDatatype(jobId, dataverseName, datatypeName);
+            usedDatatypes = getDatatypeNamesUsingThisDatatype(jobId, dataverseName, datatypeName);
         } catch (Exception e) {
             throw new MetadataException(e);
         }
@@ -434,14 +439,14 @@
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'datatype' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
-            deleteFromDatatypeSecondaryIndex(txnId, dataverseName, datatypeName);
-            List<String> nestedTypes = getNestedDatatypeNames(txnId, dataverseName, datatypeName);
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+            deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
+            List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
             for (String nestedType : nestedTypes) {
-                Datatype dt = getDatatype(txnId, dataverseName, nestedType);
+                Datatype dt = getDatatype(jobId, dataverseName, nestedType);
                 if (dt != null && dt.getIsAnonymous()) {
-                    dropDatatype(txnId, dataverseName, dt.getDatatypeName());
+                    dropDatatype(jobId, dataverseName, dt.getDatatypeName());
                 }
             }
             // TODO: Change this to be a BTree specific exception, e.g.,
@@ -453,14 +458,14 @@
         }
     }
 
-    private void forceDropDatatype(long txnId, String dataverseName, String datatypeName) throws AsterixException {
+    private void forceDropDatatype(JobId jobId, String dataverseName, String datatypeName) throws AsterixException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'datatype' dataset.
-            ITupleReference tuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
-            deleteFromDatatypeSecondaryIndex(txnId, dataverseName, datatypeName);
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+            deleteFromDatatypeSecondaryIndex(jobId, dataverseName, datatypeName);
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
         } catch (TreeIndexException e) {
@@ -472,17 +477,17 @@
         }
     }
 
-    private void deleteFromDatatypeSecondaryIndex(long txnId, String dataverseName, String datatypeName)
+    private void deleteFromDatatypeSecondaryIndex(JobId jobId, String dataverseName, String datatypeName)
             throws AsterixException {
         try {
-            List<String> nestedTypes = getNestedDatatypeNames(txnId, dataverseName, datatypeName);
+            List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
             for (String nestedType : nestedTypes) {
                 ITupleReference searchKey = createTuple(dataverseName, nestedType, datatypeName);
                 // Searches the index for the tuple to be deleted. Acquires an S
                 // lock on the DATATYPENAME_ON_DATATYPE_INDEX index.
-                ITupleReference tuple = getTupleToBeDeleted(txnId,
+                ITupleReference tuple = getTupleToBeDeleted(jobId,
                         MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey);
-                deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
+                deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, tuple);
             }
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -495,31 +500,32 @@
         }
     }
 
-    private void deleteTupleFromIndex(long txnId, IMetadataIndex index, ITupleReference tuple) throws Exception {
-        int fileId = index.getFileId();
-        BTree btree = (BTree) indexRegistry.get(fileId);
-        btree.open(fileId);
-
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
+    private void deleteTupleFromIndex(JobId jobId, IMetadataIndex index, ITupleReference tuple) throws Exception {
+        long resourceID = index.getResourceID();
+        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+        indexLifecycleManager.open(resourceID);
+        IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
         // This lock is actually an upgrade, because a deletion must be preceded
         // by a search, in order to be able to undo an aborted deletion.
         // The transaction with txnId will have an S lock on the
         // resource. Note that lock converters have a higher priority than
         // regular waiters in the LockManager.
-        transactionProvider.getLockManager().lock(txnCtx, index.getResourceId(), LockMode.EXCLUSIVE);
+        transactionProvider.getLockManager().lock(index.getDatasetId(), -1, LockMode.X, txnCtx);
         indexAccessor.delete(tuple);
-        index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOp.DELETE, tuple);
+        index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOperation.DELETE, tuple);
+        indexLifecycleManager.close(resourceID);
     }
 
     @Override
-    public Dataverse getDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException {
+    public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
             IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
             List<Dataverse> results = new ArrayList<Dataverse>();
-            searchIndex(txnId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -531,28 +537,28 @@
     }
 
     @Override
-    public List<Dataset> getDataverseDatasets(long txnId, String dataverseName) throws MetadataException,
+    public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
             IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
             List<Dataset> results = new ArrayList<Dataset>();
-            searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (Exception e) {
             throw new MetadataException(e);
         }
     }
 
-    private List<Datatype> getDataverseDatatypes(long txnId, String dataverseName) throws MetadataException,
+    private List<Datatype> getDataverseDatatypes(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
-            DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(txnId, this, false);
+            DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<Datatype>();
-            searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (Exception e) {
             throw new MetadataException(e);
@@ -560,14 +566,14 @@
     }
 
     @Override
-    public Dataset getDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+    public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
             RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName);
             DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
             List<Dataset> results = new ArrayList<Dataset>();
             IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
-            searchIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -577,13 +583,13 @@
         }
     }
 
-    private List<String> getDatasetNamesDeclaredByThisDatatype(long txnId, String dataverseName, String datatypeName)
+    private List<String> getDatasetNamesDeclaredByThisDatatype(JobId jobId, String dataverseName, String datatypeName)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, dataverseName);
             List<String> results = new ArrayList<String>();
             IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
-            searchIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
+            searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, searchKey, valueExtractor,
                     results);
             return results;
         } catch (Exception e) {
@@ -591,13 +597,13 @@
         }
     }
 
-    public List<String> getDatatypeNamesUsingThisDatatype(long txnId, String dataverseName, String datatypeName)
+    public List<String> getDatatypeNamesUsingThisDatatype(JobId jobId, String dataverseName, String datatypeName)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
             List<String> results = new ArrayList<String>();
             IValueExtractor<String> valueExtractor = new DatatypeNameValueExtractor(dataverseName, this);
-            searchIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+            searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
                     results);
             return results;
         } catch (Exception e) {
@@ -605,13 +611,13 @@
         }
     }
 
-    private List<String> getNestedDatatypeNames(long txnId, String dataverseName, String datatypeName)
+    private List<String> getNestedDatatypeNames(JobId jobId, String dataverseName, String datatypeName)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             List<String> results = new ArrayList<String>();
             IValueExtractor<String> valueExtractor = new NestedDatatypeNameValueExtractor(datatypeName);
-            searchIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
+            searchIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX, searchKey, valueExtractor,
                     results);
             return results;
         } catch (Exception e) {
@@ -619,13 +625,13 @@
         }
     }
 
-    public List<String> getDatasetNamesPartitionedOnThisNodeGroup(long txnId, String nodegroup)
+    public List<String> getDatasetNamesPartitionedOnThisNodeGroup(JobId jobId, String nodegroup)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(nodegroup);
             List<String> results = new ArrayList<String>();
             IValueExtractor<String> valueExtractor = new DatasetNameValueExtractor();
-            searchIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, searchKey, valueExtractor, results);
             return results;
         } catch (Exception e) {
             throw new MetadataException(e);
@@ -633,14 +639,14 @@
     }
 
     @Override
-    public Index getIndex(long txnId, String dataverseName, String datasetName, String indexName)
+    public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName, indexName);
             IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
             IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
             List<Index> results = new ArrayList<Index>();
-            searchIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -651,14 +657,14 @@
     }
 
     @Override
-    public List<Index> getDatasetIndexes(long txnId, String dataverseName, String datasetName)
+    public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datasetName);
             IndexTupleTranslator tupleReaderWriter = new IndexTupleTranslator(false);
             IValueExtractor<Index> valueExtractor = new MetadataEntityValueExtractor<Index>(tupleReaderWriter);
             List<Index> results = new ArrayList<Index>();
-            searchIndex(txnId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (Exception e) {
             throw new MetadataException(e);
@@ -666,14 +672,14 @@
     }
 
     @Override
-    public Datatype getDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+    public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
             RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, datatypeName);
-            DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(txnId, this, false);
+            DatatypeTupleTranslator tupleReaderWriter = new DatatypeTupleTranslator(jobId, this, false);
             IValueExtractor<Datatype> valueExtractor = new MetadataEntityValueExtractor<Datatype>(tupleReaderWriter);
             List<Datatype> results = new ArrayList<Datatype>();
-            searchIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -684,13 +690,13 @@
     }
 
     @Override
-    public NodeGroup getNodeGroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException {
+    public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(nodeGroupName);
             NodeGroupTupleTranslator tupleReaderWriter = new NodeGroupTupleTranslator(false);
             IValueExtractor<NodeGroup> valueExtractor = new MetadataEntityValueExtractor<NodeGroup>(tupleReaderWriter);
             List<NodeGroup> results = new ArrayList<NodeGroup>();
-            searchIndex(txnId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.NODEGROUP_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -701,14 +707,14 @@
     }
 
     @Override
-    public Function getFunction(long txnId, String dataverseName, String functionName, int arity)
+    public Function getFunction(JobId jobId, String dataverseName, String functionName, int arity)
             throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, functionName, "" + arity);
             FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
             List<Function> results = new ArrayList<Function>();
             IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
-            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -719,11 +725,11 @@
     }
 
     @Override
-    public void dropFunction(long txnId, String dataverseName, String functionName, int arity)
+    public void dropFunction(JobId jobId, String dataverseName, String functionName, int arity)
             throws MetadataException, RemoteException {
         Function function;
         try {
-            function = getFunction(txnId, dataverseName, functionName, arity);
+            function = getFunction(jobId, dataverseName, functionName, arity);
         } catch (Exception e) {
             throw new MetadataException(e);
         }
@@ -736,9 +742,9 @@
             ITupleReference searchKey = createTuple(dataverseName, functionName, "" + arity);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'function' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET,
+            ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET,
                     searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, datasetTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -750,11 +756,11 @@
         }
     }
 
-    private ITupleReference getTupleToBeDeleted(long txnId, IMetadataIndex metadataIndex, ITupleReference searchKey)
+    private ITupleReference getTupleToBeDeleted(JobId jobId, IMetadataIndex metadataIndex, ITupleReference searchKey)
             throws Exception {
         IValueExtractor<ITupleReference> valueExtractor = new TupleCopyValueExtractor(metadataIndex.getTypeTraits());
         List<ITupleReference> results = new ArrayList<ITupleReference>();
-        searchIndex(txnId, metadataIndex, searchKey, valueExtractor, results);
+        searchIndex(jobId, metadataIndex, searchKey, valueExtractor, results);
         if (results.isEmpty()) {
             // TODO: Temporarily a TreeIndexException to make it get caught by
             // caller in the appropriate catch block.
@@ -764,17 +770,17 @@
         return results.get(0);
     }
 
-    private <ResultType> void searchIndex(long txnId, IMetadataIndex index, ITupleReference searchKey,
+    private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
             IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
-        transactionProvider.getLockManager().lock(txnCtx, index.getResourceId(), LockMode.SHARED);
+        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(jobId);
+        transactionProvider.getLockManager().lock(index.getDatasetId(), -1, LockMode.S, txnCtx);
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
-        int fileId = index.getFileId();
-        BTree btree = (BTree) indexRegistry.get(fileId);
-        btree.open(fileId);
-        ITreeIndexFrame leafFrame = btree.getLeafFrameFactory().createFrame();
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
-        ITreeIndexCursor rangeCursor = new BTreeRangeSearchCursor((IBTreeLeafFrame) leafFrame, false);
+        long resourceID = index.getResourceID();
+        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+        indexLifecycleManager.open(resourceID);
+        IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        ITreeIndexCursor rangeCursor = new LSMBTreeRangeSearchCursor();
         IBinaryComparator[] searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
         for (int i = 0; i < searchKey.getFieldCount(); i++) {
             searchCmps[i] = comparatorFactories[i].createBinaryComparator();
@@ -786,7 +792,7 @@
         try {
             while (rangeCursor.hasNext()) {
                 rangeCursor.next();
-                ResultType result = valueExtractor.getValue(txnId, rangeCursor.getTuple());
+                ResultType result = valueExtractor.getValue(jobId, rangeCursor.getTuple());
                 if (result != null) {
                     results.add(result);
                 }
@@ -794,6 +800,44 @@
         } finally {
             rangeCursor.close();
         }
+        indexLifecycleManager.close(resourceID);
+    }
+
+    @Override
+    public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException {
+        int mostRecentDatasetId = MetadataPrimaryIndexes.FIRST_AVAILABLE_USER_DATASET_ID;
+        long resourceID = MetadataPrimaryIndexes.DATASET_DATASET.getResourceID();
+        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+        try {
+            indexLifecycleManager.open(resourceID);
+            IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            IIndexCursor rangeCursor = indexAccessor.createSearchCursor();
+
+            DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(false);
+            IValueExtractor<Dataset> valueExtractor = new MetadataEntityValueExtractor<Dataset>(tupleReaderWriter);
+            RangePredicate rangePred = new RangePredicate(null, null, true, true, null, null);
+
+            indexAccessor.search(rangeCursor, rangePred);
+            int datasetId;
+
+            try {
+                while (rangeCursor.hasNext()) {
+                    rangeCursor.next();
+                    datasetId = ((Dataset) valueExtractor.getValue(jobId, rangeCursor.getTuple())).getDatasetId();
+                    if (mostRecentDatasetId < datasetId) {
+                        mostRecentDatasetId = datasetId;
+                    }
+                }
+            } finally {
+                rangeCursor.close();
+            }
+
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+        DatasetIdFactory.initialize(mostRecentDatasetId);
     }
 
     // TODO: Can use Hyrack's TupleUtils for this, once we switch to a newer
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
index e0aa69b..329b770 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataTransactionContext.java
@@ -22,6 +22,7 @@
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
  * Used to implement serializable transactions against the MetadataCache.
@@ -53,14 +54,14 @@
 	protected MetadataCache droppedCache = new MetadataCache();
 
 	protected ArrayList<MetadataLogicalOperation> opLog = new ArrayList<MetadataLogicalOperation>();
-	private final long txnId;
+	private final JobId jobId;
 
-	public MetadataTransactionContext(long txnId) {
-		this.txnId = txnId;
+	public MetadataTransactionContext(JobId jobId) {
+		this.jobId = jobId;
 	}
 
-	public long getTxnId() {
-		return txnId;
+	public JobId getJobId() {
+		return jobId;
 	}
 
 	public void addDataverse(Dataverse dataverse) {
@@ -96,7 +97,7 @@
 
 	public void dropDataset(String dataverseName, String datasetName) {
 		Dataset dataset = new Dataset(dataverseName, datasetName, null, null,
-				null);
+				null, -1);
 		droppedCache.addDatasetIfNotExists(dataset);
 		logAndApply(new MetadataLogicalOperation(dataset, false));
 	}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
index f9e5540..88dac70 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataIndex.java
@@ -20,10 +20,12 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.service.logging.TreeLogger;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 
 /**
@@ -58,13 +60,23 @@
 
     public ARecordType getPayloadRecordType();
 
+    public void setFile(FileReference file);
+
+    public FileReference getFile();
+
     public void setFileId(int fileId);
 
     public void initTreeLogger(ITreeIndex treeIndex) throws ACIDException;
 
     public int getFileId();
 
+    public void setResourceID(long resourceID);
+
+    public long getResourceID();
+
     public byte[] getResourceId();
 
     public TreeLogger getTreeLogger();
+    
+    public DatasetId getDatasetId();
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
index adba1c2..779cbe4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
@@ -28,6 +28,7 @@
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
  * A metadata manager provides user access to Asterix metadata (e.g., types,
@@ -92,7 +93,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    public void lock(MetadataTransactionContext ctx, int lockMode) throws ACIDException, RemoteException;
+    public void lock(MetadataTransactionContext ctx, byte lockMode) throws ACIDException, RemoteException;
 
     /**
      * Releases all locks on the metadata held by the given transaction id.
@@ -397,4 +398,6 @@
 	public void dropFunction(MetadataTransactionContext ctx,
 			String dataverseName, String functionName, int arity)
 			throws MetadataException;
+	
+    public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
index 10555be..4bc513a 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
  * A metadata node stores metadata in its local storage structures (currently
@@ -47,7 +48,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    public void beginTransaction(long txnId) throws ACIDException, RemoteException;
+    public void beginTransaction(JobId jobId) throws ACIDException, RemoteException;
 
     /**
      * Commits a local transaction against the metadata.
@@ -55,7 +56,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    public void commitTransaction(long txnId) throws ACIDException, RemoteException;
+    public void commitTransaction(JobId jobId) throws ACIDException, RemoteException;
 
     /**
      * Aborts a local transaction against the metadata.
@@ -63,7 +64,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    public void abortTransaction(long txnId) throws ACIDException, RemoteException;
+    public void abortTransaction(JobId jobId) throws ACIDException, RemoteException;
 
     /**
      * Locally locks the entire metadata in given mode on behalf of given
@@ -72,7 +73,7 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    public boolean lock(long txnId, int lockMode) throws ACIDException, RemoteException;
+    public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException;
 
     /**
      * Releases all local locks of given transaction id.
@@ -80,13 +81,13 @@
      * @throws ACIDException
      * @throws RemoteException
      */
-    public boolean unlock(long txnId) throws ACIDException, RemoteException;
+    public void unlock(JobId jobId) throws ACIDException, RemoteException;
 
     /**
      * Inserts a new dataverse into the metadata, acquiring local locks on
      * behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverse
      *            Dataverse instance to be inserted.
@@ -94,13 +95,13 @@
      *             For example, if the dataverse already exists.
      * @throws RemoteException
      */
-    public void addDataverse(long txnId, Dataverse dataverse) throws MetadataException, RemoteException;
+    public void addDataverse(JobId jobId, Dataverse dataverse) throws MetadataException, RemoteException;
 
     /**
      * Retrieves a dataverse with given name, acquiring local locks on behalf of
      * the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the dataverse to retrieve.
@@ -109,13 +110,13 @@
      *             For example, if the dataverse does not exist.
      * @throws RemoteException
      */
-    public Dataverse getDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException;
+    public Dataverse getDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
 
     /**
      * Retrieves all datasets belonging to the given dataverse, acquiring local
      * locks on behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the dataverse of which to find all datasets.
@@ -123,7 +124,7 @@
      * @throws MetadataException
      *             For example, if the dataverse does not exist. RemoteException
      */
-    public List<Dataset> getDataverseDatasets(long txnId, String dataverseName) throws MetadataException,
+    public List<Dataset> getDataverseDatasets(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException;
 
     /**
@@ -131,20 +132,20 @@
      * indexes, and types, acquiring local locks on behalf of the given
      * transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @return A list of dataset instances.
      * @throws MetadataException
      *             For example, if the dataverse does not exist.
      * @throws RemoteException
      */
-    public void dropDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException;
+    public void dropDataverse(JobId jobId, String dataverseName) throws MetadataException, RemoteException;
 
     /**
      * Inserts a new dataset into the metadata, acquiring local locks on behalf
      * of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataset
      *            Dataset instance to be inserted.
@@ -152,13 +153,13 @@
      *             For example, if the dataset already exists.
      * @throws RemoteException
      */
-    public void addDataset(long txnId, Dataset dataset) throws MetadataException, RemoteException;
+    public void addDataset(JobId jobId, Dataset dataset) throws MetadataException, RemoteException;
 
     /**
      * Retrieves a dataset within a given dataverse, acquiring local locks on
      * behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Dataverse name to look for the dataset.
@@ -169,14 +170,14 @@
      *             For example, if the dataset does not exist.
      * @throws RemoteException
      */
-    public Dataset getDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+    public Dataset getDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
             RemoteException;
 
     /**
      * Retrieves all indexes of a dataset, acquiring local locks on behalf of
      * the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse which holds the given dataset.
@@ -187,14 +188,14 @@
      *             For example, if the dataset and/or dataverse does not exist.
      * @throws RemoteException
      */
-    public List<Index> getDatasetIndexes(long txnId, String dataverseName, String datasetName)
+    public List<Index> getDatasetIndexes(JobId jobId, String dataverseName, String datasetName)
             throws MetadataException, RemoteException;
 
     /**
      * Deletes the dataset with given name, and all it's associated indexes,
      * acquiring local locks on behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse which holds the given dataset.
@@ -204,7 +205,7 @@
      *             For example, if the dataset and/or dataverse does not exist.
      * @throws RemoteException
      */
-    public void dropDataset(long txnId, String dataverseName, String datasetName) throws MetadataException,
+    public void dropDataset(JobId jobId, String dataverseName, String datasetName) throws MetadataException,
             RemoteException;
 
     /**
@@ -212,7 +213,7 @@
      * the given transaction id. The index itself knows its name, and which
      * dataset it belongs to.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param index
      *            Index instance to be inserted.
@@ -220,13 +221,13 @@
      *             For example, if the index already exists.
      * @throws RemoteException
      */
-    public void addIndex(long txnId, Index index) throws MetadataException, RemoteException;
+    public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException;
 
     /**
      * Retrieves the index with given name, in given dataverse and dataset,
      * acquiring local locks on behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the datavers holding the given dataset.
@@ -238,14 +239,14 @@
      *             For example, if the index does not exist.
      * @throws RemoteException
      */
-    public Index getIndex(long txnId, String dataverseName, String datasetName, String indexName)
+    public Index getIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
             throws MetadataException, RemoteException;
 
     /**
      * Deletes the index with given name, in given dataverse and dataset,
      * acquiring local locks on behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of the datavers holding the given dataset.
@@ -256,14 +257,14 @@
      *             For example, if the index does not exist.
      * @throws RemoteException
      */
-    public void dropIndex(long txnId, String dataverseName, String datasetName, String indexName)
+    public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
             throws MetadataException, RemoteException;
 
     /**
      * Inserts a datatype, acquiring local locks on behalf of the given
      * transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param datatype
      *            Datatype instance to be inserted.
@@ -271,13 +272,13 @@
      *             For example, if the datatype already exists.
      * @throws RemoteException
      */
-    public void addDatatype(long txnId, Datatype datatype) throws MetadataException, RemoteException;
+    public void addDatatype(JobId jobId, Datatype datatype) throws MetadataException, RemoteException;
 
     /**
      * Retrieves the datatype with given name in given dataverse, acquiring
      * local locks on behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse holding the datatype.
@@ -288,14 +289,14 @@
      *             For example, if the datatype does not exist.
      * @throws RemoteException
      */
-    public Datatype getDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+    public Datatype getDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
             RemoteException;
 
     /**
      * Deletes the given datatype in given dataverse, acquiring local locks on
      * behalf of the given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param dataverseName
      *            Name of dataverse holding the datatype.
@@ -306,14 +307,14 @@
      *             deleted.
      * @throws RemoteException
      */
-    public void dropDatatype(long txnId, String dataverseName, String datatypeName) throws MetadataException,
+    public void dropDatatype(JobId jobId, String dataverseName, String datatypeName) throws MetadataException,
             RemoteException;
 
     /**
      * Inserts a node group, acquiring local locks on behalf of the given
      * transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param nodeGroup
      *            Node group instance to insert.
@@ -321,13 +322,13 @@
      *             For example, if the node group already exists.
      * @throws RemoteException
      */
-    public void addNodeGroup(long txnId, NodeGroup nodeGroup) throws MetadataException, RemoteException;
+    public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException;
 
     /**
      * Retrieves a node group, acquiring local locks on behalf of the given
      * transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param nodeGroupName
      *            Name of node group to be retrieved.
@@ -335,13 +336,13 @@
      *             For example, if the node group does not exist.
      * @throws RemoteException
      */
-    public NodeGroup getNodeGroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException;
+    public NodeGroup getNodeGroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
 
     /**
      * Deletes a node group, acquiring local locks on behalf of the given
      * transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param nodeGroupName
      *            Name of node group to be deleted.
@@ -350,13 +351,13 @@
      *             group to be deleted.
      * @throws RemoteException
      */
-    public void dropNodegroup(long txnId, String nodeGroupName) throws MetadataException, RemoteException;
+    public void dropNodegroup(JobId jobId, String nodeGroupName) throws MetadataException, RemoteException;
 
     /**
      * Inserts a node (compute node), acquiring local locks on behalf of the
      * given transaction id.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param node
      *            Node instance to be inserted.
@@ -364,11 +365,11 @@
      *             For example, if the node already exists.
      * @throws RemoteException
      */
-    public void addNode(long txnId, Node node) throws MetadataException, RemoteException;
+    public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException;
     
     /**
 	 * 
-	 * @param txnId
+	 * @param jobId
 	 *            A globally unique id for an active metadata transaction.
 	 * @param dataverseName
 	 *            dataverse asociated with the function that is to be deleted.
@@ -380,7 +381,7 @@
 	 * @throws MetadataException
 	 * @throws RemoteException
 	 */
-	public Function getFunction(long txnId, String dataverseName,
+	public Function getFunction(JobId jobId, String dataverseName,
 			String functionName, int arity) throws MetadataException,
 			RemoteException;
 
@@ -388,7 +389,7 @@
 	 * Deletes a function , acquiring local locks on behalf of the given
 	 * transaction id.
 	 * 
-	 * @param txnId
+	 * @param jobId
 	 *            A globally unique id for an active metadata transaction.
 	 * @param dataverseName
 	 *            dataverse asociated with the function that is to be deleted.
@@ -401,13 +402,13 @@
 	 *             group to be deleted.
 	 * @throws RemoteException
 	 */
-	public void dropFunction(long txnId, String dataverseName,
+	public void dropFunction(JobId jobId, String dataverseName,
 			String functionName, int arity) throws MetadataException,
 			RemoteException;
 
 	/**
 	 * 
-	 * @param txnId
+	 * @param jobId
 	 *            A globally unique id for an active metadata transaction.
 	 * @param function
 	 *            Function to be inserted
@@ -416,6 +417,8 @@
 	 *             unknown function
 	 * @throws RemoteException
 	 */
-	public void addFunction(long txnId, Function function)
+	public void addFunction(JobId jobId, Function function)
 			throws MetadataException, RemoteException;
+
+    public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
index aec55b5..d8958dd 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IValueExtractor.java
@@ -18,6 +18,7 @@
 import java.io.IOException;
 
 import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -32,7 +33,7 @@
     /**
      * Extracts an object of type T from a given tuple.
      * 
-     * @param txnId
+     * @param jobId
      *            A globally unique transaction id.
      * @param tuple
      *            Tuple from which an object shall be extracted.
@@ -41,5 +42,5 @@
      * @throws HyracksDataException
      * @throws IOException
      */
-    public T getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException;
+    public T getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException;
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 6c388c2..3b5af14 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -52,21 +52,25 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMLeafFrameFactory;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
-import edu.uci.ics.hyracks.storage.am.common.api.IFreePageManager;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexRegistry;
 import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
-import edu.uci.ics.hyracks.storage.am.common.freepage.LinkedListFreePageManager;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.util.LSMBTreeUtils;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.InMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
+import edu.uci.ics.hyracks.storage.common.file.TransientLocalResourceFactoryProvider;
 
 /**
  * Initializes the remote metadata storage facilities ("universe") using a
@@ -78,17 +82,23 @@
  * stopUniverse() should be called upon application undeployment.
  */
 public class MetadataBootstrap {
+    private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
+    private static final int DEFAULT_MEM_PAGE_SIZE = 32768;
+    private static final int DEFAULT_MEM_NUM_PAGES = 100;
+
+    private static AsterixAppRuntimeContext runtimeContext;
+
     private static IBufferCache bufferCache;
     private static IFileMapProvider fileMapProvider;
-    private static IndexRegistry<IIndex> btreeRegistry;
+    private static IIndexLifecycleManager indexLifecycleManager;
+    private static ILocalResourceRepository localResourceRepository;
+    private static IIOManager ioManager;
 
     private static String metadataNodeName;
     private static String metadataStore;
     private static HashSet<String> nodeNames;
     private static String outputDir;
 
-    private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
-
     private static IMetadataIndex[] primaryIndexes;
     private static IMetadataIndex[] secondaryIndexes;
 
@@ -104,8 +114,7 @@
 
     public static void startUniverse(AsterixProperties asterixProperties, INCApplicationContext ncApplicationContext)
             throws Exception {
-        AsterixAppRuntimeContext runtimeContext = (AsterixAppRuntimeContext) ncApplicationContext
-                .getApplicationObject();
+        runtimeContext = (AsterixAppRuntimeContext) ncApplicationContext.getApplicationObject();
 
         // Initialize static metadata objects, such as record types and metadata
         // index descriptors.
@@ -133,31 +142,25 @@
             (new File(outputDir)).mkdirs();
         }
 
-        btreeRegistry = runtimeContext.getIndexRegistry();
+        indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
+        localResourceRepository = runtimeContext.getLocalResourceRepository();
         bufferCache = runtimeContext.getBufferCache();
         fileMapProvider = runtimeContext.getFileMapManager();
-
-        // Create fileRefs to all BTree files and open them in BufferCache.
-        for (int i = 0; i < primaryIndexes.length; i++) {
-            openIndexFile(primaryIndexes[i]);
-        }
-        for (int i = 0; i < secondaryIndexes.length; i++) {
-            openIndexFile(secondaryIndexes[i]);
-        }
+        ioManager = ncApplicationContext.getRootContext().getIOManager();
 
         // Begin a transaction against the metadata.
         // Lock the metadata in X mode.
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.EXCLUSIVE);
+        MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
 
         try {
             if (isNewUniverse) {
                 for (int i = 0; i < primaryIndexes.length; i++) {
-                    createIndex(primaryIndexes[i]);
+                    enlistMetadataDataset(primaryIndexes[i], true);
                     registerTransactionalResource(primaryIndexes[i], resourceRepository);
                 }
                 for (int i = 0; i < secondaryIndexes.length; i++) {
-                    createIndex(secondaryIndexes[i]);
+                    enlistMetadataDataset(secondaryIndexes[i], true);
                     registerTransactionalResource(secondaryIndexes[i], resourceRepository);
                 }
                 insertInitialDataverses(mdTxnCtx);
@@ -169,15 +172,16 @@
                 LOGGER.info("FINISHED CREATING METADATA B-TREES.");
             } else {
                 for (int i = 0; i < primaryIndexes.length; i++) {
-                    enlistMetadataDataset(primaryIndexes[i]);
+                    enlistMetadataDataset(primaryIndexes[i], false);
                     registerTransactionalResource(primaryIndexes[i], resourceRepository);
                 }
                 for (int i = 0; i < secondaryIndexes.length; i++) {
-                    enlistMetadataDataset(secondaryIndexes[i]);
+                    enlistMetadataDataset(secondaryIndexes[i], false);
                     registerTransactionalResource(secondaryIndexes[i], resourceRepository);
                 }
                 LOGGER.info("FINISHED ENLISTMENT OF METADATA B-TREES.");
             }
+            MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
@@ -186,44 +190,26 @@
     }
 
     public static void stopUniverse() throws HyracksDataException {
-        try {
-            // Close all BTree files in BufferCache.
-            for (int i = 0; i < primaryIndexes.length; i++) {
-                bufferCache.closeFile(primaryIndexes[i].getFileId());
-            }
-            for (int i = 0; i < secondaryIndexes.length; i++) {
-                bufferCache.closeFile(secondaryIndexes[i].getFileId());
-            }
-        } catch (HyracksDataException e) {
-            // Ignore for now.
-            // TODO: If multiple NCs are running in the same VM, then we could
-            // have multiple NCs undeploying asterix concurrently.
-            // It would also mean that there is only one BufferCache. A
-            // pathological sequence of events would be that NC2
-            // closes the BufferCache and then NC1 enters this portion of the
-            // code and tries to close unopened files.
-            // What we really want is to check whether the BufferCache is open
-            // in a synchronized block.
-            // The BufferCache api currently does not allow us to check for
-            // openness.
-            // Swallowing the exceptions is a simple fix for now.
+        // Close all BTree files in BufferCache.
+        for (int i = 0; i < primaryIndexes.length; i++) {
+            long resourceID = localResourceRepository
+                    .getResourceByName(primaryIndexes[i].getFile().getFile().getPath()).getResourceId();
+            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.unregister(resourceID);
         }
-    }
-
-    private static void openIndexFile(IMetadataIndex index) throws HyracksDataException, ACIDException {
-        String filePath = metadataStore + index.getFileNameRelativePath();
-        FileReference file = new FileReference(new File(filePath));
-        bufferCache.createFile(file);
-        int fileId = fileMapProvider.lookupFileId(file);
-        bufferCache.openFile(fileId);
-        index.setFileId(fileId);
+        for (int i = 0; i < secondaryIndexes.length; i++) {
+            long resourceID = localResourceRepository.getResourceByName(
+                    secondaryIndexes[i].getFile().getFile().getPath()).getResourceId();
+            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.unregister(resourceID);
+        }
     }
 
     private static void registerTransactionalResource(IMetadataIndex index,
             TransactionalResourceRepository resourceRepository) throws ACIDException {
-        int fileId = index.getFileId();
-        ITreeIndex treeIndex = (ITreeIndex) btreeRegistry.get(fileId);
-        byte[] resourceId = DataUtil.intToByteArray(fileId);
+        long resourceID = index.getResourceID();
+        ITreeIndex treeIndex = (ITreeIndex) indexLifecycleManager.getIndex(resourceID);
+        byte[] resourceId = DataUtil.longToByteArray(resourceID);
         resourceRepository.registerTransactionalResource(resourceId, treeIndex);
         index.initTreeLogger(treeIndex);
     }
@@ -241,7 +227,7 @@
                     primaryIndexes[i].getNodeGroupName());
             MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(primaryIndexes[i].getDataverseName(),
                     primaryIndexes[i].getIndexedDatasetName(), primaryIndexes[i].getPayloadRecordType().getTypeName(),
-                    id, DatasetType.INTERNAL));
+                    id, DatasetType.INTERNAL, primaryIndexes[i].getDatasetId().getId()));
         }
     }
 
@@ -304,33 +290,38 @@
 
     }
 
-    public static void createIndex(IMetadataIndex dataset) throws Exception {
-        int fileId = dataset.getFileId();
-        ITypeTraits[] typeTraits = dataset.getTypeTraits();
-        IBinaryComparatorFactory[] comparatorFactories = dataset.getKeyBinaryComparatorFactory();
-        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
+    public static void enlistMetadataDataset(IMetadataIndex index, boolean create) throws Exception {
+        String filePath = metadataStore + index.getFileNameRelativePath();
+        FileReference file = new FileReference(new File(filePath));
+        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), DEFAULT_MEM_PAGE_SIZE,
+                DEFAULT_MEM_NUM_PAGES, new TransientFileMapManager());
+        ITypeTraits[] typeTraits = index.getTypeTraits();
+        IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
         ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaDataFrameFactory);
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, typeTraits.length, comparatorFactories,
-                freePageManager, interiorFrameFactory, leafFrameFactory);
-        btree.create(fileId);
-        btreeRegistry.register(fileId, btree);
-    }
-
-    public static void enlistMetadataDataset(IMetadataIndex dataset) throws Exception {
-        int fileId = dataset.getFileId();
-        ITypeTraits[] typeTraits = dataset.getTypeTraits();
-        IBinaryComparatorFactory[] comparatorFactories = dataset.getKeyBinaryComparatorFactory();
-        TypeAwareTupleWriterFactory tupleWriterFactory = new TypeAwareTupleWriterFactory(typeTraits);
-        ITreeIndexFrameFactory leafFrameFactory = new BTreeNSMLeafFrameFactory(tupleWriterFactory);
-        ITreeIndexFrameFactory interiorFrameFactory = new BTreeNSMInteriorFrameFactory(tupleWriterFactory);
-        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
-        IFreePageManager freePageManager = new LinkedListFreePageManager(bufferCache, 0, metaDataFrameFactory);
-        BTree btree = new BTree(bufferCache, NoOpOperationCallback.INSTANCE, typeTraits.length, comparatorFactories,
-                freePageManager, interiorFrameFactory, leafFrameFactory);
-        btreeRegistry.register(fileId, btree);
+        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(DEFAULT_MEM_NUM_PAGES,
+                metaDataFrameFactory);
+        LSMBTree lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
+                bufferCache, fileMapProvider, typeTraits, comparatorFactories, runtimeContext.getFlushController(),
+                runtimeContext.getLSMMergePolicy(), runtimeContext.getLSMOperationTracker(),
+                runtimeContext.getLSMIOScheduler());
+        long resourceID = -1;
+        if (create) {
+            lsmBtree.create();
+            resourceID = runtimeContext.getResourceIdFactory().createId();
+            //resourceID = indexArtifactMap.create(file.getFile().getPath(), ioManager.getIODevices());
+            //TODO
+            //replace the transient resource factory provider with the persistent one.
+            ILocalResourceFactoryProvider localResourceFactoryProvider = new TransientLocalResourceFactoryProvider();
+            ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
+            localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
+                    .getPath()));
+        } else {
+            resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
+        }
+        index.setResourceID(resourceID);
+        index.setFile(file);
+        indexLifecycleManager.register(resourceID, lsmBtree);
+        indexLifecycleManager.open(resourceID);
     }
 
     public static String getOutputDir() {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
index 165a605..d1e377e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataIndex.java
@@ -30,12 +30,13 @@
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.service.logging.DataUtil;
 import edu.uci.ics.asterix.transaction.management.service.logging.TreeLogger;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 
 /**
@@ -62,15 +63,20 @@
     protected final IBinaryComparatorFactory[] bcfs;
     // Hash function factories for key fields of btree tuple. Created in c'tor.
     protected final IBinaryHashFunctionFactory[] bhffs;
+
+    protected FileReference file;
     // Identifier of file BufferCache backing this metadata btree index.
     protected int fileId;
+    protected long resourceID;
     // Resource id of this index for use in transactions.
     protected byte[] indexResourceId;
     // Logger for tree indexes.
     private TreeLogger treeLogger;
+    // datasetId
+    private final DatasetId datasetId;
 
     public MetadataIndex(String datasetName, String indexName, int numFields, IAType[] keyTypes, String[] keyNames,
-            ARecordType payloadType) throws AsterixRuntimeException {
+            ARecordType payloadType, int datasetId) throws AsterixRuntimeException {
         // Sanity checks.
         if (keyTypes.length != keyNames.length) {
             throw new AsterixRuntimeException("Unequal number of key types and names given.");
@@ -116,14 +122,15 @@
         // Create binary comparator factories.
         bcfs = new IBinaryComparatorFactory[keyTypes.length];
         for (int i = 0; i < keyTypes.length; i++) {
-            bcfs[i] = AqlBinaryComparatorFactoryProvider.INSTANCE
-                    .getBinaryComparatorFactory(keyTypes[i], true);
+            bcfs[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyTypes[i], true);
         }
         // Create binary hash function factories.
         bhffs = new IBinaryHashFunctionFactory[keyTypes.length];
         for (int i = 0; i < keyTypes.length; i++) {
             bhffs[i] = AqlBinaryHashFunctionFactoryProvider.INSTANCE.getBinaryHashFunctionFactory(keyTypes[i]);
         }
+
+        this.datasetId = new DatasetId(datasetId);
     }
 
     @Override
@@ -224,4 +231,30 @@
     public TreeLogger getTreeLogger() {
         return treeLogger;
     }
+
+    @Override
+    public void setFile(FileReference file) {
+        this.file = file;
+    }
+
+    @Override
+    public FileReference getFile() {
+        return this.file;
+    }
+
+    @Override
+    public void setResourceID(long resourceID) {
+        this.resourceID = resourceID;
+        this.indexResourceId = DataUtil.longToByteArray(resourceID);
+    }
+
+    @Override
+    public long getResourceID() {
+        return resourceID;
+    }
+
+    @Override
+    public DatasetId getDatasetId() {
+        return datasetId;
+    }
 }
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
index 2a40e73..a81b73e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataPrimaryIndexes.java
@@ -32,6 +32,16 @@
     public static IMetadataIndex NODE_DATASET;
     public static IMetadataIndex NODEGROUP_DATASET;
     public static IMetadataIndex FUNCTION_DATASET;
+    
+    public static final int METADATA_DATASET_ID = 0;
+    public static final int DATAVERSE_DATASET_ID = 1;
+    public static final int DATASET_DATASET_ID = 2;
+    public static final int DATATYPE_DATASET_ID = 3;
+    public static final int INDEX_DATASET_ID = 4;
+    public static final int NODE_DATASET_ID = 5;
+    public static final int NODEGROUP_DATASET_ID = 6;
+    public static final int FUNCTION_DATASET_ID = 7;
+    public static final int FIRST_AVAILABLE_USER_DATASET_ID = 8;
 
 
     /**
@@ -49,30 +59,30 @@
         }
 
         DATAVERSE_DATASET = new MetadataIndex("Dataverse", null, 2, new IAType[] { BuiltinType.ASTRING },
-                new String[] { "DataverseName" }, MetadataRecordTypes.DATAVERSE_RECORDTYPE);
+                new String[] { "DataverseName" }, MetadataRecordTypes.DATAVERSE_RECORDTYPE, DATAVERSE_DATASET_ID);
 
         DATASET_DATASET = new MetadataIndex("Dataset", null, 3,
                 new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
-                        "DatasetName" }, MetadataRecordTypes.DATASET_RECORDTYPE);
+                        "DatasetName" }, MetadataRecordTypes.DATASET_RECORDTYPE, DATASET_DATASET_ID);
 
         DATATYPE_DATASET = new MetadataIndex("Datatype", null, 3, new IAType[] { BuiltinType.ASTRING,
                 BuiltinType.ASTRING }, new String[] { "DataverseName", "DatatypeName" },
-                MetadataRecordTypes.DATATYPE_RECORDTYPE);
+                MetadataRecordTypes.DATATYPE_RECORDTYPE, DATATYPE_DATASET_ID);
 
         INDEX_DATASET = new MetadataIndex("Index", null, 4, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
                 BuiltinType.ASTRING }, new String[] { "DataverseName", "DatasetName", "IndexName" },
-                MetadataRecordTypes.INDEX_RECORDTYPE);
+                MetadataRecordTypes.INDEX_RECORDTYPE, INDEX_DATASET_ID);
 
         NODE_DATASET = new MetadataIndex("Node", null, 2, new IAType[] { BuiltinType.ASTRING },
-                new String[] { "NodeName" }, MetadataRecordTypes.NODE_RECORDTYPE);
+                new String[] { "NodeName" }, MetadataRecordTypes.NODE_RECORDTYPE, NODE_DATASET_ID);
 
         NODEGROUP_DATASET = new MetadataIndex("Nodegroup", null, 2, new IAType[] { BuiltinType.ASTRING },
-                new String[] { "GroupName" }, MetadataRecordTypes.NODEGROUP_RECORDTYPE);
+                new String[] { "GroupName" }, MetadataRecordTypes.NODEGROUP_RECORDTYPE, NODEGROUP_DATASET_ID);
         
         FUNCTION_DATASET = new MetadataIndex("Function", null, 4,
 				new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
 						BuiltinType.ASTRING }, new String[] { "DataverseName",
-                        "FunctionName", "FunctionArity" }, MetadataRecordTypes.FUNCTION_RECORDTYPE);
+                        "FunctionName", "FunctionArity" }, MetadataRecordTypes.FUNCTION_RECORDTYPE, FUNCTION_DATASET_ID);
 
     }
 }
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
index 6936d0d..52e69a0 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -158,6 +158,7 @@
 
 	// Helper constants for accessing fields in an ARecord of type
 	// DatasetRecordType.
+	
 	public static final int DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
 	public static final int DATASET_ARECORD_DATASETNAME_FIELD_INDEX = 1;
 	public static final int DATASET_ARECORD_DATATYPENAME_FIELD_INDEX = 2;
@@ -166,11 +167,12 @@
 	public static final int DATASET_ARECORD_EXTERNALDETAILS_FIELD_INDEX = 5;
 	public static final int DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX = 6;
 	public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 7;
+	public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 8;
 
 	private static final ARecordType createDatasetRecordType() {
 		String[] fieldNames = { "DataverseName", "DatasetName", "DataTypeName",
 				"DatasetType", "InternalDetails", "ExternalDetails",
-				"FeedDetails", "Timestamp" };
+				"FeedDetails", "Timestamp", "DatasetId" };
 
 		List<IAType> internalRecordUnionList = new ArrayList<IAType>();
 		internalRecordUnionList.add(BuiltinType.ANULL);
@@ -191,7 +193,7 @@
 
 		IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING,
 				BuiltinType.ASTRING, BuiltinType.ASTRING, internalRecordUnion,
-				externalRecordUnion, feedRecordUnion, BuiltinType.ASTRING };
+				externalRecordUnion, feedRecordUnion, BuiltinType.ASTRING, BuiltinType.AINT32 };
 		return new ARecordType("DatasetRecordType", fieldNames, fieldTypes,
 				true);
 	}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
index a23e7d7..f696714 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataSecondaryIndexes.java
@@ -44,14 +44,14 @@
 
         GROUPNAME_ON_DATASET_INDEX = new MetadataIndex("Dataset", "GroupName", 3, new IAType[] { BuiltinType.ASTRING,
                 BuiltinType.ASTRING, BuiltinType.ASTRING },
-                new String[] { "GroupName", "DataverseName", "DatasetName" }, null);
+                new String[] { "GroupName", "DataverseName", "DatasetName" }, null, MetadataPrimaryIndexes.DATASET_DATASET_ID);
 
         DATATYPENAME_ON_DATASET_INDEX = new MetadataIndex("Dataset", "DatatypeName", 3, new IAType[] {
                 BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
-                "DatatypeName", "DatasetName" }, null);
+                "DatatypeName", "DatasetName" }, null, MetadataPrimaryIndexes.DATASET_DATASET_ID);
 
         DATATYPENAME_ON_DATATYPE_INDEX = new MetadataIndex("Datatype", "DatatypeName", 3, new IAType[] {
                 BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
-                "NestedDatatypeName", "TopDatatypeName" }, null);
+                "NestedDatatypeName", "TopDatatypeName" }, null, MetadataPrimaryIndexes.DATATYPE_DATASET_ID);
     }
 }
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index ba7c797..6973f28 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -163,10 +163,6 @@
         return ng.getNodeNames();
     }
 
-    public String[] getStores(String nodeName) {
-        return stores.get(nodeName);
-    }
-
     public Map<String, String[]> getAllStores() {
         return stores;
     }
@@ -180,13 +176,13 @@
     }
 
     public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
-    	try {
+        try {
             return metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
         } catch (MetadataException e) {
             throw new AlgebricksException(e);
         }
     }
-    
+
     public Index getDatasetPrimaryIndex(String dataverseName, String datasetName) throws AlgebricksException {
         try {
             return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
@@ -202,7 +198,7 @@
             throw new AlgebricksException(e);
         }
     }
-    
+
     public void setOutputFile(FileSplit outputFile) {
         this.outputFile = outputFile;
     }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index b8f0258..6d96d91 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -20,6 +20,7 @@
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
 import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
@@ -45,7 +46,7 @@
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.runtime.base.AsterixTupleFilterFactory;
-import edu.uci.ics.asterix.runtime.transaction.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -69,6 +70,7 @@
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ILinearizeComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
@@ -78,25 +80,30 @@
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.frames.BTreeNSMInteriorFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexBulkLoadOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.dataflow.TreeIndexInsertUpdateDeleteOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.common.tuples.TypeAwareTupleWriterFactory;
-import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.dataflow.LSMRTreeDataflowHelperFactory;
 import edu.uci.ics.hyracks.storage.am.rtree.dataflow.RTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.HilbertDoubleComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.ZCurveDoubleComparatorFactory;
+import edu.uci.ics.hyracks.storage.am.rtree.linearize.ZCurveIntComparatorFactory;
 
 public class AqlMetadataProvider implements IMetadataProvider<AqlSourceId, String> {
-    private final long txnId;
+    private final JobId jobId;
     private boolean isWriteTransaction;
     private final AqlCompiledMetadataDeclarations metadata;
 
-    public AqlMetadataProvider(long txnId, boolean isWriteTransaction, AqlCompiledMetadataDeclarations metadata) {
-        this.txnId = txnId;
+    public AqlMetadataProvider(JobId jobId, boolean isWriteTransaction, AqlCompiledMetadataDeclarations metadata) {
+        this.jobId = jobId;
         this.isWriteTransaction = isWriteTransaction;
         this.metadata = metadata;
     }
@@ -299,12 +306,10 @@
                 outputVars, keysStartIndex, numKeys, typeEnv, context);
         ITypeTraits[] typeTraits = null;
 
-	if (isSecondary) {
-        	typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv,
-                    context);
+        if (isSecondary) {
+            typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
         } else {
-        	typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
-                    context);
+            typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv, context);
         }
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
@@ -315,9 +320,11 @@
             throw new AlgebricksException(e);
         }
         BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
-                appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first, typeTraits,
-                comparatorFactories, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
-                new BTreeDataflowHelperFactory(), retainInput, NoOpOperationCallbackProvider.INSTANCE);
+                appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+                typeTraits, comparatorFactories, lowKeyFields, highKeyFields, lowKeyInclusive, highKeyInclusive,
+                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE), retainInput, NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
     }
 
@@ -332,6 +339,8 @@
             isSecondary = !indexName.equals(primaryIndex.getIndexName());
         }
 
+        IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+                dataset, recType, context.getBinaryComparatorFactoryProvider());
         int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         ISerializerDeserializer[] recordFields;
         IBinaryComparatorFactory[] comparatorFactories;
@@ -395,9 +404,12 @@
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         RTreeSearchOperatorDescriptor rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, recDesc,
-                appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first, typeTraits,
-                comparatorFactories, keyFields, new RTreeDataflowHelperFactory(valueProviderFactories), false,
-                NoOpOperationCallbackProvider.INSTANCE);
+                appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+                typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(valueProviderFactories,
+                        RTreePolicyType.RTREE, primaryComparatorFactories, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
+                                comparatorFactories.length)), false, NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
     }
 
@@ -504,14 +516,16 @@
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
+                appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
                 splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
-                GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
+                GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
                 NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOp indexOp,
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOperation indexOp,
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
             throws AlgebricksException {
@@ -545,9 +559,12 @@
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
+                spec, recordDesc, appContext.getStorageManagerInterface(),
+                appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                comparatorFactories, fieldPermutation, indexOp, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE), null,
+                NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
@@ -556,7 +573,7 @@
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
             throws AlgebricksException {
-        return getInsertOrDeleteRuntime(IndexOp.INSERT, dataSource, propagatedSchema, keys, payload, recordDesc,
+        return getInsertOrDeleteRuntime(IndexOperation.INSERT, dataSource, propagatedSchema, keys, payload, recordDesc,
                 context, spec);
     }
 
@@ -565,11 +582,11 @@
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
             throws AlgebricksException {
-        return getInsertOrDeleteRuntime(IndexOp.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
+        return getInsertOrDeleteRuntime(IndexOperation.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
                 context, spec);
     }
 
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOp indexOp,
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOperation indexOp,
             IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
@@ -604,7 +621,7 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec) throws AlgebricksException {
-        return getIndexInsertOrDeleteRuntime(IndexOp.INSERT, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+        return getIndexInsertOrDeleteRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
                 primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
@@ -614,7 +631,7 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec) throws AlgebricksException {
-        return getIndexInsertOrDeleteRuntime(IndexOp.DELETE, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+        return getIndexInsertOrDeleteRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
                 primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
@@ -634,7 +651,7 @@
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeDmlRuntime(String datasetName,
             String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+            JobGenContext context, JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
         int numKeys = primaryKeys.size() + secondaryKeys.size();
         // generate field permutations
         int[] fieldPermutation = new int[numKeys];
@@ -687,16 +704,19 @@
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
-                new BTreeDataflowHelperFactory(), filterFactory, NoOpOperationCallbackProvider.INSTANCE, txnId);
+                spec, recordDesc, appContext.getStorageManagerInterface(),
+                appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                comparatorFactories, fieldPermutation, indexOp, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE),
+                filterFactory, NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeDmlRuntime(String datasetName,
             String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+            JobGenContext context, JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
         Dataset dataset = metadata.findDataset(datasetName);
         String itemTypeName = dataset.getItemTypeName();
         IAType itemType = metadata.findType(itemTypeName);
@@ -743,26 +763,45 @@
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
             ++i;
         }
-
+        IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+                dataset, recType, context.getBinaryComparatorFactoryProvider());
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
-                new RTreeDataflowHelperFactory(valueProviderFactories), filterFactory,
-                NoOpOperationCallbackProvider.INSTANCE, txnId);
+                spec, recordDesc, appContext.getStorageManagerInterface(),
+                appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                comparatorFactories, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
+                        valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        AsterixRuntimeComponentsProvider.INSTANCE, AsterixRuntimeComponentsProvider.INSTANCE,
+                        proposeLinearizer(nestedKeyType.getTypeTag(), comparatorFactories.length)), filterFactory,
+                NoOpOperationCallbackProvider.INSTANCE);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
     }
 
-    public long getTxnId() {
-        return txnId;
+    public JobId getJobId() {
+        return jobId;
     }
 
     public static ITreeIndexFrameFactory createBTreeNSMInteriorFrameFactory(ITypeTraits[] typeTraits) {
         return new BTreeNSMInteriorFrameFactory(new TypeAwareTupleWriterFactory(typeTraits));
     }
 
+    public static ILinearizeComparatorFactory proposeLinearizer(ATypeTag keyType, int numKeyFields)
+            throws AlgebricksException {
+        if (numKeyFields / 2 == 2 && (keyType == ATypeTag.DOUBLE)) {
+            return new HilbertDoubleComparatorFactory(2);
+        } else if (keyType == ATypeTag.DOUBLE) {
+            return new ZCurveDoubleComparatorFactory(numKeyFields / 2);
+        } else if (keyType == ATypeTag.INT8 || keyType == ATypeTag.INT16 || keyType == ATypeTag.INT32
+                || keyType == ATypeTag.INT64) {
+            return new ZCurveIntComparatorFactory(numKeyFields / 2);
+        } else {
+            throw new AlgebricksException("Cannot propose linearizer for key with type " + keyType + ".");
+        }
+    }
+
     @Override
     public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
         return AsterixBuiltinFunctions.lookupFunction(fid);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
index d383955..63e837b 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
@@ -34,14 +34,16 @@
     private final String itemTypeName;
     private final DatasetType datasetType;
     private IDatasetDetails datasetDetails;
+    private final int datasetId;
 
     public Dataset(String dataverseName, String datasetName, String itemTypeName, IDatasetDetails datasetDetails,
-            DatasetType datasetType) {
+            DatasetType datasetType, int datasetId) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.itemTypeName = itemTypeName;
         this.datasetType = datasetType;
         this.datasetDetails = datasetDetails;
+        this.datasetId = datasetId;
     }
 
     public String getDataverseName() {
@@ -68,6 +70,10 @@
         this.datasetDetails = datasetDetails;
     }
 
+    public int getDatasetId() {
+        return datasetId;
+    }
+
     @Override
     public Object addToCache(MetadataCache cache) {
         return cache.addDatasetIfNotExists(this);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
index fbec4b1..e00e9e0 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -40,10 +40,14 @@
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.FileStructure;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails.PartitioningStrategy;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableString;
 import edu.uci.ics.asterix.om.base.AOrderedList;
 import edu.uci.ics.asterix.om.base.ARecord;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.base.IACursor;
+import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
@@ -68,9 +72,14 @@
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(MetadataRecordTypes.DATASET_RECORDTYPE);
+    private AMutableInt32 aInt32;
+    protected ISerializerDeserializer<AInt32> aInt32Serde;
 
-    public DatasetTupleTranslator(boolean getTuple) {
+    @SuppressWarnings("unchecked")
+	public DatasetTupleTranslator(boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.DATASET_DATASET.getFieldCount());
+        aInt32 = new AMutableInt32(-1); 
+        aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
     }
 
     @Override
@@ -85,6 +94,7 @@
     }
 
     private Dataset createDatasetFromARecord(ARecord datasetRecord) {
+
         String dataverseName = ((AString) datasetRecord
                 .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
         String datasetName = ((AString) datasetRecord
@@ -93,6 +103,8 @@
                 .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATATYPENAME_FIELD_INDEX)).getStringValue();
         DatasetType datasetType = DatasetType.valueOf(((AString) datasetRecord.getValueByPos(3)).getStringValue());
         IDatasetDetails datasetDetails = null;
+    	int datasetId = ((AInt32) datasetRecord
+                .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATASETID_FIELD_INDEX)).getIntegerValue();
         switch (datasetType) {
             case FEED:
             case INTERNAL: {
@@ -177,7 +189,7 @@
                 }
                 datasetDetails = new ExternalDatasetDetails(adapter, properties);
         }
-        return new Dataset(dataverseName, datasetName, typeName, datasetDetails, datasetType);
+        return new Dataset(dataverseName, datasetName, typeName, datasetDetails, datasetType, datasetId);
     }
 
     @Override
@@ -228,7 +240,13 @@
         aString.setValue(Calendar.getInstance().getTime().toString());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
-
+        
+        // write field 8
+        fieldValue.reset();
+        aInt32.setValue(dataset.getDatasetId());
+        aInt32Serde.serialize(aInt32, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATASETID_FIELD_INDEX, fieldValue);
+        
         // write record
         recordBuilder.write(tupleBuilder.getDataOutput(), true);
         tupleBuilder.addFieldEndOffset();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
index b39a1bf..3931a17 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatatypeTupleTranslator.java
@@ -49,6 +49,7 @@
 import edu.uci.ics.asterix.om.types.AbstractCollectionType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -81,11 +82,11 @@
     private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(MetadataRecordTypes.DATATYPE_RECORDTYPE);
     private final MetadataNode metadataNode;
-    private final long txnId;
+    private final JobId jobId;
 
-    public DatatypeTupleTranslator(long txnId, MetadataNode metadataNode, boolean getTuple) {
+    public DatatypeTupleTranslator(JobId jobId, MetadataNode metadataNode, boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.DATATYPE_DATASET.getFieldCount());
-        this.txnId = txnId;
+        this.jobId = jobId;
         this.metadataNode = metadataNode;
     }
 
@@ -182,7 +183,7 @@
         IAType type = AsterixBuiltinTypeMap.getBuiltinTypes().get(typeName);
         if (type == null) {
             try {
-                return metadataNode.getDatatype(txnId, dataverseName, typeName).getDatatype();
+                return metadataNode.getDatatype(jobId, dataverseName, typeName).getDatatype();
             } catch (RemoteException e) {
                 throw new MetadataException(e);
             }
@@ -417,9 +418,9 @@
         MetadataNode mn = MetadataNode.INSTANCE;
         if (typeName == null) {
             typeName = suggestedTypeName;
-            metadataNode.addDatatype(txnId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType, true));
+            metadataNode.addDatatype(jobId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType, true));
             try {
-                mn.insertIntoDatatypeSecondaryIndex(txnId, topLevelType.getDataverseName(), typeName,
+                mn.insertIntoDatatypeSecondaryIndex(jobId, topLevelType.getDataverseName(), typeName,
                         topLevelType.getDatatypeName());
             } catch (BTreeDuplicateKeyException e) {
                 // The key may have been inserted by a previous DDL statement or
@@ -427,12 +428,12 @@
             }
             return typeName;
         }
-        Datatype dt = mn.getDatatype(txnId, topLevelType.getDataverseName(), typeName);
+        Datatype dt = mn.getDatatype(jobId, topLevelType.getDataverseName(), typeName);
         if (dt == null) {
             throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
         }
         try {
-            mn.insertIntoDatatypeSecondaryIndex(txnId, topLevelType.getDataverseName(), typeName,
+            mn.insertIntoDatatypeSecondaryIndex(jobId, topLevelType.getDataverseName(), typeName,
                     topLevelType.getDatatypeName());
         } catch (BTreeDuplicateKeyException e) {
             // The key may have been inserted by a previous DDL statement or by
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
index a6bbb8f..e554643 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatasetNameValueExtractor.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
 import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -32,7 +33,7 @@
  */
 public class DatasetNameValueExtractor implements IValueExtractor<String> {
     @Override
-    public String getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+    public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
         byte[] serRecord = tuple.getFieldData(2);
         int recordStartOffset = tuple.getFieldStart(2);
         int recordLength = tuple.getFieldLength(2);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
index 16a501d..2e4dbe4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/DatatypeNameValueExtractor.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.asterix.metadata.MetadataNode;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
 import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -42,7 +43,7 @@
     }
 
     @Override
-    public String getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+    public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
         byte[] serRecord = tuple.getFieldData(2);
         int recordStartOffset = tuple.getFieldStart(2);
         int recordLength = tuple.getFieldLength(2);
@@ -50,10 +51,10 @@
         DataInput in = new DataInputStream(stream);
         String typeName = ((AString) AObjectSerializerDeserializer.INSTANCE.deserialize(in)).getStringValue();
         try {
-            if (metadataNode.getDatatype(txnId, dataverseName, typeName).getIsAnonymous()) {
+            if (metadataNode.getDatatype(jobId, dataverseName, typeName).getIsAnonymous()) {
                 // Get index 0 because it is anonymous type, and it is used in
                 // only one non-anonymous type.
-                typeName = metadataNode.getDatatypeNamesUsingThisDatatype(txnId, dataverseName, typeName).get(0);
+                typeName = metadataNode.getDatatypeNamesUsingThisDatatype(jobId, dataverseName, typeName).get(0);
             }
         } catch (RemoteException e) {
             throw new MetadataException(e);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
index 34f9b29..7ae334e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/MetadataEntityValueExtractor.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IMetadataEntityTupleTranslator;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -34,7 +35,7 @@
     }
 
     @Override
-    public T getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException {
+    public T getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException, IOException {
         return tupleReaderWriter.getMetadataEntytiFromTuple(tuple);
     }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
index 74dabad..d4109cd 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/NestedDatatypeNameValueExtractor.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
 import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -40,7 +41,7 @@
     }
 
     @Override
-    public String getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException {
+    public String getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException {
         byte[] serRecord = tuple.getFieldData(2);
         int recordStartOffset = tuple.getFieldStart(2);
         int recordLength = tuple.getFieldLength(2);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
index da608e5..03bc963 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/valueextractors/TupleCopyValueExtractor.java
@@ -20,6 +20,7 @@
 
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -44,7 +45,7 @@
     }
 
     @Override
-    public ITupleReference getValue(long txnId, ITupleReference tuple) throws MetadataException, HyracksDataException,
+    public ITupleReference getValue(JobId jobId, ITupleReference tuple) throws MetadataException, HyracksDataException,
             IOException {
         int numBytes = tupleWriter.bytesRequired(tuple);
         tupleBytes = new byte[numBytes];
diff --git a/asterix-om/pom.xml b/asterix-om/pom.xml
index a14f3c2..ff9e143 100644
--- a/asterix-om/pom.xml
+++ b/asterix-om/pom.xml
@@ -1,4 +1,5 @@
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+	xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 	<modelVersion>4.0.0</modelVersion>
 	<parent>
 		<artifactId>asterix</artifactId>
@@ -32,23 +33,19 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-invertedindex</artifactId>
-			<version>0.2.2-SNAPSHOT</version>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.asterix</groupId>
-			<artifactId>asterix-hyracks-glue</artifactId>
-			<version>0.0.4-SNAPSHOT</version>
-			<scope>compile</scope>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-algebricks-compiler</artifactId>
 			<version>0.2.2-SNAPSHOT</version>
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-rtree</artifactId>
+			<artifactId>
+				hyracks-storage-am-lsm-invertedindex
+			</artifactId>
+			<version>0.2.2-SNAPSHOT</version>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
 			<version>0.2.2-SNAPSHOT</version>
 		</dependency>
 	</dependencies>
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
index cb6838e..82780bb 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementToken.java
@@ -3,7 +3,7 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 
 public class AListElementToken implements IToken {
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
index fc46039..1ccf2db 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AListElementTokenFactory.java
@@ -1,7 +1,7 @@
 package edu.uci.ics.asterix.dataflow.data.common;
 
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
 public class AListElementTokenFactory implements ITokenFactory {
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
index 667bfe7..89a3ff7 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizer.java
@@ -5,9 +5,9 @@
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.EnumDeserializer;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
 public class AOrderedListBinaryTokenizer implements IBinaryTokenizer {
 
@@ -16,13 +16,13 @@
     protected int length;
     protected int listLength;
     protected int itemIndex;
-    
+
     protected final IToken token;
-    
+
     public AOrderedListBinaryTokenizer(ITokenFactory tokenFactory) {
         token = tokenFactory.createToken();
     }
-    
+
     @Override
     public IToken getToken() {
         return token;
@@ -58,11 +58,11 @@
         this.listLength = getNumberOfItems(data, start);
         this.itemIndex = 0;
     }
-    
+
     protected int getItemOffset(byte[] data, int start, int itemIndex) throws AsterixException {
         return AOrderedListSerializerDeserializer.getItemOffset(data, start, itemIndex);
     }
-    
+
     protected int getNumberOfItems(byte[] data, int start) {
         return AOrderedListSerializerDeserializer.getNumberOfItems(data, start);
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
index d7712ea..6c5077b 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AOrderedListBinaryTokenizerFactory.java
@@ -1,18 +1,18 @@
 package edu.uci.ics.asterix.dataflow.data.common;
 
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
 public class AOrderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
 
     private static final long serialVersionUID = 1L;
     private final ITokenFactory tokenFactory;
-    
+
     public AOrderedListBinaryTokenizerFactory(ITokenFactory tokenFactory) {
         this.tokenFactory = tokenFactory;
     }
-    
+
     @Override
     public IBinaryTokenizer createTokenizer() {
         return new AOrderedListBinaryTokenizer(tokenFactory);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
index 11ab251..7e0ac32 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizer.java
@@ -2,14 +2,14 @@
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
 public class AUnorderedListBinaryTokenizer extends AOrderedListBinaryTokenizer {
 
     public AUnorderedListBinaryTokenizer(ITokenFactory tokenFactory) {
         super(tokenFactory);
     }
-    
+
     @Override
     protected int getItemOffset(byte[] data, int start, int itemIndex) throws AsterixException {
         return AUnorderedListSerializerDeserializer.getItemOffset(data, start, itemIndex);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
index 54b7692..fc55aa6 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/AUnorderedListBinaryTokenizerFactory.java
@@ -1,18 +1,18 @@
 package edu.uci.ics.asterix.dataflow.data.common;
 
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
 public class AUnorderedListBinaryTokenizerFactory implements IBinaryTokenizerFactory {
-    
+
     private static final long serialVersionUID = 1L;
     private final ITokenFactory tokenFactory;
-    
+
     public AUnorderedListBinaryTokenizerFactory(ITokenFactory tokenFactory) {
         this.tokenFactory = tokenFactory;
     }
-    
+
     @Override
     public IBinaryTokenizer createTokenizer() {
         return new AUnorderedListBinaryTokenizer(tokenFactory);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
index cddb014..b9af9bf 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/IBinaryTokenizerFactoryProvider.java
@@ -1,9 +1,11 @@
 package edu.uci.ics.asterix.dataflow.data.common;
 
 import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public interface IBinaryTokenizerFactoryProvider {
     public IBinaryTokenizerFactory getWordTokenizerFactory(ATypeTag typeTag, boolean hashedTokens);
-    public IBinaryTokenizerFactory getNGramTokenizerFactory(ATypeTag typeTag, int gramLength, boolean usePrePost, boolean hashedTokens);
+
+    public IBinaryTokenizerFactory getNGramTokenizerFactory(ATypeTag typeTag, int gramLength, boolean usePrePost,
+            boolean hashedTokens);
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
index 2a7fdb5..3d66912 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
@@ -1,10 +1,6 @@
 package edu.uci.ics.asterix.dataflow.data.common;
 
-import java.util.Collections;
-import java.util.List;
-
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedListCursor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
 
 // TODO: Should go into hyracks.
 public class ListEditDistanceSearchModifier implements IInvertedIndexSearchModifier {
@@ -15,17 +11,6 @@
         this.edThresh = edThresh;
     }
 
-    @Override
-    public int getOccurrenceThreshold(List<IInvertedListCursor> invListCursors) {
-        return invListCursors.size() - edThresh;
-    }
-
-    @Override
-    public int getPrefixLists(List<IInvertedListCursor> invListCursors) {
-        Collections.sort(invListCursors);
-        return invListCursors.size() - getOccurrenceThreshold(invListCursors) + 1;
-    }
-
     public int getEdThresh() {
         return edThresh;
     }
@@ -33,4 +18,14 @@
     public void setEdThresh(int edThresh) {
         this.edThresh = edThresh;
     }
+
+    @Override
+    public int getOccurrenceThreshold(int numQueryTokens) {
+        return numQueryTokens - edThresh;
+    }
+
+    @Override
+    public int getNumPrefixLists(int numQueryTokens) {
+        return numQueryTokens - getOccurrenceThreshold(numQueryTokens) + 1;
+    }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
index b117a36..4148c85 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
@@ -1,7 +1,7 @@
 package edu.uci.ics.asterix.dataflow.data.common;
 
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifier;
-import edu.uci.ics.hyracks.storage.am.invertedindex.api.IInvertedIndexSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifier;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.api.IInvertedIndexSearchModifierFactory;
 
 // TODO: Should go into hyracks.
 public class ListEditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
@@ -9,11 +9,11 @@
     private static final long serialVersionUID = 1L;
 
     private final int edThresh;
-    
+
     public ListEditDistanceSearchModifierFactory(int edThresh) {
         this.edThresh = edThresh;
     }
-    
+
     @Override
     public IInvertedIndexSearchModifier createSearchModifier() {
         return new ListEditDistanceSearchModifier(edThresh);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
index 859b0f2..41a921e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryTokenizerFactoryProvider.java
@@ -1,15 +1,15 @@
 package edu.uci.ics.asterix.formats.nontagged;
 
-import edu.uci.ics.asterix.dataflow.data.common.IBinaryTokenizerFactoryProvider;
 import edu.uci.ics.asterix.dataflow.data.common.AListElementTokenFactory;
 import edu.uci.ics.asterix.dataflow.data.common.AOrderedListBinaryTokenizerFactory;
 import edu.uci.ics.asterix.dataflow.data.common.AUnorderedListBinaryTokenizerFactory;
+import edu.uci.ics.asterix.dataflow.data.common.IBinaryTokenizerFactoryProvider;
 import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
 
 public class AqlBinaryTokenizerFactoryProvider implements IBinaryTokenizerFactoryProvider {
 
@@ -17,14 +17,16 @@
 
     private static final IBinaryTokenizerFactory aqlStringTokenizer = new DelimitedUTF8StringBinaryTokenizerFactory(
             true, true, new UTF8WordTokenFactory(ATypeTag.STRING.serialize(), ATypeTag.INT32.serialize()));
-    
+
     private static final IBinaryTokenizerFactory aqlHashingStringTokenizer = new DelimitedUTF8StringBinaryTokenizerFactory(
             true, true, new HashedUTF8WordTokenFactory(ATypeTag.INT32.serialize(), ATypeTag.INT32.serialize()));
 
-    private static final IBinaryTokenizerFactory orderedListTokenizer = new AOrderedListBinaryTokenizerFactory(new AListElementTokenFactory());
-    
-    private static final IBinaryTokenizerFactory unorderedListTokenizer = new AUnorderedListBinaryTokenizerFactory(new AListElementTokenFactory());
-    
+    private static final IBinaryTokenizerFactory orderedListTokenizer = new AOrderedListBinaryTokenizerFactory(
+            new AListElementTokenFactory());
+
+    private static final IBinaryTokenizerFactory unorderedListTokenizer = new AUnorderedListBinaryTokenizerFactory(
+            new AListElementTokenFactory());
+
     @Override
     public IBinaryTokenizerFactory getWordTokenizerFactory(ATypeTag typeTag, boolean hashedTokens) {
         switch (typeTag) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
index 895dac4..48dde10 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/NGramUTF8StringBinaryTokenizerFactory.java
@@ -19,30 +19,29 @@
 
 package edu.uci.ics.asterix.formats.nontagged;
 
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
 
 // TODO: Should be moved into Hyracks with the rest of the tokenizer code.
-public class NGramUTF8StringBinaryTokenizerFactory implements
-		IBinaryTokenizerFactory {
+public class NGramUTF8StringBinaryTokenizerFactory implements IBinaryTokenizerFactory {
 
-	private static final long serialVersionUID = 1L;
-	private final int gramLength;
-	private final boolean usePrePost;
-	private final boolean ignoreTokenCount;
-	private final boolean sourceHasTypeTag;
-	private final ITokenFactory tokenFactory;
+    private static final long serialVersionUID = 1L;
+    private final int gramLength;
+    private final boolean usePrePost;
+    private final boolean ignoreTokenCount;
+    private final boolean sourceHasTypeTag;
+    private final ITokenFactory tokenFactory;
 
-	public NGramUTF8StringBinaryTokenizerFactory(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
-			boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
-		this.gramLength = gramLength;
-		this.usePrePost = usePrePost;
-	    this.ignoreTokenCount = ignoreTokenCount;
-		this.sourceHasTypeTag = sourceHasTypeTag;
-		this.tokenFactory = tokenFactory;
-	}
+    public NGramUTF8StringBinaryTokenizerFactory(int gramLength, boolean usePrePost, boolean ignoreTokenCount,
+            boolean sourceHasTypeTag, ITokenFactory tokenFactory) {
+        this.gramLength = gramLength;
+        this.usePrePost = usePrePost;
+        this.ignoreTokenCount = ignoreTokenCount;
+        this.sourceHasTypeTag = sourceHasTypeTag;
+        this.tokenFactory = tokenFactory;
+    }
 
     @Override
     public IBinaryTokenizer createTokenizer() {
diff --git a/asterix-runtime/pom.xml b/asterix-runtime/pom.xml
index 77952a6..eb81fe5 100644
--- a/asterix-runtime/pom.xml
+++ b/asterix-runtime/pom.xml
@@ -76,6 +76,11 @@
 		        <type>jar</type>
 		        <scope>compile</scope>
 		</dependency>
+		<dependency>
+		         <groupId>edu.uci.ics.hyracks</groupId>
+		         <artifactId>hyracks-api</artifactId>
+		         <version>0.2.1-SNAPSHOT</version>
+	        </dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
index 188135c..0660743 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/GramTokensEvaluator.java
@@ -15,10 +15,10 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.BooleanSerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IntArray;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IntArray;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
 
 public class GramTokensEvaluator implements ICopyEvaluator {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
index f0f36f5..ac0864c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/WordTokensEvaluator.java
@@ -13,9 +13,9 @@
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IntArray;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IntArray;
 
 public class WordTokensEvaluator implements ICopyEvaluator {
     protected final DataOutput out;
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
index 184d974..942d2ee 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedGramTokensDescriptor.java
@@ -12,9 +12,9 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
 
 public class CountHashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
index 7bfbe49..d0a7f4e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CountHashedWordTokensDescriptor.java
@@ -12,10 +12,10 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
 public class CountHashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
index 558ae7e..5d56ae6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GramTokensDescriptor.java
@@ -12,9 +12,9 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8NGramTokenFactory;
 
 public class GramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
index 04088a2..ef5a926 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedGramTokensDescriptor.java
@@ -12,9 +12,9 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8NGramTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.NGramUTF8StringBinaryTokenizer;
 
 public class HashedGramTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
index dcd5f70..9021233 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/HashedWordTokensDescriptor.java
@@ -12,10 +12,10 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.HashedUTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
 
 public class HashedWordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
index 5e9065b..61b7009 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/WordTokensDescriptor.java
@@ -12,10 +12,10 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizer;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.ITokenFactory;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.UTF8WordTokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.DelimitedUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.ITokenFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.UTF8WordTokenFactory;
 
 public class WordTokensDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
index 2d49982..b6f70bc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
@@ -3,6 +3,7 @@
 import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext.TransactionType;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
@@ -13,11 +14,11 @@
 public class JobEventListenerFactory implements IJobletEventListenerFactory {
 
     private static final long serialVersionUID = 1L;
-    private final long txnId;
+    private final JobId jobId;
     private final boolean transactionalWrite;
 
-    public JobEventListenerFactory(long txnId, boolean transactionalWrite) {
-        this.txnId = txnId;
+    public JobEventListenerFactory(JobId jobId, boolean transactionalWrite) {
+        this.jobId = jobId;
         this.transactionalWrite = transactionalWrite;
     }
 
@@ -30,7 +31,7 @@
                 try {
                     ITransactionManager txnManager = ((AsterixAppRuntimeContext) jobletContext.getApplicationContext()
                             .getApplicationObject()).getTransactionProvider().getTransactionManager();
-                    TransactionContext txnContext = txnManager.getTransactionContext(txnId);
+                    TransactionContext txnContext = txnManager.getTransactionContext(jobId);
                     txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
                             : TransactionType.READ);
                     txnManager.completedTransaction(txnContext, !(jobStatus == JobStatus.FAILURE));
@@ -41,7 +42,12 @@
 
             @Override
             public void jobletStart() {
-
+                try {
+                    ((AsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
+                            .getTransactionProvider().getTransactionManager().getTransactionContext(jobId);
+                } catch (ACIDException e) {
+                    throw new Error(e);
+                }
             }
 
         };
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
index 7e4704b..3f6cbff 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/datagen/AdmDataGen.java
@@ -46,6 +46,7 @@
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.tools.translator.ADGenDmlTranslator;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.data.utils.WriteValueTools;
@@ -928,7 +929,7 @@
         Query q = (Query) parser.Statement();
         aql.close();
         // TODO: Need to fix how to use transactions here.
-        MetadataTransactionContext mdTxnCtx = new MetadataTransactionContext(-1);
+        MetadataTransactionContext mdTxnCtx = new MetadataTransactionContext(new JobId(-1));
         ADGenDmlTranslator dmlt = new ADGenDmlTranslator(mdTxnCtx, q.getPrologDeclList());
         dmlt.translate();
         AqlCompiledMetadataDeclarations acmd = dmlt.getCompiledDeclarations();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java
new file mode 100644
index 0000000..d5e525a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DatasetLockInfo.java
@@ -0,0 +1,504 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import edu.uci.ics.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();
+    }
+
+    /////////////////////////////////////////////////////////
+    //  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;
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
index 695abde..6b6f8e8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/DeadlockDetector.java
@@ -1,67 +1,237 @@
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
-import java.util.ArrayList;
-import java.util.Iterator;
+import java.util.HashMap;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
- * @author pouria Performing a DFS search, upon adding each waiter to a waiting
+ * @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 {
-    LockManager lockMgr;
 
-    ArrayList<Long> grantedList;
-    ArrayList<Long> nextTxrIDs;
-    ArrayList<Long> visited;
-    ArrayList<Long> nextGrantedTxIDs;
+    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;
 
-    public DeadlockDetector(LockManager lm) {
-        this.lockMgr = lm;
-        this.grantedList = new ArrayList<Long>();
-        this.nextTxrIDs = new ArrayList<Long>();
-        this.visited = new ArrayList<Long>();
-        this.nextGrantedTxIDs = new ArrayList<Long>();
+    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 nextHolder;
+        int visitedHolder;
+        int callerId = entityInfoManager.getJobId(entityInfo);
+        int datasetId = entityInfoManager.getDatasetId(entityInfo);
+        int hashValue = entityInfoManager.getPKHashVal(entityInfo);
+        int resource;
+        PrimitiveIntHashMap tempHolderList;
 
-    public synchronized boolean isSafeToAdd(long reqTxId, byte[] resourceId) {
-        grantedList.clear();
-        lockMgr.getLockInfo(resourceId).getGrantedListTxIDs(grantedList);
-        visited.clear();
-        while (grantedList.size() > 0) { // Doing a DFS for loop detection
-            nextTxrIDs.clear();
-            for (long grantee : grantedList) {
-                TxrInfo nextTInfoList = lockMgr.getTxrInfo(grantee);
-                if (nextTInfoList == null) {
-                    continue;
-                }
-                byte[] nextWaitOnRid = nextTInfoList.getWaitOnRid();
-                if (nextWaitOnRid == null) {
-                    continue;
-                }
-                nextGrantedTxIDs.clear();
-                lockMgr.getLockInfo(nextWaitOnRid).getGrantedListTxIDs(nextGrantedTxIDs);
-                if (nextGrantedTxIDs.contains(reqTxId)) {
-                    return false;
-                }
-                removeVisitedTxIDs();
-                nextTxrIDs.addAll(nextGrantedTxIDs);
-                visited.add(grantee);
-            }
-            grantedList.clear();
-            grantedList.addAll(nextTxrIDs);
+        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;
     }
 
-    private void removeVisitedTxIDs() {
-        Iterator<Long> txIdIt = nextGrantedTxIDs.iterator();
-        while (txIdIt.hasNext()) {
-            if (visited.contains(txIdIt.next())) {
-                txIdIt.remove();
+    /**
+     * 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/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java
new file mode 100644
index 0000000..b8820c4
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityInfoManager.java
@@ -0,0 +1,704 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+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 {
+
+    public static final int SHRINK_TIMER_THRESHOLD = 120000; //2min
+
+    private ArrayList<ChildEntityInfoArrayManager> pArray;
+    private int allocChild; //used to allocate the next free EntityInfo slot.
+    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 = 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() {
+        pArray = new ArrayList<ChildEntityInfoArrayManager>();
+        pArray.add(new ChildEntityInfoArrayManager());
+        allocChild = 0;
+        occupiedSlots = 0;
+        isShrinkTimerOn = false;
+    }
+
+    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 >= 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;
+        boolean bContiguous = true;
+        int decreaseCount = 0;
+        int size = pArray.size();
+        int maxDecreaseCount = size / 2;
+        ChildEntityInfoArrayManager child;
+        for (i = size - 1; i >= 0; i--) {
+            child = pArray.get(i);
+            if (child.isEmpty() || child.isDeinitialized()) {
+                if (bContiguous) {
+                    pArray.remove(i);
+                    if (++decreaseCount == maxDecreaseCount) {
+                        break;
+                    }
+                } else {
+                    bContiguous = false;
+                    if (child.isEmpty()) {
+                        child.deinitialize();
+                        if (++decreaseCount == maxDecreaseCount) {
+                            break;
+                        }
+                    }
+                }
+            } else {
+                bContiguous = false;
+            }
+        }
+
+        //reset allocChild when the child is removed or deinitialized.
+        size = pArray.size();
+        if (allocChild >= size || pArray.get(allocChild).isDeinitialized()) {
+            //set allocChild to any initialized one.
+            //It is guaranteed that there is at least one initialized child.
+            for (i = 0; i < size; i++) {
+                if (!pArray.get(i).isDeinitialized()) {
+                    allocChild = i;
+                    break;
+                }
+            }
+        }
+    }
+
+    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 + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
+            s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
+            s.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++) {
+                s.append(j).append(": ");
+                s.append("\t" + child.getJobId(j));
+                s.append("\t" + child.getDatasetId(j));
+                s.append("\t" + child.getPKHashVal(j));
+                s.append("\t" + child.getDatasetLockMode(j));
+                s.append("\t" + child.getDatasetLockCount(j));
+                s.append("\t" + child.getEntityLockMode(j));
+                s.append("\t" + child.getEntityLockCount(j));
+                s.append("\t" + child.getNextEntityActor(j));
+                s.append("\t" + child.getPrevJobResource(j));
+                s.append("\t" + child.getNextJobResource(j));
+                //s.append("\t" + child.getNextDatasetActor(j));
+                s.append("\n");
+            }
+            s.append("\n");
+        }
+        return s.toString();
+    }
+
+    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;
+    }
+
+    //////////////////////////////////////////////////////////////////
+    //   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);
+    //    }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
new file mode 100644
index 0000000..2542157
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
@@ -0,0 +1,802 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants;
+import edu.uci.ics.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;
+        boolean bContiguous = true;
+        int decreaseCount = 0;
+        int size = pArray.size();
+        int maxDecreaseCount = size / 2;
+        ChildEntityLockInfoArrayManager child;
+        for (i = size - 1; i >= 0; i--) {
+            child = pArray.get(i);
+            if (child.isEmpty() || child.isDeinitialized()) {
+                if (bContiguous) {
+                    pArray.remove(i);
+                    if (++decreaseCount == maxDecreaseCount) {
+                        break;
+                    }
+                } else {
+                    bContiguous = false;
+                    if (child.isEmpty()) {
+                        child.deinitialize();
+                        if (++decreaseCount == maxDecreaseCount) {
+                            break;
+                        }
+                    }
+                }
+            } else {
+                bContiguous = false;
+            }
+        }
+
+        //reset allocChild when the child is removed or deinitialized.
+        size = pArray.size();
+        if (allocChild >= size || pArray.get(allocChild).isDeinitialized()) {
+            //set allocChild to any initialized one.
+            //It is guaranteed that there is at least one initialized child.
+            for (i = 0; i < size; i++) {
+                if (!pArray.get(i).isDeinitialized()) {
+                    allocChild = i;
+                    break;
+                }
+            }
+        }
+    }
+
+    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 + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
+            s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
+            s.append("\tX\t").append("S\t").append("LH\t").append("FW\t").append("UP\n");
+            for (int j = 0; j < ChildEntityLockInfoArrayManager.NUM_OF_SLOTS; j++) {
+                s.append(j).append(": ");
+                s.append("\t" + child.getXCount(j));
+                s.append("\t" + child.getSCount(j));
+                s.append("\t" + child.getLastHolder(j));
+                s.append("\t" + child.getFirstWaiter(j));
+                s.append("\t" + child.getUpgrader(j));
+                s.append("\n");
+            }
+            s.append("\n");
+        }
+        return s.toString();
+    }
+
+    //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;
+    }
+
+    //////////////////////////////////////////////////////////////////
+    //   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);
+    }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
index b8c0e17..207ee58 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/ILockManager.java
@@ -15,12 +15,16 @@
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 
 /**
- * @author pouria Interface for the lockManager
+ * Interface for the lockManager
+ * 
+ * @author pouria 
+ * @author kisskys
+ * 
  */
-
 public interface ILockManager {
 
     /**
@@ -36,13 +40,13 @@
      * has a "weaker" lock, then the request would be interpreted as a convert
      * request
      * Waiting transaction would eventually garb the lock, or get timed-out
-     * 
-     * @param resourceID
-     * @param mode
-     * @return
+     * @param datasetId
+     * @param entityHashValue
+     * @param lockMode
+     * @param txnContext
      * @throws ACIDException
      */
-    public boolean lock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
+    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext) throws ACIDException;
 
     /**
      * The method releases "All" the locks taken/waiting-on by a specific
@@ -50,58 +54,63 @@
      * potential waiters, which can be waken up based on their requested lock
      * mode and the waiting policy would be waken up
      * 
-     * @param context
-     * @return
+     * @param txnContext
      * @throws ACIDException
      */
-    public Boolean releaseLocks(TransactionContext context) throws ACIDException;
+    public void releaseLocks(TransactionContext txnContext) throws ACIDException;
 
     /**
-     * Releases "All" the locks by a transaction on a "single specific" resource
-     * Upon releasing, potential waiters, which can be waken up based on their
-     * requested lock mode and the waiting policy would be waken up
      * 
-     * @param resourceID
-     * @return
-     * @throws ACIDException
+     * @param datasetId
+     * @param entityHashValue
+     * @param txnContext
+     * @throws ACIDException TODO
      */
-    public boolean unlock(TransactionContext context, byte[] resourceID) throws ACIDException;
-
-    /**
-     * Request to convert granted lockMode of a transaction on a specific
-     * resource. Requesting transaction would either grab the lock, or sent to
-     * waiting based on the type of the request, and current mask on the
-     * resource and possible set of waiting converters
-     * - If the transaction does not have any lock on the resource, then an
-     * exception is thrown - If the transaction already has a stronger lock,
-     * then no operation is taken
-     * 
-     * @param context
-     * @param resourceID
-     * @param mode
-     * @return
-     * @throws ACIDException
-     */
-    public boolean convertLock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
+    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException;
 
     /**
      * Call to lock and unlock a specific resource in a specific lock mode
-     * 
+     * @param datasetId
+     * @param entityHashValue
+     * @param lockMode TODO
      * @param context
-     * @param resourceID
-     * @param mode
-     * @param timeout
+     * 
      * @return
      * @throws ACIDException
      */
-    public boolean getInstantlock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
+    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext context) throws ACIDException;
 
+
+    /**
+     * 
+     * @param datasetId
+     * @param entityHashValue
+     * @param lockMode
+     * @param context
+     * @return
+     * @throws ACIDException
+     */
+    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext context) throws ACIDException;
+    
+    /**
+     * 
+     * @param datasetId
+     * @param entityHashValue
+     * @param lockMode
+     * @param txnContext
+     * @return
+     * @throws ACIDException
+     */
+    boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+            throws ACIDException;
     /**
      * Prints out the contents of the transactions' table in a readable fashion
      * 
      * @return
      * @throws ACIDException
      */
-    public String getDebugLockStatus() throws ACIDException;
+    public String prettyPrint() throws ACIDException;
+
+
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
new file mode 100644
index 0000000..1a3ac03
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
@@ -0,0 +1,292 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+
+public class JobInfo {
+    private EntityInfoManager entityInfoManager;
+    private LockWaiterManager lockWaiterManager;
+    private TransactionContext 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 dLockHT; //used for keeping dataset-granule-lock's count acquired by this job. 
+
+    public JobInfo(EntityInfoManager entityInfoManager, LockWaiterManager lockWaiterManager, TransactionContext txnCtx) {
+        this.entityInfoManager = entityInfoManager;
+        this.lockWaiterManager = lockWaiterManager;
+        this.jobCtx = txnCtx;
+        this.lastHoldingResource = -1;
+        this.firstWaitingResource = -1;
+        this.upgradingResource = -1;
+        //this.dLockHT = new PrimitiveIntHashMap(1<<6, 1<<3, 180000);
+    }
+
+    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;
+
+        //increaseDatasetLockCount(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 increaseDatasetLockCount(int entityInfo) {
+     * int datasetId = entityInfoManager.getDatasetId(entityInfo);
+     * int count = dLockHT.get(datasetId);
+     * if (count == -1) {
+     * dLockHT.upsert(datasetId, 1);
+     * } else {
+     * dLockHT.upsert(datasetId, count+1);
+     * }
+     * }
+     * public void decreaseDatasetLockCount(int entityInfo) {
+     * int datasetId = entityInfoManager.getDatasetId(entityInfo);
+     * int count = dLockHT.get(datasetId);
+     * if (count > 1) {
+     * dLockHT.upsert(datasetId, count-1);
+     * } else if (count == 1) {
+     * dLockHT.remove(datasetId);
+     * } else if (count <= 0 ) {
+     * throw new IllegalStateException("Illegal state of datasetLock count in JobInfo's dLockHT");
+     * }
+     * }
+     * 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();
+    }
+    
+    /////////////////////////////////////////////////////////
+    //  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/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockInfo.java
deleted file mode 100644
index f3bd6f3..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockInfo.java
+++ /dev/null
@@ -1,621 +0,0 @@
-package edu.uci.ics.asterix.transaction.management.service.locking;
-
-import java.util.ArrayList;
-import java.util.Iterator;
-
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-
-/**
- * @author pouria An instance shows information on a "single resource" about
- *         1) current granted locks on the resource to all transactions 2) locks
- *         that are being waiting on by all converting transactions 3) locks
- *         that are being waiting on by all regular waiting transactions
- *         Each lock mode is interpreted as an integer, and it has a
- *         corresponding bit in the mask variable mask variable should be
- *         interpreted as a sequence of bits, where the i-th bit is 1, if and
- *         only if some transaction(s) have a lock of mode i on this resource
- *         counter is an array which has an entry for each lock mode, and its
- *         i-th entry shows the total number of locks of mode i, granted to all
- *         transactions
- */
-
-public class LockInfo {
-    final int NUM_OF_LOCK_MODES = 32;
-    final int TX_ARRAY_SIZE = 50;
-    final int EOL = -1;
-
-    public static final int NOT_FOUND = -2;
-    public static final int UNKNOWN_IX = -3;
-    public static final int ANY_LOCK_MODE = -4;
-    public static final int UNKNOWN_LOCK_MODE = -5;
-
-    private int mask; // i-th bit corresponds to the i-th lock mode
-    private int[] counter; // i-th entry shows total num of granted locks of
-                           // mode i
-
-    private ArrayList<Integer> grantedList; // (contains index of entries, in
-                                            // the txId, mode and counter lists)
-    private ArrayList<WaitingInfo> convertList; // Waiting Converters
-    private ArrayList<WaitingInfo> waitList; // Regular Waiters
-
-    int nextFreeIx; // Head of free entries lists
-    private ArrayList<long[]> txIdList; // i-th entry shows the id of the
-                                        // granted/waiting transactions
-    private ArrayList<int[]> modeList; // i-th entry shows the mode of the
-                                       // granted/waiting-on lock
-    private ArrayList<int[]> counterList; // i-th entry shows the number of
-                                          // locks (with the defined mode) a
-                                          // transaction has taken (In a free
-                                          // entry is used as the next ptr (next
-                                          // free entry))
-
-    public LockInfo() {
-        this.mask = 0;
-        this.counter = new int[NUM_OF_LOCK_MODES];
-        this.grantedList = new ArrayList<Integer>();
-        this.waitList = new ArrayList<WaitingInfo>();
-        this.convertList = new ArrayList<WaitingInfo>();
-        nextFreeIx = 0;
-        this.txIdList = new ArrayList<long[]>();
-        txIdList.add(new long[TX_ARRAY_SIZE]);
-        this.modeList = new ArrayList<int[]>();
-        modeList.add(new int[TX_ARRAY_SIZE]);
-        this.counterList = new ArrayList<int[]>();
-        counterList.add(initArray(0));
-    }
-
-    private int[] initArray(int ixToStart) {
-        int[] n = new int[TX_ARRAY_SIZE];
-        for (int i = 0; i < TX_ARRAY_SIZE - 1; i++) { // Initializing a new set
-                                                      // of entries, attaching
-                                                      // them
-            n[i] = (++ixToStart); // to the chain of free entries
-        }
-        n[TX_ARRAY_SIZE - 1] = EOL;
-        return n;
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @return the index of the entry corresponding to the transaction with the
-     *         specified granted lock
-     * @throws ACIDException
-     */
-    public int findInGrantedList(long txId, int lMode) throws ACIDException {
-        for (int i : grantedList) {
-            if ((getTxId(i) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(i)))) {
-                return i;
-            }
-        }
-        return NOT_FOUND;
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @return the index of the entry corresponding to the transaction which is
-     *         waiting (as a converter) for the specified lock
-     * @throws ACIDException
-     */
-    public int findInConvertList(long txId, int lMode) throws ACIDException {
-        for (WaitingInfo wi : convertList) {
-            int i = wi.getWaitingEntry().getIX();
-            if ((getTxId(i) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(i)))) {
-                return i;
-            }
-        }
-        return NOT_FOUND;
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @return the index of the entry corresponding to the transaction which is
-     *         waiting (as a regular waiter) for the specified lock
-     * @throws ACIDException
-     */
-    public int findInWaitList(long txId, int lMode) throws ACIDException {
-        for (WaitingInfo wi : waitList) {
-            int i = wi.getWaitingEntry().getIX();
-            if ((getTxId(i) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(i)))) {
-                return i;
-            }
-        }
-        return NOT_FOUND;
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @return the object, on which the specified transaction is waiting for the
-     *         specified lock
-     * @throws ACIDException
-     */
-    public WaitingInfo getWaitingOnObject(long txId, int lMode) throws ACIDException {
-        WaitingInfo wObj = null;
-        Iterator<WaitingInfo> cIt = convertList.iterator();
-        while (cIt.hasNext()) {
-            wObj = cIt.next();
-            int ix = wObj.getWaitingEntry().getIX();
-            if ((getTxId(ix) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(ix)))) {
-                return wObj;
-            }
-        }
-
-        Iterator<WaitingInfo> wIt = waitList.iterator();
-        while (wIt.hasNext()) {
-            wObj = wIt.next();
-            int ix = wObj.getWaitingEntry().getIX();
-            if ((getTxId(ix) == txId) && ((lMode == ANY_LOCK_MODE) || (lMode == getLockMode(ix)))) {
-                return wObj;
-            }
-        }
-        throw new ACIDException("Waiting Entry for transaction " + txId + " Could not be found");
-    }
-
-    public Iterator<WaitingInfo> getIteratorOnConverter() {
-        return (convertList.iterator());
-    }
-
-    public Iterator<WaitingInfo> getIteratorOnWaiters() {
-        return (waitList.iterator());
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param eix
-     *            index of the entry corresponding to the transaction, its
-     *            granted lock and its counter
-     * @throws ACIDException
-     */
-    public void addToGranted(long txId, int lMode, int eix) throws ACIDException {
-        if (eix == UNKNOWN_IX) {
-            eix = findInGrantedList(txId, lMode);
-        }
-        if (eix == NOT_FOUND) { // new lock of mode lMode for Txr
-            int ix = allocateEntryForRequest();
-            grantedList.add(ix);
-            setTxId(txId, ix);
-            setLockMode(lMode, ix);
-            setReqCount(1, ix);
-            mask |= (0x01 << lMode);
-        } else { // Redundant lock of mode lMode for Txr
-            incReqCount(eix);
-        }
-        counter[lMode]++;
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param eix
-     *            index of the entry corresponding to the transaction, its
-     *            granted lock and its counter
-     * @throws ACIDException
-     */
-    public void removeFromGranted(long txId, int lMode, int eix) throws ACIDException {
-        removeFromGranted(txId, lMode, true, eix);
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param forced
-     *            whether to remove all the locks, with the given mode, grabbed
-     *            by the transaction or consider the counter (removing just one
-     *            lock in case the transaction has several locks with the
-     *            specified mode)
-     * @param eix
-     *            index of the entry corresponding to the transaction, its
-     *            granted lock and its counter
-     * @throws ACIDException
-     */
-    private void removeFromGranted(long txId, int lMode, boolean forced, int eix) throws ACIDException {
-        if (eix == UNKNOWN_IX) {
-            eix = findInGrantedList(txId, lMode);
-            if (eix == NOT_FOUND) {
-                return;
-            }
-        }
-
-        if (lMode == ANY_LOCK_MODE) {
-            lMode = getLockMode(eix);
-        }
-
-        int count = getReqCount(eix);
-        if (!forced) {
-            if (count > 1) {
-                setReqCount((count - 1), eix);
-                counter[lMode]--;
-                return;
-            }
-        }
-        // forced or count is 1
-        grantedList.remove((new Integer(eix)));
-        freeEntry(eix);
-        counter[lMode] -= count;
-        if (counter[lMode] == 0) { // No one else has lock with this mode
-            mask &= (~(0x00 | (0x01 << lMode)));
-        }
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param entry
-     *            the object, specified transaction is going to wait on
-     * @throws ACIDException
-     */
-    public void addToConvert(long txId, int lMode, WaitEntry entry) throws ACIDException {
-        int eix = findInConvertList(txId, lMode);
-        if (eix == NOT_FOUND) {
-            int ix = allocateEntryForRequest();
-            entry.setIx(ix);
-            entry.setForWait();
-            convertList.add(new WaitingInfo(entry));
-            setTxId(txId, ix);
-            setLockMode(lMode, ix);
-            setReqCount(1, ix);
-        } else {
-            throw new ACIDException("Adding an already existing converter");
-        }
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param eix
-     *            index of the entry corresponding to the transaction in the
-     *            converters list
-     * @throws ACIDException
-     */
-    public void prepareToRemoveFromConverters(long txId, int lMode, int eix) throws ACIDException {
-        prepareToRemoveFromConverters(txId, lMode, true, eix);
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param forced
-     *            whether to ignore the counter and remove the transaction from
-     *            the converters list or consider the request counter
-     * @param eix
-     *            index of the entry corresponding to the transaction in the
-     *            converters list
-     * @throws ACIDException
-     */
-    private void prepareToRemoveFromConverters(long txId, int lMode, boolean forced, int eix) throws ACIDException {
-        if (eix == UNKNOWN_IX) {
-            eix = findInConvertList(txId, lMode);
-            if (eix == NOT_FOUND) {
-                throw new ACIDException("Lock entry not found in the waiting list");
-            }
-        }
-        freeEntry(eix);
-    }
-
-    /**
-     * @param txId
-     * @return the object specified transaction is waiting on for conversion
-     * @throws ACIDException
-     */
-    public WaitEntry removeFromConverters(long txId) throws ACIDException {
-        Iterator<WaitingInfo> it = convertList.iterator();
-        while (it.hasNext()) {
-            WaitingInfo next = it.next();
-            if (getTxId(next.getWaitingEntry().getIX()) == txId) {
-                it.remove();
-                return next.getWaitingEntry();
-            }
-        }
-        return null;
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param entry
-     * @throws ACIDException
-     */
-    public void addToWaiters(long txId, int lMode, WaitEntry entry) throws ACIDException {
-        int ix = allocateEntryForRequest();
-        entry.setIx(ix);
-        entry.setForWait();
-        waitList.add(new WaitingInfo(entry));
-        setTxId(txId, ix);
-        setLockMode(lMode, ix);
-        setReqCount(1, ix);
-    }
-
-    public void prepareToRemoveFromWaiters(long txId, int lMode, int eix) throws ACIDException {
-        prepareToRemoveFromWaiters(txId, lMode, true, eix);
-    }
-
-    /**
-     * Removes and recycles the entry containing the information about the
-     * transaction, its lock mode and the counter
-     * 
-     * @param txId
-     * @param lMode
-     * @param forced
-     * @param eix
-     *            index of the entry, needs to be freed
-     * @throws ACIDException
-     */
-    private void prepareToRemoveFromWaiters(long txId, int lMode, boolean forced, int eix) throws ACIDException {
-        if (eix == UNKNOWN_IX) {
-            eix = findInWaitList(txId, lMode);
-            if (eix == NOT_FOUND) {
-                throw new ACIDException("Lock entry not found in the waiting list");
-            }
-        }
-        freeEntry(eix);
-    }
-
-    /**
-     * @param txId
-     * @return the object the transaction is waiting on (as a regular waiter)
-     * @throws ACIDException
-     */
-    public WaitEntry removeFromWaiters(long txId) throws ACIDException {
-        Iterator<WaitingInfo> it = waitList.iterator();
-        while (it.hasNext()) {
-            WaitingInfo next = it.next();
-            if (getTxId(next.getWaitingEntry().getIX()) == txId) {
-                it.remove();
-                return next.getWaitingEntry();
-            }
-        }
-        return null;
-    }
-
-    /**
-     * @param lMode
-     * @param eix
-     *            index of the entry corresponding to the transaction's lock and
-     *            its counter
-     */
-    public void grantRedundantLock(int lMode, int eix) {
-        incReqCount(eix);
-        counter[lMode]++;
-    }
-
-    /**
-     * @param txId
-     * @param eix
-     *            index of the entry corresponding to the transaction
-     * @return the actual lock mode, granted to the specified transaction
-     * @throws ACIDException
-     */
-    public int getGrantedLockMode(long txId, int eix) throws ACIDException {
-        if (eix != UNKNOWN_IX) {
-            return getLockMode(eix);
-        }
-        int ix = findInGrantedList(txId, ANY_LOCK_MODE);
-        if (ix == NOT_FOUND) {
-            return UNKNOWN_LOCK_MODE;
-        }
-        return getLockMode(ix);
-    }
-
-    /**
-     * @param txId
-     * @param eix
-     *            index of the entry corresponding to the transaction
-     * @return the actual lock mode, the specified transaction is waiting to
-     *         convert to
-     * @throws ACIDException
-     */
-    public int getConvertLockMode(long txId, int eix) throws ACIDException {
-        if (eix != UNKNOWN_IX) {
-            return getLockMode(eix);
-        }
-        int ix = findInConvertList(txId, ANY_LOCK_MODE);
-        if (ix == NOT_FOUND) {
-            return UNKNOWN_LOCK_MODE;
-        }
-        return getLockMode(ix);
-    }
-
-    /**
-     * @param txId
-     * @param eix
-     *            index of the entry corresponding to the transaction
-     * @return the actual lock mode, the specified transaction is waiting to
-     *         grab
-     * @throws ACIDException
-     */
-    public int getWaitLockMode(long txId, int eix) throws ACIDException {
-        if (eix != UNKNOWN_IX) {
-            return getLockMode(eix);
-        }
-        int ix = findInWaitList(txId, ANY_LOCK_MODE);
-        if (ix == NOT_FOUND) {
-            return UNKNOWN_LOCK_MODE;
-        }
-        return getLockMode(ix);
-    }
-
-    public boolean isConvertListEmpty() {
-        return (!(convertList.size() > 0));
-    }
-
-    public int getMask() {
-        return mask;
-    }
-
-    /**
-     * @param txId
-     * @param lMode
-     * @param eix
-     *            index of the entry corresponding to the transaction's
-     *            currently grabbed lock
-     * @return the updated as if the granted lock to the specified transaction
-     *         gets removed from it (Mainly used to exclude self-conflicts when
-     *         checking for conversions)
-     * @throws ACIDException
-     */
-    public int getUpdatedMask(long txId, int lMode, int eix) throws ACIDException {
-        if (eix == UNKNOWN_IX) {
-            eix = findInGrantedList(txId, lMode);
-        }
-        if (eix == NOT_FOUND) {
-            return mask;
-        }
-
-        int txCount = getReqCount(eix);
-        int totalCount = getLockAggCounter(lMode);
-
-        if (totalCount == txCount) { // txId is the only lock-holder with this
-                                     // mode
-            return (mask & (~(0x00 | (0x01 << lMode))));
-        }
-
-        return mask;
-    }
-
-    /**
-     * @param lmix
-     * @return the total number of locks of the specified mode, grabbed on this
-     *         resource (by all transactions)
-     */
-    private int getLockAggCounter(int lmix) {
-        return counter[lmix];
-    }
-
-    /**
-     * Populates the grantedIDs list with the ids of all transactions in the
-     * granted list
-     * 
-     * @param grantedIDs
-     */
-    public void getGrantedListTxIDs(ArrayList<Long> grantedIDs) {
-        Iterator<Integer> gIt = grantedList.iterator();
-        while (gIt.hasNext()) {
-            grantedIDs.add(getTxId(gIt.next()));
-        }
-    }
-
-    /**
-     * @return the index of an entry that can be used to capture one transaction
-     *         and its requested/granted lock mode and the counter
-     */
-    private int allocateEntryForRequest() {
-        if (nextFreeIx == EOL) {
-            nextFreeIx = txIdList.size() * TX_ARRAY_SIZE;
-            txIdList.add(new long[TX_ARRAY_SIZE]);
-            modeList.add(new int[TX_ARRAY_SIZE]);
-            counterList.add(initArray(nextFreeIx));
-        }
-        int ixToRet = nextFreeIx;
-        nextFreeIx = getReqCount(nextFreeIx);
-        return ixToRet;
-    }
-
-    /**
-     * @param ix
-     *            index of the entry, to be recycled
-     */
-    private void freeEntry(int ix) {
-        setReqCount(nextFreeIx, ix); // counter holds ptr to next free entry in
-                                     // free entries
-        nextFreeIx = ix;
-    }
-
-    /**
-     * @param ix
-     *            index of the entry that captures the transaction id
-     * @return id of the transaction whose info is captured in the specified
-     *         index
-     */
-    public long getTxId(int ix) {
-        return (txIdList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]);
-    }
-
-    /**
-     * @param txId
-     * @param ix
-     *            index of the entry that will capture the transaction id
-     */
-    private void setTxId(long txId, int ix) {
-        txIdList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE] = txId;
-    }
-
-    /**
-     * @param ix
-     *            index of the entry that captures the lock mode
-     *            requested/grabbed by the specified transaction
-     * @return
-     */
-    public int getLockMode(int ix) {
-        return (modeList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]);
-    }
-
-    /**
-     * @param lMode
-     * @param index
-     *            of the entry that will capture the lock mode requested/grabbed
-     *            by the specified transaction
-     */
-    private void setLockMode(int lMode, int ix) {
-        modeList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE] = lMode;
-    }
-
-    /**
-     * @param ix
-     * @return index of the entry that captures the counter of locks
-     */
-    public int getReqCount(int ix) {
-        return (counterList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]);
-    }
-
-    /**
-     * @param count
-     * @param ix
-     *            index of the entry that captures the counter of locks
-     */
-    private void setReqCount(int count, int ix) {
-        counterList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE] = count;
-    }
-
-    /**
-     * @param ix
-     *            index of the counter, needed to be incremented on behalf of a
-     *            transaction
-     */
-    private void incReqCount(int ix) {
-        counterList.get(ix / TX_ARRAY_SIZE)[ix % TX_ARRAY_SIZE]++;
-    }
-}
-
-class WaitingInfo {
-    /**
-     * An object of this class captures the information corresponding to a
-     * regular or converter waiter
-     */
-
-    private boolean isVictim; // Whether the corresponding transaction is an
-                              // Victim or it can be waken up safely
-    private WaitEntry waitEntry; // The object, on which the waiter is waiting.
-                                 // This object is mainly used to notify the
-                                 // waiter, to be waken up
-
-    public WaitingInfo(WaitEntry waitEntry) {
-        this.waitEntry = waitEntry;
-        this.isVictim = false;
-    }
-
-    public boolean isVictim() {
-        return isVictim;
-    }
-
-    public void setAsVictim() {
-        this.isVictim = true;
-    }
-
-    public WaitEntry getWaitingEntry() {
-        return this.waitEntry;
-    }
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index 22cab54..0d2f3f6 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -1,982 +1,1698 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Hashtable;
+import java.io.Serializable;
+import java.util.HashMap;
 import java.util.Iterator;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.logging.Level;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
 
 /**
- * @author pouria 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.
- * @see ILockManager, DeadlockDetector, TimeOutDetector, ILockMatrix,
- *      LockMatrix, TxrInfo and LockInfo
+ * 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 {
 
     private static final Logger LOGGER = Logger.getLogger(LockManager.class.getName());
+    private static final int LOCK_MANAGER_INITIAL_HASH_TABLE_SIZE = 50;// do we need this?
+    public static final boolean IS_DEBUG_MODE = false;//true
 
-    final int INIT_TABLE_SIZE = 50;
-    private LMTables lmTables;
-    ILockMatrix lMtx;
+    private TransactionProvider txnProvider;
 
-    WaitObjectManager woManager;
-    TimeOutDetector toutDetector;
-    DeadlockDetector deadlockDetector;
+    //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 HashMap<JobId, JobInfo> jobHT;
+    private HashMap<DatasetId, DatasetLockInfo> datasetResourceHT;
 
-    public LockManager(TransactionProvider factory) throws ACIDException {
-        Properties p = new Properties();
-        InputStream is = null;
-        ILockMatrix lockMatrix = null;
-        int[] confTab = null;
-        int[] convTab = null;
+    private EntityLockInfoManager entityLockInfoManager;
+    private EntityInfoManager entityInfoManager;
+    private LockWaiterManager lockWaiterManager;
 
-        try {
-            File file = new File(TransactionManagementConstants.LockManagerConstants.LOCK_CONF_DIR + File.separator
-                    + TransactionManagementConstants.LockManagerConstants.LOCK_CONF_FILE);
-            if (file.exists()) {
-                is = new FileInputStream(TransactionManagementConstants.LockManagerConstants.LOCK_CONF_DIR
-                        + File.separator + TransactionManagementConstants.LockManagerConstants.LOCK_CONF_FILE);
-                p.load(is);
-                confTab = getConfTab(p);
-                convTab = getConvTab(p);
-            } else {
-                if (LOGGER.isLoggable(Level.SEVERE)) {
-                    LOGGER.severe("Lock configuration file not found, using defaults !");
-                }
-                confTab = TransactionManagementConstants.LockManagerConstants.LOCK_CONFLICT_MATRIX;
-                convTab = TransactionManagementConstants.LockManagerConstants.LOCK_CONVERT_MATRIX;
-            }
-            lockMatrix = new LockMatrix(confTab, convTab);
-            initialize(lockMatrix);
-        } catch (IOException ioe) {
-            if (is != null) {
-                try {
-                    is.close();
-                } catch (IOException e) {
-                    throw new ACIDException("unable to close input stream ", e);
-                }
-            }
-            throw new ACIDException(" unable to create LockManager", ioe);
-        }
-    }
+    private DeadlockDetector deadlockDetector;
+    private TimeOutDetector toutDetector;
+    private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
 
-    private static int[] getConfTab(Properties properties) {
-        return null;
-    }
+    private int tryLockDatasetGranuleRevertOperation;
 
-    private static int[] getConvTab(Properties properties) {
-        return null;
-    }
+    private LockRequestTracker lockRequestTracker; //for debugging
+    private ConsecutiveWakeupContext consecutiveWakeupContext;
 
-    private void initialize(ILockMatrix matrix) throws ACIDException {
-        this.lmTables = new LMTables(INIT_TABLE_SIZE);
-        this.lMtx = matrix;
-        woManager = new WaitObjectManager();
-        this.deadlockDetector = new DeadlockDetector(this);
+    public LockManager(TransactionProvider txnProvider) throws ACIDException {
+        this.txnProvider = txnProvider;
+        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();
+        this.lockWaiterManager = new LockWaiterManager();
+        this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
+        this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
+                entityInfoManager, lockWaiterManager);
         this.toutDetector = new TimeOutDetector(this);
+        this.tempDatasetIdObj = new DatasetId(0);
+        this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
+
+        if (IS_DEBUG_MODE) {
+            this.lockRequestTracker = new LockRequestTracker();
+        }
     }
 
     @Override
-    public boolean lock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException {
-        long txId = context.getTransactionID();
-        TxrInfo txrInfo = null;
-        WaitEntry waitObj = null;
-        Boolean isConverting = false;
-        int grantedMode = -1;
-        LockInfo lInfo = null;
-        boolean shouldAbort = false;
+    public void lock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+            throws ACIDException {
+        internalLock(datasetId, entityHashValue, lockMode, txnContext);
+    }
 
-        synchronized (lmTables) {
-            txrInfo = lmTables.getTxrInfo(txId);
-            if (txrInfo == null) {
-                txrInfo = new TxrInfo(context);
-                lmTables.putTxrInfo(txId, txrInfo);
-            }
+    private void internalLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+            throws ACIDException {
 
-            lInfo = lmTables.getLockInfo(resourceID);
-            if (lInfo == null) { // First lock on the resource, grant it
-                lInfo = new LockInfo();
-                lInfo.addToGranted(txId, mode, LockInfo.NOT_FOUND);
-                lmTables.putLockInfo(resourceID, lInfo);
-                txrInfo.addGrantedLock(resourceID, mode);
-                return true;
-            }
+        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;
 
-            int eix = lInfo.findInGrantedList(txId, LockInfo.ANY_LOCK_MODE);
-            if (eix == LockInfo.NOT_FOUND) { // First lock by this Txr on the
-                                             // resource
-                if (!lInfo.isConvertListEmpty()) { // If Some converter(s)
-                                                   // is(are) waiting, Txr needs
-                                                   // to wait for fairness
+        latchLockTable();
+        validateJob(txnContext);
 
-                    // ----Deadlock Detection ---
-                    if (!isDeadlockFree(txId, resourceID)) {
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("DEADLOCK DETECTED FOR TRANSACTION " + txId);
-                        }
-                        context.setStatus(TransactionContext.TIMED_OUT_SATUS);
-                        shouldAbort = true;
-                    }
-                    // ---------------------------
-
-                    else { // Safe to wait
-                        waitObj = woManager.allocate();
-                        if (waitObj == null) {
-                            throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr " + txId);
-                        }
-                        lInfo.addToWaiters(txId, mode, waitObj);
-                        txrInfo.setWaitOnRid(resourceID);
-                        context.setStartWaitTime(System.currentTimeMillis());
-
-                    }
-                } else { // No converter(s) waiting
-                    int mask = lInfo.getMask();
-                    if (lMtx.conflicts(mask, mode)) { // If There is conflict,
-                                                      // Txr needs to wait
-
-                        // ----Deadlock Detection ---
-                        if (!isDeadlockFree(txId, resourceID)) {
-                            if (LOGGER.isLoggable(Level.INFO)) {
-                                LOGGER.info("DEADLOCK DETECTED FOR TRANSACTION " + txId);
-                            }
-                            context.setStatus(TransactionContext.TIMED_OUT_SATUS);
-                            shouldAbort = true;
-                        }
-                        // ---------------------------
-                        else { // Safe to wait
-                            waitObj = woManager.allocate();
-                            if (waitObj == null) {
-                                throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr "
-                                        + txId);
-                            }
-                            lInfo.addToWaiters(txId, mode, waitObj);
-                            txrInfo.setWaitOnRid(resourceID);
-                            context.setStartWaitTime(System.currentTimeMillis());
-                        }
-                    } else { // No conflicts with the current mask, just grant
-                        // it
-                        lInfo.addToGranted(txId, mode, LockInfo.NOT_FOUND);
-                        txrInfo.addGrantedLock(resourceID, mode);
-                        return true;
-                    }
-                }
-            }
-
-            else { // Redundant or Conversion
-                grantedMode = lInfo.getGrantedLockMode(txId, eix);
-                if (grantedMode == mode) {
-                    lInfo.grantRedundantLock(mode, eix);
-                    return true; // No need to update tInfo
-                } else {
-                    if (lMtx.isConversion(grantedMode, mode)) {
-                        isConverting = true;
-                    } else {
-                        return true; // Txr already has a stronger lock on the
-                                     // resource
-                    }
-
-                }
-            }
+        if (IS_DEBUG_MODE) {
+            trackLockRequest("Requested", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
+                    dLockInfo, eLockInfo);
         }
 
-        if (isConverting) {
+        dLockInfo = datasetResourceHT.get(datasetId);
+        jobInfo = jobHT.get(jobId);
+
+        //#. 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 (IS_DEBUG_MODE) {
+                trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
+                        dLockInfo, eLockInfo);
+            }
+
+            unlatchLockTable();
+            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 (IS_DEBUG_MODE) {
+            trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext, dLockInfo,
+                    eLockInfo);
+        }
+        unlatchLockTable();
+        return;
+    }
+
+    private void validateJob(TransactionContext txnContext) throws ACIDException {
+        if (txnContext.getTxnState() == TransactionState.ABORTED) {
+            unlatchLockTable();
+            throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
+        } else if (txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
             try {
-                return convertLockForNewTransaction(context, resourceID, grantedMode, mode);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
+                requestAbort(txnContext);
+            } finally {
+                unlatchLockTable();
             }
         }
+    }
 
-        if (shouldAbort) {
-            requestTxrAbort(context);
-            return false;
-        }
+    private int lockDatasetGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
+            TransactionContext 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;
 
-        // Txr needs to wait and it is safe to wait
-        synchronized (waitObj) {
-            while (waitObj.needWait()) {
-                try {
-                    waitObj.wait();
-                } catch (InterruptedException e) {
-                    e.printStackTrace();
+        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);
                 }
             }
-            // Txr just woke up
-            woManager.deAllocate(waitObj);
-            if (context.getStatus() == TransactionContext.TIMED_OUT_SATUS) { // selected
-                                                                             // as
-                                                                             // a
-                                                                             // victim
-                requestTxrAbort(context);
-                return false;
+
+            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]
+                //There has been no same caller as (jId, dId, entityHashValue) triplet.
+                //But there could be the same caller as (jId, dId) pair.
+                //For example, two requests (J1, D1, E1) and (J1, D1, E2) are considered as duplicated call in dataset-granule perspective.
+                //Therefore, the above duplicated call case 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, datasetLockMode)) {
+                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);
+                }
             }
         }
 
-        synchronized (context) {
-            context.setStatus(TransactionContext.ACTIVE_STATUS);
-            context.setStartWaitTime(TransactionContext.INVALID_TIME);
-        }
+        return entityInfo;
+    }
 
-        synchronized (lmTables) {
-            txrInfo = lmTables.getTxrInfo(txId);
-            if (txrInfo == null) {
-                throw new ACIDException("Transaction " + txId + " removed from Txr Table Unexpectedlly");
+    private void lockEntityGranule(DatasetId datasetId, int entityHashValue, byte lockMode,
+            int entityInfoFromDLockInfo, TransactionContext 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;
             }
-            txrInfo.addGrantedLock(resourceID, mode);
-            txrInfo.setWaitOnRid(null);
-        }
 
-        return true; // Arriving here when Txr wakes up and it successfully
-                     // locks the resource
+            //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 boolean convertLock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException {
-        long txId = context.getTransactionID();
-        int curMode = -1;
-        TxrInfo txrInfo = null;
-        LockInfo lInfo = null;
-        synchronized (lmTables) {
-            txrInfo = lmTables.getTxrInfo(txId);
+    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException {
+        JobId jobId = txnContext.getJobId();
+        int eLockInfo = -1;
+        DatasetLockInfo dLockInfo = null;
+        JobInfo jobInfo;
+        int entityInfo = -1;
 
-            if (txrInfo == null) {
-                throw new ACIDException("No lock is granted to the transaction, to convert");
+        if (IS_DEBUG_MODE) {
+            if (entityHashValue == -1) {
+                throw new UnsupportedOperationException(
+                        "Unsupported unlock request: dataset-granule unlock is not supported");
             }
+        }
 
-            TInfo tInfo = txrInfo.getTxrInfo(resourceID, LockInfo.ANY_LOCK_MODE, TxrInfo.NOT_KNOWN_IX);
-            if (tInfo == null) {
-                throw new ACIDException("No lock is granted to the transaction on the resource, to convert");
+        latchLockTable();
+        validateJob(txnContext);
+
+        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 (IS_DEBUG_MODE) {
+            if (dLockInfo == null || jobInfo == null) {
+                throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
             }
-
-            curMode = tInfo.getMode();
-            if (mode == curMode) { // Redundant
-                return true; // We do not increment the counter, because it is a
-                // conversion
+        }
+        eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+        if (IS_DEBUG_MODE) {
+            if (eLockInfo == -1) {
+                throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
             }
+        }
 
-            if (!lMtx.isConversion(curMode, mode)) {
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Transaction " + txId + " already has grabbed a stronger mode (" + curMode + ") than "
-                            + mode);
+        //find the corresponding entityInfo
+        entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId.getId(), entityHashValue);
+        if (IS_DEBUG_MODE) {
+            if (entityInfo == -1) {
+                throw new IllegalStateException("Invalid unlock request[" + jobId.getId() + "," + datasetId.getId()
+                        + "," + entityHashValue + "]: Corresponding lock info doesn't exist.");
+            }
+        }
+
+        //decrease the corresponding count of dLockInfo/eLockInfo/entityInfo
+        dLockInfo.decreaseLockCount(entityInfoManager.getDatasetLockMode(entityInfo) == LockMode.S ? LockMode.IS
+                : LockMode.IX);
+        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;
                 }
-
-                return true;
+                waiterObjId = waiterObj.getNextWaiterObjId();
             }
-
-            lInfo = lmTables.getLockInfo(resourceID);
-            if (lInfo == null) {
-                throw new ACIDException("No lock on the resource, to convert");
+            if (threadCount == 0) {
+                if (entityInfoManager.getEntityLockMode(entityInfo) == LockMode.X) {
+                    //TODO
+                    //write a commit log for the unlocked resource
+                    //need to figure out that instantLock() also needs to write a commit log. 
+                }
+                entityInfoManager.deallocate(entityInfo);
             }
         }
 
-        try {
-            return convertLockForNewTransaction(context, resourceID, curMode, mode);
-        } catch (InterruptedException e) {
-            e.printStackTrace();
+        //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);
         }
 
-        throw new ACIDException("Problem in Lock Converting for Transaction " + txId
-                + " (We unexpectedly returned from convert lock for new transaction)");
+        //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 (IS_DEBUG_MODE) {
+            trackLockRequest("Granted", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
+                    dLockInfo, eLockInfo);
+        }
+        unlatchLockTable();
     }
 
-    private boolean convertLockForNewTransaction(TransactionContext context, byte[] resourceId, int curMode, int reqMode)
-            throws ACIDException, InterruptedException {
-        long txId = context.getTransactionID();
-        WaitEntry waitObj = null;
-        boolean shouldAbort = false;
-        TxrInfo txrInfo = null;
-        synchronized (lmTables) {
-            LockInfo lInfo = lmTables.getLockInfo(resourceId);
-            txrInfo = lmTables.getTxrInfo(txId);
-            // ---Check if the conversion is already done---
-            int eix = lInfo.findInGrantedList(txId, reqMode);
-            if (eix != LockInfo.NOT_FOUND) {
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Conversion already done for Transaction " + txId + " for lock " + reqMode
-                            + " on resource ");
+    @Override
+    public void releaseLocks(TransactionContext 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();
+
+        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) {
+            unlatchLockTable();
+            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!!!");
                 }
-                return true;
             }
-            // --------------------------------------------
-
-            int updatedMask = lInfo.getUpdatedMask(txId, curMode, LockInfo.UNKNOWN_IX);
-            if (lMtx.conflicts(updatedMask, reqMode)) { // if Conflicting, Txr
-                                                        // needs to wait
-
-                // ---- Deadlock Detection ---
-                if (!isDeadlockFree(txId, resourceId)) {
-                    if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("DEADLOCK DETECTED IN CONVERSION FOR TRANSACTION ");
-                    }
-                    context.setStatus(TransactionContext.TIMED_OUT_SATUS);
-                    shouldAbort = true;
+            
+            //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);
 
-                else {
-                    waitObj = woManager.allocate();
-                    if (waitObj == null) {
-                        throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr " + txId);
+            //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);
                     }
-                    lInfo.addToConvert(txId, reqMode, waitObj);
-                    txrInfo.setWaitOnRid(resourceId);
-                    context.setStartWaitTime(System.currentTimeMillis());
+                }
+                
+                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);
+                    //                    if (IS_DEBUG_MODE) {
+                    //                        System.out.println("removed PK["+entityHashValue+"]");
+                    //                    }
                 }
             }
 
-            else { // no conflicts, grant it
-                lInfo.removeFromGranted(txId, curMode, LockInfo.UNKNOWN_IX);
-                lInfo.addToGranted(txId, reqMode, LockInfo.NOT_FOUND);
-                txrInfo.removeLock(resourceId, curMode, TxrInfo.NOT_KNOWN_IX);
-                txrInfo.addGrantedLock(resourceId, reqMode);
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Transaction " + txId + " could convert to " + reqMode + " lock on resource ");
+            //deallocate entityInfo
+            entityInfoManager.deallocate(entityInfo);
+            //            if (IS_DEBUG_MODE) {
+            //                System.out.println("dellocate EntityInfo["+entityInfo+"]");
+            //            }
+
+            entityInfo = prevEntityInfo;
+        }
+
+        //remove JobInfo
+        jobHT.remove(jobId);
+        
+        if (existWaiter) {
+            txnContext.setStatus(TransactionContext.TIMED_OUT_STATUS);
+            txnContext.setTxnState(TransactionState.ABORTED);
+        }
+
+        if (IS_DEBUG_MODE) {
+            trackLockRequest("Granted", RequestType.RELEASE_LOCKS, new DatasetId(0), 0, (byte) 0, txnContext,
+                    dLockInfo, eLockInfo);
+        }
+        unlatchLockTable();
+    }
+
+    @Override
+    public void instantLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+            throws ACIDException {
+
+        //        try {
+        //            internalLock(datasetId, entityHashValue, lockMode, txnContext);
+        //            return;
+        //        } finally {
+        //            unlock(datasetId, entityHashValue, txnContext);
+        //        }
+        internalLock(datasetId, entityHashValue, lockMode, txnContext);
+        unlock(datasetId, entityHashValue, txnContext);
+    }
+
+    @Override
+    public boolean tryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+            throws ACIDException {
+        return internalTryLock(datasetId, entityHashValue, lockMode, txnContext);
+    }
+
+    @Override
+    public boolean instantTryLock(DatasetId datasetId, int entityHashValue, byte lockMode, TransactionContext txnContext)
+            throws ACIDException {
+        boolean isGranted = false;
+        //        try {
+        //            isGranted = internalTryLock(datasetId, entityHashValue, lockMode, txnContext);
+        //            return isGranted;
+        //        } finally {
+        //            if (isGranted) {
+        //                unlock(datasetId, entityHashValue, txnContext);
+        //            }
+        //        }
+        isGranted = internalTryLock(datasetId, entityHashValue, lockMode, txnContext);
+        if (isGranted) {
+            unlock(datasetId, entityHashValue, txnContext);
+        }
+        return isGranted;
+    }
+
+    private boolean internalTryLock(DatasetId datasetId, int entityHashValue, byte lockMode,
+            TransactionContext txnContext) 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 = false;
+
+        latchLockTable();
+        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 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 (IS_DEBUG_MODE) {
+                trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
+                        dLockInfo, eLockInfo);
+            }
+
+            unlatchLockTable();
+            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);
                 }
-                return true;
             }
         }
 
-        if (shouldAbort) {
-            requestTxrAbort(context);
-            return false;
+        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);
+            }
         }
 
-        // Txr needs to wait, and it is safe
-        synchronized (waitObj) {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Transaction " + txId + " needs to wait for convert " + reqMode);
-            }
-            while (waitObj.needWait()) {
+        unlatchLockTable();
 
-                waitObj.wait();
-            }
+        return isSuccess;
+    }
 
-            if (context.getStatus() == TransactionContext.TIMED_OUT_SATUS) { // selected
-                // as
-                // a
-                // victim
-                requestTxrAbort(context);
-                woManager.deAllocate(waitObj);
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Transaction " + txId + " wakes up and victimied for convert " + reqMode);
+    private void trackLockRequest(String msg, int requestType, DatasetId datasetIdObj, int entityHashValue,
+            byte lockMode, TransactionContext txnContext, DatasetLockInfo dLockInfo, int eLockInfo) {
+        StringBuilder s = new StringBuilder();
+        LockRequest request = new LockRequest(Thread.currentThread().getName(), requestType, datasetIdObj,
+                entityHashValue, lockMode, txnContext);
+        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, TransactionContext 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,
+            TransactionContext 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)) {
+                
+                //[Notice]
+                //There has been no same caller as (jId, dId, entityHashValue) triplet.
+                //But there could be the same caller as (jId, dId) pair.
+                //For example, two requests (J1, D1, E1) and (J1, D1, E2) are considered as duplicated call in dataset-granule perspective.
+                //Therefore, the above duplicated call case 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, TransactionContext 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;
             }
-        }
 
-        synchronized (context) {
-            context.setStatus(TransactionContext.ACTIVE_STATUS);
-            context.setStartWaitTime(TransactionContext.INVALID_TIME);
-        }
-
-        synchronized (lmTables) {
-            txrInfo = lmTables.getTxrInfo(txId);
-            if (txrInfo == null) {
-                throw new ACIDException("Transaction " + txId + " removed from Txr Table Unexpectedlly");
+            //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;
             }
-            txrInfo.removeLock(resourceId, curMode, TxrInfo.NOT_KNOWN_IX);
-            txrInfo.addGrantedLock(resourceId, reqMode);
-            txrInfo.setWaitOnRid(null);
+
+            //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);
         }
 
-        woManager.deAllocate(waitObj);
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Transaction " + txId + " wakes up and convert to " + reqMode);
-        }
         return true;
     }
 
-    @Override
-    public boolean unlock(TransactionContext context, byte[] resourceID) throws ACIDException {
-        long txId = context.getTransactionID();
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Transaction " + txId + " wants to unlock on ");
-        }
-        synchronized (lmTables) {
-            TxrInfo txrInfo = lmTables.getTxrInfo(txId);
-            if (txrInfo == null) {
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Transaction " + txId + " has no locks on resource ");
-                }
-                return true;
-            }
-
-            TInfo transactionInfo = txrInfo.getTxrInfo(resourceID, LockInfo.ANY_LOCK_MODE, TxrInfo.NOT_KNOWN_IX);
-            if (transactionInfo == null) {
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Transaction " + txId + " has no locks on resource ");
-                }
-                return true;
-            }
-
-            int lockMode = transactionInfo.getMode();
-
-            LockInfo lInfo = (LockInfo) lmTables.getLockInfo(resourceID);
-            if (lInfo == null) {
-                throw new ACIDException("Trying to unlock() a lock, on a non-existing resource");
-            }
-            txrInfo.removeLock(resourceID, lockMode, TxrInfo.NOT_KNOWN_IX);
-            lInfo.removeFromGranted(txId, lockMode, LockInfo.UNKNOWN_IX);
-
-            Iterator<WaitingInfo> convIt = lInfo.getIteratorOnConverter();
-            while (convIt.hasNext()) {
-                WaitingInfo nextConvInfo = convIt.next();
-                if (nextConvInfo.isVictim()) {
-                    continue;
-                }
-                WaitEntry nextConv = nextConvInfo.getWaitingEntry();
-                synchronized (nextConv) {
-                    int reqIx = nextConv.getIX(); // entry ix for the (new)
-                    // requested lock
-                    long convIx = lInfo.getTxId(reqIx);
-                    long convTxId = lInfo.getTxId(reqIx);
-                    int curConvMode = lInfo.getGrantedLockMode(convTxId, LockInfo.UNKNOWN_IX);
-                    int reqConvMode = lInfo.getLockMode(reqIx);
-                    int updatedMask = lInfo.getUpdatedMask(convIx, curConvMode, LockInfo.UNKNOWN_IX);
-                    if (lMtx.conflicts(updatedMask, reqConvMode)) { // We found
-                                                                    // conflict,
-                                                                    // no more
-                                                                    // transactions
-                                                                    // need to
-                                                                    // be waken
-                                                                    // up
-                        context.setStartWaitTime(TransactionContext.INVALID_TIME);
-                        if (txrInfo.getSize() == 0) {
-                            lmTables.removeTxrInfo(txId);
-                            if (LOGGER.isLoggable(Level.INFO)) {
-                                LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
-                            }
-                        }
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("Transaction " + txId + " unlocked its lock");
-                        }
-                        return true;
-                    }
-                    // Converter is ready to wake up
-                    lmTables.getTxrInfo(convTxId).getContext().setStatus(TransactionContext.ACTIVE_STATUS);
-                    lInfo.removeFromGranted(convTxId, curConvMode, LockInfo.UNKNOWN_IX /* curIx */);
-                    lInfo.addToGranted(convTxId, reqConvMode, LockInfo.NOT_FOUND);
-                    lInfo.prepareToRemoveFromConverters(convTxId, reqConvMode, reqIx);
-                    nextConv.wakeUp();
-                    convIt.remove();
-                    nextConv.notifyAll();
-                }
-            }
-
-            Iterator<WaitingInfo> waitIt = lInfo.getIteratorOnWaiters();
-            while (waitIt.hasNext()) {
-                WaitingInfo nextWaiterInfo = waitIt.next();
-                if (nextWaiterInfo.isVictim()) {
-                    continue;
-                }
-                WaitEntry nextWaiter = nextWaiterInfo.getWaitingEntry();
-                synchronized (nextWaiter) {
-                    int waitIx = nextWaiter.getIX();
-                    long waitTxId = lInfo.getTxId(waitIx);
-                    int reqLock = lInfo.getLockMode(waitIx);
-                    int mask = lInfo.getMask();
-                    if (lMtx.conflicts(mask, reqLock)) {
-                        if (LOGGER.isLoggable(Level.INFO)) {
-                            LOGGER.info("Transaction " + txId + " unlocked its lock on ");
-                        }
-
-                        context.setStartWaitTime(TransactionContext.INVALID_TIME);
-                        if (txrInfo.getSize() == 0) {
-                            lmTables.removeTxrInfo(txId);
-                            if (LOGGER.isLoggable(Level.INFO)) {
-                                LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
-                            }
-                        }
-                        return true;
-                    }
-                    lInfo.addToGranted(waitTxId, reqLock, LockInfo.NOT_FOUND);
-                    lInfo.prepareToRemoveFromWaiters(waitTxId, reqLock, waitIx);
-                    nextWaiter.wakeUp();
-                    waitIt.remove();
-                    nextWaiter.notifyAll();
-                }
-            }
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Transaction " + txId + " unlocked its lock");
-            }
-            context.setStartWaitTime(TransactionContext.INVALID_TIME);
-            if (txrInfo.getSize() == 0) {
-                lmTables.removeTxrInfo(txId);
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
-                }
-            }
-            return true;
-        }
+    private void latchLockTable() {
+        lockTableLatch.writeLock().lock();
     }
 
-    @Override
-    public boolean getInstantlock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException {
-        throw new ACIDException("Instant Locking is not supported");
+    private void unlatchLockTable() {
+        lockTableLatch.writeLock().unlock();
     }
 
-    public Iterator<Long> getTxrInfoIterator() {
-        return lmTables.getIteratorOnTxrs();
+    private void latchWaitNotify() {
+        waiterLatch.writeLock().lock();
     }
 
-    @Override
-    public synchronized Boolean releaseLocks(TransactionContext context) throws ACIDException {
-        long txId = context.getTransactionID();
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in unlock)");
-        }
-        synchronized (lmTables) {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Transaction " + txId + " started releasing its locks !");
-            }
-            TxrInfo txrInfo = lmTables.getTxrInfo(txId);
-            if (txrInfo == null) {
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Transaction with ID " + txId
-                            + " has no locks to release. (Returning from Release Locks)");
-                }
-                return true;
-            }
-            // First Remove from the waiting list (if waiting)
-            byte[] waitOnRid = txrInfo.getWaitOnRid();
-            if (waitOnRid != null) {
-                LockInfo lInfo = (LockInfo) lmTables.getLockInfo(waitOnRid);
-                if ((lInfo.removeFromConverters(txId)) == null) {
-                    if ((lInfo.removeFromWaiters(txId)) == null) {
-                        throw new ACIDException("Transaction " + txId
-                                + " Not Found in the convert/wait list of the resource, it should have waited for");
-                    }
-                }
-            }
+    private void unlatchWaitNotify() {
+        waiterLatch.writeLock().unlock();
+    }
 
-            Iterator<TInfo> tInfoIt = txrInfo.getIterator();
+    private int handleLockWaiter(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isUpgrade,
+            boolean isDatasetLockInfo, TransactionContext txnContext, JobInfo jobInfo, int duplicatedWaiterObjId)
+            throws ACIDException {
+        int waiterId = -1;
+        LockWaiter waiter;
+        int waiterCount = 0;
+        boolean isInterruptedExceptionOccurred = false;
 
-            while (tInfoIt.hasNext()) {
-                TInfo nextInfo = tInfoIt.next();
-                byte[] nextRid = nextInfo.getResourceId();
-                int nextLockMode = nextInfo.getMode();
-                LockInfo lInfo = lmTables.getLockInfo(nextRid);
-                lInfo.removeFromGranted(txId, nextLockMode, LockInfo.UNKNOWN_IX); // Remove
-                                                                                  // transaction's
-                                                                                  // granted
-                                                                                  // lock
-                // Now lets try to wake up Waiting Transactions
-                // First go through the ConvertList
-                Iterator<WaitingInfo> convIt = lInfo.getIteratorOnConverter();
-                boolean checkWaiters = true;
-                while (convIt.hasNext()) {
-                    WaitingInfo nextConvInfo = convIt.next();
-                    if (nextConvInfo.isVictim()) {
-                        continue;
-                    }
-                    WaitEntry nextConv = nextConvInfo.getWaitingEntry();
-                    synchronized (nextConv) {
-                        int reqIx = nextConv.getIX();
-                        long convIx = lInfo.getTxId(reqIx);
-                        int curIx = lInfo.findInGrantedList(convIx, LockInfo.ANY_LOCK_MODE); // index
-                                                                                             // of
-                                                                                             // the
-                                                                                             // entry
-                                                                                             // for
-                                                                                             // the
-                                                                                             // (old)
-                                                                                             // already
-                                                                                             // granted
-                                                                                             // lock
-                        long convTxId = lInfo.getTxId(reqIx);
-                        int curConvMode = lInfo.getGrantedLockMode(convTxId, curIx);
-                        int reqConvMode = lInfo.getLockMode(reqIx);
-                        int updatedMask = lInfo.getUpdatedMask(convIx, curConvMode, curIx);
-                        if (lMtx.conflicts(updatedMask, reqConvMode)) {
-                            checkWaiters = false;
-                            break;
-                        }
-                        lInfo.removeFromGranted(convTxId, curConvMode, curIx);
-                        lInfo.addToGranted(convTxId, reqConvMode, LockInfo.NOT_FOUND);
-                        lInfo.prepareToRemoveFromConverters(convTxId, reqConvMode, reqIx);
-                        lmTables.getTxrInfo(convTxId).getContext().setStartWaitTime(TransactionContext.INVALID_TIME);
-                        nextConv.wakeUp();
-                        convIt.remove();
-                        nextConv.notifyAll();
-                    }
-                }
-
-                if (checkWaiters) {
-                    // Going through the WaitList
-                    Iterator<WaitingInfo> waitIt = lInfo.getIteratorOnWaiters();
-                    while (waitIt.hasNext()) {
-                        WaitingInfo nextWaiterInfo = waitIt.next();
-                        if (nextWaiterInfo.isVictim()) {
-                            continue;
-                        }
-                        WaitEntry nextWaiter = nextWaiterInfo.getWaitingEntry();
-                        synchronized (nextWaiter) {
-                            int waitIx = nextWaiter.getIX();
-                            long waitTxId = lInfo.getTxId(waitIx);
-                            int reqLock = lInfo.getLockMode(waitIx);
-                            int mask = lInfo.getMask();
-                            if (lMtx.conflicts(mask, reqLock)) {
-                                break;
-                            }
-                            lInfo.addToGranted(waitTxId, reqLock, LockInfo.NOT_FOUND);
-                            lInfo.prepareToRemoveFromWaiters(waitTxId, reqLock, waitIx);
-                            lmTables.getTxrInfo(waitTxId).getContext()
-                                    .setStartWaitTime(TransactionContext.INVALID_TIME);
-                            nextWaiter.wakeUp();
-                            waitIt.remove();
-                            nextWaiter.notifyAll();
-                        }
-                    }
-                }
-            }
-
-            context.setStartWaitTime(TransactionContext.INVALID_TIME);
-            if ((lmTables.removeTxrInfo(txId)) == null) { // Remove Txr's entry
-                                                          // from the
-                                                          // transactions' table
-                throw new ACIDException("Transaction " + txId + " Not found in transactions table for removal");
+        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 {
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Entry for Transaction " + txId + " removed from Txr Table (in release locks)");
+                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);
+                    }
                 }
             }
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Transaction " + txId + " released its locks successfully !");
+            waiter.increaseWaiterCount();
+            waiter.setFirstGetUp(true);
+
+            latchWaitNotify();
+            unlatchLockTable();
+            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");
             }
 
-            return true;
+            //waiter woke up -> remove/deallocate waiter object and abort if timeout
+            latchLockTable();
+
+            if (txnContext.getStatus() == TransactionContext.TIMED_OUT_STATUS || waiter.isVictim()) {
+                try {
+                    requestAbort(txnContext);
+                } finally {
+                    unlatchLockTable();
+                }
+            }
+
+            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);
+            }
+            try {
+                requestAbort(txnContext);
+            } finally {
+                unlatchLockTable();
+            }
         }
+
+        return waiterCount;
     }
 
-    private boolean isDeadlockFree(long txId, byte[] resourceId) {
-        return deadlockDetector.isSafeToAdd(txId, resourceId);
+    private boolean isDeadlockFree(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo, boolean isUpgrade) {
+        return deadlockDetector.isSafeToAdd(dLockInfo, eLockInfo, entityInfo, isDatasetLockInfo, isUpgrade);
     }
 
-    private void requestTxrAbort(TransactionContext context) throws ACIDException {
-        context.setStartWaitTime(TransactionContext.INVALID_TIME);
-        throw new ACIDException("Transaction " + context.getTransactionID()
+    private void requestAbort(TransactionContext txnContext) throws ACIDException {
+        txnContext.setStatus(TransactionContext.TIMED_OUT_STATUS);
+        txnContext.setStartWaitTime(TransactionContext.INVALID_TIME);
+        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 getDebugLockStatus() throws ACIDException {
-        String s = "\nLock Status (For Debug Purpose):\n";
-        synchronized (lmTables) {
-            Iterator<Long> txIdIt = getTxrInfoIterator();
-            while (txIdIt.hasNext()) {
-                long nextTxId = txIdIt.next();
-                TxrInfo nextInfoList = lmTables.getTxrInfo(nextTxId);
-                byte[] nextWaitOnRid = nextInfoList.getWaitOnRid();
-                String status = (nextWaitOnRid == null ? " ACTIVE" : " WAITING");
-                if ((nextWaitOnRid != null)) {
-                    LockInfo lInfo = (LockInfo) lmTables.getLockInfo(nextWaitOnRid);
-                    int wlModeIx = lInfo.findInConvertList(nextTxId, LockInfo.ANY_LOCK_MODE);
-                    if (wlModeIx == LockInfo.NOT_FOUND) {
-                        wlModeIx = lInfo.findInWaitList(nextTxId, LockInfo.ANY_LOCK_MODE);
-                    }
-                    int wlMode = lInfo.getLockMode(wlModeIx);
-                    String wLModeRep = (wlMode == 0 ? "S" : "X");
-                    status += " for " + wLModeRep + " lock";
-                }
-
-                String lockModes = "";
-                Iterator<TInfo> tInfoIt = nextInfoList.getIterator();
-                while (tInfoIt.hasNext()) {
-                    TInfo next = tInfoIt.next();
-                    int nextLockMode = next.getMode();
-                    lockModes += (nextLockMode == 0 ? "S" : "X");
-                    lockModes += ", ";
-                }
-                s += "Transaction: " + nextTxId + "\t- (Status: " + status + ") --> Granted Locks List: ( " + lockModes
-                        + " )\n";
-            }
-
-        }
+    public String prettyPrint() throws ACIDException {
+        StringBuilder s = new StringBuilder("\n########### LockManager Status #############\n");
         return s + "\n";
     }
 
     public void sweepForTimeout() throws ACIDException {
-        synchronized (lmTables) {
-            Iterator<Long> txrIt = lmTables.getIteratorOnTxrs();
-            while (txrIt.hasNext()) {
-                long nextTxrID = txrIt.next();
-                TxrInfo nextTxrInfo = lmTables.getTxrInfo(nextTxrID);
-                if (toutDetector.isVictim(nextTxrInfo)) {
-                    nextTxrInfo.getContext().setStatus(TransactionContext.TIMED_OUT_SATUS);
-                    LockInfo nextLockInfo = lmTables.getLockInfo(nextTxrInfo.getWaitOnRid());
-                    synchronized (nextLockInfo) {
-                        WaitingInfo nextVictim = nextLockInfo.getWaitingOnObject(nextTxrID, LockInfo.ANY_LOCK_MODE);
-                        nextVictim.setAsVictim();
-                        toutDetector.addToVictimsList(nextVictim.getWaitingEntry());
-                    }
-                }
+        JobInfo jobInfo;
+        int waiterObjId;
+        LockWaiter waiterObj;
+
+        latchLockTable();
+
+        Iterator<Entry<JobId, JobInfo>> iter = jobHT.entrySet().iterator();
+        while (iter.hasNext()) {
+            Map.Entry<JobId, JobInfo> pair = (Map.Entry<JobId, JobInfo>) iter.next();
+            jobInfo = pair.getValue();
+            waiterObjId = jobInfo.getFirstWaitingResource();
+            while (waiterObjId != -1) {
+                waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+                toutDetector.checkAndSetVictim(waiterObj);
+                waiterObjId = waiterObj.getNextWaiterObjId();
             }
         }
-    }
 
-    public LockInfo getLockInfo(byte[] resourceID) {
-        return lmTables.getLockInfo(resourceID);
-    }
-
-    public TxrInfo getTxrInfo(long txrId) {
-        return lmTables.getTxrInfo(txrId);
+        unlatchLockTable();
     }
 }
 
-class LMTables {
-    /**
-     * An instance of this class mainly manages and synchronizes the access to
-     * the lock manager hash tables
-     */
+class ConsecutiveWakeupContext {
+    private boolean IS;
+    private boolean IX;
+    private boolean S;
+    private boolean X;
 
-    private ILockHashTable<byte[], LockInfo> resToLInfo; // mapping from
-                                                         // resourceID to
-                                                         // information about
-                                                         // the locks on it
-    private ILockHashTable<Long, TxrInfo> txToTxrInfo; // mapping from
-                                                       // transactionID to
-                                                       // informations about its
-                                                       // lock(s)
-
-    public LMTables(int initialSize) {
-        resToLInfo = new ResourcesHT(initialSize);
-        txToTxrInfo = new TransactionsHT(initialSize);
+    public void reset() {
+        IS = false;
+        IX = false;
+        S = false;
+        X = false;
     }
 
-    public LockInfo getLockInfo(byte[] resourceId) {
-        return resToLInfo.get(resourceId);
-    }
+    public boolean isCompatible(byte lockMode) {
+        switch (lockMode) {
+            case LockMode.IX:
+                return !S && !X;
 
-    public void putLockInfo(byte[] resourceID, LockInfo lInfo) {
-        resToLInfo.put(resourceID, lInfo);
-    }
+            case LockMode.IS:
+                return !X;
 
-    public TxrInfo getTxrInfo(long txrId) {
-        return txToTxrInfo.get(txrId);
-    }
+            case LockMode.X:
+                return !IS && !IX && !S && !X;
 
-    public void putTxrInfo(long txrId, TxrInfo txrInfo) {
-        txToTxrInfo.put(txrId, txrInfo);
-    }
+            case LockMode.S:
+                return !IX && !X;
 
-    public TxrInfo removeTxrInfo(long txId) {
-        return txToTxrInfo.remove(txId);
-    }
-
-    public int getTxrTableSize() {
-        return txToTxrInfo.getKeysetSize();
-    }
-
-    public Iterator<Long> getIteratorOnTxrs() {
-        return ((TransactionsHT) txToTxrInfo).getIteratorOnTxs();
-    }
-
-}
-
-class ResourcesHT implements ILockHashTable<byte[], LockInfo> {
-
-    private Hashtable<LockTag, LockInfo> table;
-    private LockTag tag;
-
-    public ResourcesHT(int initCapacity) {
-        this.table = new Hashtable<LockTag, LockInfo>(initCapacity);
-        this.tag = new LockTag(null);
-    }
-
-    @Override
-    public synchronized void put(byte[] rsId, LockInfo info) {
-        table.put(new LockTag(rsId), (LockInfo) info);
-    }
-
-    @Override
-    public synchronized LockInfo get(byte[] rsId) {
-        tag.setRsId(rsId);
-        return (table.get(tag));
-    }
-
-    @Override
-    public LockInfo remove(byte[] rsId) {
-        tag.setRsId(rsId);
-        return (table.remove(tag));
-    }
-
-    @Override
-    public int getKeysetSize() {
-        return table.size();
-    }
-
-}
-
-class TransactionsHT implements ILockHashTable<Long, TxrInfo> {
-
-    private Hashtable<Long, TxrInfo> table;
-
-    public TransactionsHT(int initCapacity) {
-        this.table = new Hashtable<Long, TxrInfo>(initCapacity);
-    }
-
-    @Override
-    public synchronized void put(Long key, TxrInfo value) {
-        table.put(key, value);
-
-    }
-
-    @Override
-    public synchronized TxrInfo get(Long key) {
-        return (table.get(key));
-    }
-
-    public Iterator<Long> getIteratorOnTxs() {
-        return table.keySet().iterator();
-    }
-
-    @Override
-    public TxrInfo remove(Long key) {
-        return table.remove(key);
-    }
-
-    @Override
-    public int getKeysetSize() {
-        return table.size();
-    }
-
-}
-
-class LockTag {
-    /**
-     * Used as a wrapper around byte[], which is used as the key for the
-     * hashtables
-     */
-
-    byte[] rsId;
-
-    public LockTag(byte[] rsId) {
-        setRsId(rsId);
-    }
-
-    public void setRsId(byte[] rsId) {
-        this.rsId = rsId;
-    }
-
-    @Override
-    public int hashCode() {
-        return Arrays.hashCode(rsId);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if ((o == null) || !(o instanceof LockTag)) {
-            return false;
-        }
-        return Arrays.equals(((LockTag) o).rsId, this.rsId);
-    }
-}
-
-class WaitObjectManager {
-    /**
-     * Manages the set of waiting objects (objects used to manage waiters) to
-     * avoid object/garbage creation as much as possible
-     */
-    final int EOL = -1;
-    ArrayList<WaitEntry> list;
-    AtomicInteger max;
-    int nextFree;
-
-    public WaitObjectManager() {
-        list = new ArrayList<WaitEntry>();
-        nextFree = EOL;
-        max = new AtomicInteger(0);
-    }
-
-    public WaitEntry allocate() throws ACIDException {
-        WaitEntry o = null;
-        synchronized (list) {
-            if (nextFree == EOL) {
-                o = new WaitEntry(max.getAndIncrement(), LockInfo.UNKNOWN_IX, EOL);
-                list.add(o);
-                return o;
-            }
-            o = list.get(nextFree);
-            nextFree = o.getNext();
-            o.setNext(EOL);
-        }
-        return o;
-    }
-
-    public void deAllocate(Object o) {
-        synchronized (list) {
-            ((WaitEntry) o).setNext(nextFree);
-            nextFree = ((WaitEntry) o).getId();
+            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");
+        }
+
+    }
+
 }
 
-class WaitEntry {
-    /**
-     * Captures the information about a waiting transaction
-     */
+/******************************************
+ * datasetResourceHT
+ ******************************************/
+/*
+class DatasetId implements Serializable {
+    int id;
 
-    private int id; // ID of this object (used for managing the waiting objects
-                    // and recycling them)
-    private int eix; // index of the entry corresponding to the waiting
-                     // transaction
-    private boolean shouldWait; // whether the waiter needs to continue its
-                                // waiting or not
-    private int next; // The next waitEntry in the chain of wait Entries (used
-                      // for managing the waiting objects and recycling them)
-
-    public WaitEntry(int id, int eix, int next) {
+    public DatasetId(int id) {
         this.id = id;
-        this.eix = eix;
-        shouldWait = true;
-        this.next = next;
     }
 
-    public int getIX() {
-        return eix;
-    }
-
-    public void setIx(int eix) {
-        this.eix = eix;
+    public void setId(int id) {
+        this.id = id;
     }
 
     public int getId() {
         return id;
     }
 
-    public void setNext(int n) {
-        next = n;
+    @Override
+    public int hashCode() {
+        return id;
     }
 
-    public int getNext() {
-        return next;
+    @Override
+    public boolean equals(Object o) {
+        if ((o == null) || !(o instanceof DatasetId)) {
+            return false;
+        }
+        return ((DatasetId) o).id == this.id;
     }
-
-    public boolean needWait() {
-        return shouldWait;
-    }
-
-    public void wakeUp() {
-        this.shouldWait = false;
-    }
-
-    public void setForWait() {
-        this.shouldWait = true;
-    }
-}
\ No newline at end of file
+};
+*/
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
new file mode 100644
index 0000000..063194b
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
@@ -0,0 +1,617 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+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 edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+
+public class LockManagerDeterministicUnitTest {
+
+    public static void main(String args[]) throws ACIDException, IOException {
+        //initialize controller thread
+        String requestFileName = new String(
+                "src/main/java/edu/uci/ics/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;
+    TransactionProvider txnProvider;
+    WorkerReadyQueue workerReadyQueue;
+    ArrayList<LockRequest> requestList;
+    ArrayList<ArrayList<Integer>> expectedResultList;
+    int resultListIndex;
+    LockManager lockMgr;
+    String requestFileName;
+    long defaultWaitTime;
+
+    public LockRequestController(String requestFileName) throws ACIDException {
+        this.txnProvider = new TransactionProvider("LockManagerPredefinedUnitTest");;
+        this.workerReadyQueue = new WorkerReadyQueue();
+        this.requestList = new ArrayList<LockRequest>();
+        this.expectedResultList = new ArrayList<ArrayList<Integer>>();
+        this.lockMgr = (LockManager) 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 ***");
+        }
+    }
+
+    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;
+    TransactionProvider txnProvider;
+    ILockManager lockMgr;
+    WorkerReadyQueue workerReadyQueue;
+    LockRequest lockRequest;
+    boolean needWait;
+    boolean isAwaken;
+    boolean isDone;
+
+    public LockRequestWorker(TransactionProvider txnProvider, WorkerReadyQueue workerReadyQueue, String threadName) {
+        this.txnProvider = txnProvider;
+        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.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
+                    if (lockRequest.txnContext.getTxnState() != TransactionState.ABORTED) {
+                        lockRequest.txnContext.setTxnState(TransactionState.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.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);
+    }
+}
+
+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/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
new file mode 100644
index 0000000..7add782
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerRandomUnitTest.java
@@ -0,0 +1,610 @@
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.util.ArrayList;
+import java.util.Random;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+
+/**
+ * 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 {
+        int i;
+        TransactionProvider txnProvider = new TransactionProvider("LockManagerRandomUnitTest");
+        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);
+        }
+    }
+
+    private static void generateEntityLockThread(TransactionProvider 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(TransactionProvider 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(TransactionProvider 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(TransactionProvider txnProvider) {
+        try {
+            return new TransactionContext(new JobId(jobId++), txnProvider);
+        } catch (ACIDException e) {
+            e.printStackTrace();
+            return null;
+        }
+    }
+
+}
+
+class LockRequestProducer implements Runnable {
+
+    private static final long serialVersionUID = -3191274684985609965L;
+    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() != TransactionState.ABORTED) {
+                        txnContext.setTxnState(TransactionState.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.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
+                        if (request.txnContext.getTxnState() != TransactionState.ABORTED) {
+                            request.txnContext.setTxnState(TransactionState.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.getStatus() == TransactionContext.TIMED_OUT_STATUS) {
+                        if (request.txnContext.getTxnState() != TransactionState.ABORTED) {
+                            request.txnContext.setTxnState(TransactionState.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.txnContext);
+                break;
+            case RequestType.RELEASE_LOCKS:
+                lockMgr.releaseLocks(request.txnContext);
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported lock method");
+        }
+        try {
+            Thread.sleep((long)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 TransactionContext 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,
+            TransactionContext 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;
+    }
+
+    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");
+        switch (lockMode) {
+            case LockMode.S:
+                s.append("S");
+                break;
+            case LockMode.X:
+                s.append("X");
+                break;
+            case LockMode.IS:
+                s.append("IS");
+                break;
+            case LockMode.IX:
+                s.append("IX");
+                break;
+            default:
+                throw new UnsupportedOperationException("Unsupported lock mode");
+        }
+        s.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/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMatrix.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMatrix.java
deleted file mode 100644
index 651909d..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockMatrix.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package edu.uci.ics.asterix.transaction.management.service.locking;
-
-/**
- * @author pouria An implementation of the ILockMatrix Each lock mode is shown
- *         as an integer. More specifically:
- *         - i-th entry of the conflictTable corresponds to the i-th lock mode
- *         and it shows the conflicting mask of that mode. j-th bit of the i-th
- *         entry is 1 if and only if i-th lock mode conflicts with the j-th lock
- *         mode.
- *         - i-th entry of the conversionTable corresponds to the i-th lock mode
- *         and it shows whether going from that mode to a new mode is actually a
- *         conversion or not. j-th bit of the i-th entry is 1 if and only if
- *         j-th lock mode is "stronger" than the i-th mode, i.e. lock changing
- *         from i-th mode to the j-th mode is actually a conversion.
- */
-public class LockMatrix implements ILockMatrix {
-
-    int[] conflictTable;
-    int[] conversionTable;
-
-    public LockMatrix(int[] confTab, int[] convTab) {
-        this.conflictTable = confTab;
-        this.conversionTable = convTab;
-    }
-
-    @Override
-    public boolean conflicts(int reqMask, int lockMode) {
-        return ((reqMask & conflictTable[lockMode]) != 0);
-    }
-
-    @Override
-    public boolean isConversion(int currentLockMode, int reqLockMode) {
-        return ((conversionTable[currentLockMode] & (0x01 << reqLockMode)) != 0);
-    }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile
new file mode 100644
index 0000000..fc2a883
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile
@@ -0,0 +1,20 @@
+T1 L	J1 D1 E1 S
+T3 L	J3 D1 E-1 S
+T2 L	J2 D1 E-1 X
+T4 L	J4 D1 E1 S
+T0 CST	1 3 -1
+T1 L	J1 D1 E2 X
+T0 CST	3 -1
+T3 RL	J3 D1 E-1 S
+T0 CST	1 3 -1
+T1 UL	J1 D1 E1 S
+T0 CST	1 3 -1
+T1 UL	J1 D1 E2 X
+T0 CST	1 2 3 -1
+T3 END
+T1 END
+T2 RL	J2 D1 E-1 X
+T2 END
+T0 CST	4 -1
+T4 UL	J4 D1 E1 S
+T4 END
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestTracker.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestTracker.java
new file mode 100644
index 0000000..ba47b5a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestTracker.java
@@ -0,0 +1,57 @@
+package edu.uci.ics.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();
+    }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiter.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiter.java
new file mode 100644
index 0000000..2015aec
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiter.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.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/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
new file mode 100644
index 0000000..dbe76ff
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
@@ -0,0 +1,377 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+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;
+        boolean bContiguous = true;
+        int decreaseCount = 0;
+        int size = pArray.size();
+        int maxDecreaseCount = size / 2;
+        ChildLockWaiterArrayManager child;
+        for (i = size - 1; i >= 0; i--) {
+            child = pArray.get(i);
+            if (child.isEmpty() || child.isDeinitialized()) {
+                if (bContiguous) {
+                    pArray.remove(i);
+                    if (++decreaseCount == maxDecreaseCount) {
+                        break;
+                    }
+                } else {
+                    bContiguous = false;
+                    if (child.isEmpty()) {
+                        child.deinitialize();
+                        if (++decreaseCount == maxDecreaseCount) {
+                            break;
+                        }
+                    }
+                }
+            } else {
+                bContiguous = false;
+            }
+        }
+
+        //reset allocChild when the child is removed or deinitialized.
+        size = pArray.size();
+        if (allocChild >= size || pArray.get(allocChild).isDeinitialized()) {
+            //set allocChild to any initialized one.
+            //It is guaranteed that there is at least one initialized child.
+            for (i = 0; i < size; i++) {
+                if (!pArray.get(i).isDeinitialized()) {
+                    allocChild = i;
+                    break;
+                }
+            }
+        }
+    }
+
+    public String prettyPrint() {
+        StringBuilder s = new StringBuilder("\n########### LockWaiterManager Status #############\n");
+        int size = pArray.size();
+        ChildLockWaiterArrayManager child;
+        LockWaiter waiter;
+
+        for (int i = 0; i < size; i++) {
+            child = pArray.get(i);
+            if (child.isDeinitialized()) {
+                continue;
+            }
+            s.append("child[" + i + "]: occupiedSlots:" + child.getNumOfOccupiedSlots());
+            s.append(" freeSlotNum:" + child.getFreeSlotNum() + "\n");
+            for (int j = 0; j < ChildLockWaiterArrayManager.NUM_OF_SLOTS; j++) {
+                waiter = child.getLockWaiter(j);
+                s.append(j).append(": ");
+                s.append("\t" + waiter.getEntityInfoSlot());
+                s.append("\t" + waiter.needWait());
+                s.append("\t" + waiter.isVictim());
+                s.append("\n");
+            }
+            s.append("\n");
+        }
+        return s.toString();
+    }
+    
+    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;
+    }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
new file mode 100644
index 0000000..be9c080
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/PrimitiveIntHashMap.java
@@ -0,0 +1,592 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.transaction.management.service.locking;
+
+import java.util.ArrayList;
+
+/**
+ * PrimitiveIntHashMap supports primitive int type as key and value.
+ * The hash map grows when the available slots in a bucket are overflowed.
+ * Also, the hash map shrinks according to the following shrink policy.
+ * : Shrink when the resource under-utilization lasts for a certain threshold time. 
+ *   
+ * @author kisskys
+ *
+ */
+public class PrimitiveIntHashMap {
+    private final int CHILD_BUCKETS; //INIT_NUM_OF_BUCKETS;
+    private final int NUM_OF_SLOTS; //NUM_OF_SLOTS_IN_A_BUCKET;
+    private final int SHRINK_TIMER_THRESHOLD;
+    
+    private int occupiedSlots;
+    private ArrayList<ChildIntArrayManager> pArray; //parent array
+    private int hashMod;
+    private long shrinkTimer;
+    private boolean isShrinkTimerOn;
+    private int iterBucketIndex;
+    private int iterSlotIndex;
+    private int iterChildIndex;
+    private KeyValuePair iterPair;
+
+//    ////////////////////////////////////////////////
+//    // begin of unit test
+//    ////////////////////////////////////////////////
+//
+//    /**
+//     * @param args
+//     */
+//    public static void main(String[] args) {
+//        int i, j;
+//        int k = 0;
+//        int num = 5;
+//        int key[] = new int[500];
+//        int val[] = new int[500];
+//        KeyValuePair pair;
+//        PrimitiveIntHashMap map = new PrimitiveIntHashMap(1<<4, 1<<3, 5);
+//        
+//        for (j=0; j < num; j++) {
+//            
+//            k += 100;
+//            //generate data
+//            for (i=0; i < k; i++) {
+//                key[i] = i;
+//                val[i] = i;
+//            }
+//            
+//            //put data to map
+//            for (i=0; i < k-30; i++) {
+//                map.put(key[i], val[i]);
+//            }
+//            
+//            //put data to map
+//            for (i=0; i < k-30; i++) {
+//                map.put(key[i], val[i]);
+//            }
+//            
+//            map.beginIterate();
+//            pair = map.getNextKeyValue();
+//            i = 0;
+//            while (pair != null) {
+//                i++;
+//                System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
+//                pair = map.getNextKeyValue();
+//            }
+//            
+//            //System.out.println(map.prettyPrint());
+//            
+//            for (i=k-20; i< k; i++) { //skip X70~X79
+//                map.put(key[i], val[i]);
+//            }
+//            
+//            System.out.println(map.prettyPrint());
+//            
+//            //remove data to map
+//            for (i=0; i < k-10; i++) { 
+//                map.remove(key[i]);
+//                try {
+//                    Thread.currentThread().sleep(1);
+//                } catch (InterruptedException e) {
+//                    e.printStackTrace();
+//                }
+//            }
+//            
+//            map.beginIterate();
+//            pair = map.getNextKeyValue();
+//            i = 0;
+//            while (pair != null) {
+//                i++;
+//                System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
+//                pair = map.getNextKeyValue();
+//            }
+//            
+//            //remove data to map
+//            for (i=0; i < k-10; i++) { 
+//                map.remove(key[i]);
+//                try {
+//                    Thread.currentThread().sleep(1);
+//                } catch (InterruptedException e) {
+//                    // TODO Auto-generated catch block
+//                    e.printStackTrace();
+//                }
+//            }
+//            
+//            System.out.println(map.prettyPrint());
+//            
+//            //get data from map
+//            for (i=0; i < k; i++) {
+//                System.out.println(""+i+"=> key:"+ key[i] + ", val:"+val[i] +", result: " + map.get(key[i]));  
+//            }
+//        }
+//        
+//        map.beginIterate();
+//        pair = map.getNextKeyValue();
+//        i = 0;
+//        while (pair != null) {
+//            i++;
+//            System.out.println("["+i+"] key:"+ pair.key + ", val:"+ pair.value);
+//            pair = map.getNextKeyValue();
+//        }
+//    }
+//
+//    ////////////////////////////////////////////////
+//    // end of unit test
+//    ////////////////////////////////////////////////
+    
+    public PrimitiveIntHashMap() {
+        CHILD_BUCKETS = 1<<9; //INIT_NUM_OF_BUCKETS;
+        NUM_OF_SLOTS = 1<<3; //NUM_OF_SLOTS_IN_A_BUCKET;
+        SHRINK_TIMER_THRESHOLD = 120000; //2min
+        pArray = new ArrayList<ChildIntArrayManager>();
+        pArray.add(new ChildIntArrayManager(this));
+        hashMod = CHILD_BUCKETS;
+        occupiedSlots = 0;
+        iterPair = new KeyValuePair();
+    }
+    
+    public PrimitiveIntHashMap(int childBuckets, int numOfSlots, int shrinkTimerThreshold) {
+        CHILD_BUCKETS = childBuckets;
+        NUM_OF_SLOTS = numOfSlots;
+        SHRINK_TIMER_THRESHOLD = shrinkTimerThreshold;
+        pArray = new ArrayList<ChildIntArrayManager>();
+        pArray.add(new ChildIntArrayManager(this));
+        hashMod = CHILD_BUCKETS;
+        occupiedSlots = 0;
+        iterPair = new KeyValuePair();
+    }
+    
+    public void put(int key, int value) {
+        int growCount = 0;
+        int bucketNum = hash(key);
+        ChildIntArrayManager child = pArray.get(bucketNum/CHILD_BUCKETS);
+        while (child.isFull(bucketNum%CHILD_BUCKETS)) {
+            growHashMap();
+            bucketNum = hash(key);
+            child = pArray.get(bucketNum/CHILD_BUCKETS);
+            if (growCount > 2) {
+                //changeHashFunc();
+            }
+            growCount++;
+        }
+        occupiedSlots += child.put(bucketNum%CHILD_BUCKETS, key, value, false);
+    }
+    
+    public void upsert (int key, int value) {
+        int growCount = 0;
+        int bucketNum = hash(key);
+        ChildIntArrayManager child = pArray.get(bucketNum/CHILD_BUCKETS);
+        while (child.isFull(bucketNum%CHILD_BUCKETS)) {
+            growHashMap();
+            bucketNum = hash(key);
+            child = pArray.get(bucketNum/CHILD_BUCKETS);
+            if (growCount > 2) {
+                //changeHashFunc();
+            }
+            growCount++;
+        }
+        occupiedSlots += child.put(bucketNum%CHILD_BUCKETS, key, value, true);
+    }
+    
+    private int hash(int key) {
+        return key%hashMod;
+    }
+    
+    private void growHashMap() {
+        int size = pArray.size();
+        int i; 
+        
+        //grow buckets by adding more child
+        for (i=0; i<size; i++) { 
+            pArray.add(new ChildIntArrayManager(this));
+        }
+        
+        //increase hashMod
+        hashMod *= 2;
+        
+        //re-hash
+        rehash(0, size, hashMod/2);
+    }
+    
+    private void shrinkHashMap() {
+        int size = pArray.size();
+        int i;
+        
+        //decrease hashMod
+        hashMod /= 2;
+        
+        //re-hash
+        rehash(size/2, size, hashMod*2);
+        
+        //shrink buckets by removing child(s)
+        for (i=size-1; i>=size/2;i--) {
+            pArray.remove(i);
+        }
+    }
+    
+    private void rehash(int begin, int end, int oldHashMod) {
+        int i, j, k;
+        int key, value;
+        ChildIntArrayManager child;
+        
+        //re-hash
+        for (i=begin; i<end; i++) {
+            child = pArray.get(i);
+            for (j=0; j<CHILD_BUCKETS; j++) {
+                if (child.cArray[j][0] == 0) {
+                    continue;
+                }
+                for (k=1; k<NUM_OF_SLOTS; k++) {
+                    //if the hashValue of the key is different, then re-hash it.
+                    key = child.cArray[j][k*2];
+                    if (hash(key) != key%oldHashMod) {
+                        value = child.cArray[j][k*2+1];
+                        //remove existing key and value
+                        //Notice! To avoid bucket iteration, child.remove() is not used.
+                        child.cArray[j][k*2] = -1;
+                        child.cArray[j][0]--;
+                        //re-hash it 
+                        pArray.get(hash(key)/CHILD_BUCKETS).put(hash(key)%CHILD_BUCKETS, key, value, false);
+                    }
+                }
+            }
+        }
+    }
+    
+//    private void changeHashFunc() {
+//        //TODO need to implement.
+//        throw new UnsupportedOperationException("changeHashFunc() not implemented");
+//    }
+    
+    public int get(int key) {
+        int bucketNum = hash(key);
+        return pArray.get(bucketNum/CHILD_BUCKETS).get(bucketNum%CHILD_BUCKETS, key);
+    }
+    
+    public void remove(int key) {
+        int bucketNum = hash(key);
+        occupiedSlots -= pArray.get(bucketNum/CHILD_BUCKETS).remove(bucketNum%CHILD_BUCKETS, key);
+        
+        if (needShrink()) {
+            shrinkHashMap();
+        }
+    }
+    
+    /**
+     * Shrink policy:
+     * Shrink when the resource under-utilization lasts for a certain amount of time. 
+     * @return
+     */
+    private boolean needShrink() {
+        int size = pArray.size();
+        int usedSlots = occupiedSlots;
+        if (usedSlots == 0) {
+            usedSlots = 1;
+        }
+        if (size > 1 && size*CHILD_BUCKETS*NUM_OF_SLOTS/usedSlots >= 3 && isSafeToShrink()) {
+            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;
+    }
+    
+    private boolean isSafeToShrink() {
+        int i, j;
+        int size = pArray.size();
+        //Child: 0, 1, 2, 3, 4, 5, 6, 7 
+        //[HChild(Head Child):0 and TChild(Tail Child): 4], [1(H),5(T)], [2(H),6(T)] and so on. 
+        //When the map shrinks, the sum of occupied slots in H/TChild should not exceed the NUM_OF_SLOTS-1.
+        //Then it is safe to shrink. Otherwise, unsafe.
+        ChildIntArrayManager HChild, TChild; 
+        
+        for (i=0; i<size/2; i++){
+            HChild = pArray.get(i);
+            TChild = pArray.get(size/2+i);
+            for (j=0; j<CHILD_BUCKETS; j++) {
+                if (HChild.cArray[j][0] + TChild.cArray[j][0] > NUM_OF_SLOTS-1) {
+                    return false;
+                }
+            }
+        }
+        return true;
+    }
+    
+    public String prettyPrint() {
+        StringBuilder s = new StringBuilder("\n########### PrimitiveIntHashMap Status #############\n");
+        ChildIntArrayManager child;
+        int i, j, k;
+        int size = pArray.size();
+        for (i=0; i<size;i++) {
+            child = pArray.get(i);
+            s.append("child[").append(i).append("]\n");
+            for (j=0; j<CHILD_BUCKETS;j++) {
+                s.append(j).append(" ");
+                for (k=0; k<NUM_OF_SLOTS;k++) {
+                    s.append("[").append(child.cArray[j][k*2]).append(",").append(child.cArray[j][k*2+1]).append("] ");
+                }
+                s.append("\n");
+            }
+        }
+        return s.toString();
+    }
+    
+    public int getNumOfSlots() {
+        return NUM_OF_SLOTS;
+    }
+    
+    public int getNumOfChildBuckets() {
+        return CHILD_BUCKETS;
+    }
+    
+    public void clear(boolean needShrink) {
+        int size = pArray.size();
+        for (int i=size-1; i >= 0; i--) {
+            if (needShrink && i != 0) {
+                pArray.remove(i);
+            } else {
+                pArray.get(i).clear();
+            }
+        }
+        occupiedSlots = 0;
+    }
+    
+    ///////////////////////////////////////
+    // iterate method
+    ///////////////////////////////////////
+    
+    public void beginIterate() {
+        iterChildIndex = 0;
+        iterBucketIndex = 0;
+        iterSlotIndex = 1;
+    }
+    
+    public KeyValuePair getNextKeyValue() {
+        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
+            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
+                if (iterSlotIndex ==1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
+                    continue;
+                }
+                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
+                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2];
+                    if (iterPair.key == -1) {
+                        continue;
+                    }
+                    iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2+1];
+                    iterSlotIndex++;
+                    return iterPair;
+                }
+            }
+        }
+        return null;
+    }
+    
+    public int getNextKey() {
+        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
+            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
+                if (iterSlotIndex ==1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
+                    continue;
+                }
+                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
+                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2];
+                    if (iterPair.key == -1) {
+                        continue;
+                    }
+                    iterSlotIndex++;
+                    return iterPair.key;
+                }
+            }
+        }
+        return -1;
+    }
+    
+    public int getNextValue() {
+        for (; iterChildIndex < pArray.size(); iterChildIndex++, iterBucketIndex = 0) {
+            for (; iterBucketIndex < CHILD_BUCKETS; iterBucketIndex++, iterSlotIndex = 1) {
+                if (iterSlotIndex ==1 && pArray.get(iterChildIndex).cArray[iterBucketIndex][0] == 0) {
+                    continue;
+                }
+                for (; iterSlotIndex < NUM_OF_SLOTS; iterSlotIndex++) {
+                    iterPair.key = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2];
+                    if (iterPair.key == -1) {
+                        continue;
+                    }
+                    iterPair.value = pArray.get(iterChildIndex).cArray[iterBucketIndex][iterSlotIndex*2+1];
+                    iterSlotIndex++;
+                    return iterPair.value;
+                }
+            }
+        }
+        return -1;
+    }
+    
+    public static class KeyValuePair {
+        public int key;
+        public int value; 
+    }
+}
+
+class ChildIntArrayManager {
+    private final int DIM1_SIZE; 
+    private final int DIM2_SIZE; 
+    private final int NUM_OF_SLOTS;
+    public int[][] cArray; //child array
+    
+    public ChildIntArrayManager(PrimitiveIntHashMap parentHashMap) {
+        DIM1_SIZE = parentHashMap.getNumOfChildBuckets();
+        DIM2_SIZE = parentHashMap.getNumOfSlots() * 2; //2: Array of [key, value] pair
+        NUM_OF_SLOTS = parentHashMap.getNumOfSlots() ;
+        initialize();
+    }
+
+    private void initialize() {
+        cArray = new int[DIM1_SIZE][DIM2_SIZE];
+        int i, j;
+        for (i = 0; i < DIM1_SIZE; i++) {
+            //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
+            //cArray[i][1] is not used.
+            cArray[i][0] = 0;
+            for (j = 1; j < NUM_OF_SLOTS; j++) {
+                cArray[i][j*2] = -1; // -1 represent that the slot is empty
+            }
+        }
+    }
+    
+    public void clear() {
+        int i, j;
+        for (i = 0; i < DIM1_SIZE; i++) {
+            //cArray[i][0] is used as a counter to count how many slots are used in this bucket.
+            //cArray[i][1] is not used.
+            if (cArray[i][0] == 0) {
+                continue;
+            }
+            cArray[i][0] = 0;
+            for (j = 1; j < NUM_OF_SLOTS; j++) {
+                cArray[i][j*2] = -1; // -1 represent that the slot is empty
+            }
+        }
+    }
+    
+    public void deinitialize() {
+        cArray = null;
+    }
+    
+    public void allocate() {
+        initialize();
+    }
+
+    public boolean isFull(int bucketNum) {
+        return cArray[bucketNum][0] == NUM_OF_SLOTS-1;
+    }
+    
+    public boolean isEmpty(int bucketNum) {
+        return cArray[bucketNum][0] == 0;
+    }
+
+    /**
+     * Put key,value into a slot in the bucket if the key doesn't exist.
+     * Update value if the key exists and if isUpsert is true
+     * No need to call get() to check the existence of the key before calling put().
+     * Notice! Caller should make sure that there is an available slot.
+     * 
+     * @param bucketNum
+     * @param key
+     * @param value
+     * @param isUpsert
+     * @return 1 for new insertion, 0 for key duplication 
+     */
+    public int put(int bucketNum, int key, int value, boolean isUpsert) {
+        int i;
+        int emptySlot=-1;
+
+        if (cArray[bucketNum][0] == 0) {
+            cArray[bucketNum][2] = key;
+            cArray[bucketNum][3] = value;
+            cArray[bucketNum][0]++;
+            return 1;
+        }
+
+        for (i = 1; i < NUM_OF_SLOTS; i++) {
+            if (cArray[bucketNum][i*2] == key) {
+                if (isUpsert) {
+                    cArray[bucketNum][emptySlot*2+1] = value;
+                }
+                return 0;
+            }
+            else if (cArray[bucketNum][i*2] == -1) {
+                emptySlot = i;
+            }
+        }
+        
+        if (emptySlot == -1) {
+            throw new UnsupportedOperationException("error");
+        }
+        
+        cArray[bucketNum][emptySlot*2] = key;
+        cArray[bucketNum][emptySlot*2+1] = value;
+        cArray[bucketNum][0]++;
+        return 1;
+    }
+
+    public int get(int bucketNum, int key) {
+        int i;
+        
+        if (cArray[bucketNum][0] == 0) {
+            return -1;
+        }
+
+        for (i = 1; i < NUM_OF_SLOTS; i++) {
+            if (cArray[bucketNum][i*2] == key) {
+                return cArray[bucketNum][i*2+1];
+            }
+        }
+        return -1;
+    }
+    
+    /**
+     * remove key if it exists. Otherwise, ignore it.
+     * @param bucketNum
+     * @param key
+     * @return 1 for success, 0 if the key doesn't exist 
+     */
+    public int remove(int bucketNum, int key) {
+        int i;
+        
+        if (cArray[bucketNum][0] == 0) {
+            return 0;
+        }
+
+        for (i = 1; i < NUM_OF_SLOTS; i++) {
+            if (cArray[bucketNum][i*2] == key) {
+                cArray[bucketNum][i*2] = -1;
+                cArray[bucketNum][0]--;
+                return 1;
+            }
+        }
+        
+        return 0;
+    }
+}
+
+
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
index 6c391f4..699fd74 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TimeOutDetector.java
@@ -6,7 +6,8 @@
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 
 /**
- * @author pouria Any transaction which has been waiting for a lock for more
+ * @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
@@ -15,14 +16,14 @@
 
 public class TimeOutDetector {
     static final long TIME_OUT_THRESHOLD = 60000;
-    static final long SWEEP_PERIOD = 120000;
+    static final long SWEEP_PERIOD = 10000;//120000;
 
     LockManager lockMgr;
     Thread trigger;
-    LinkedList<WaitEntry> victimsWObjs;
+    LinkedList<LockWaiter> victimList;
 
     public TimeOutDetector(LockManager lockMgr) {
-        this.victimsWObjs = new LinkedList<WaitEntry>();
+        this.victimList = new LinkedList<LockWaiter>();
         this.lockMgr = lockMgr;
         this.trigger = new Thread(new TimeoutTrigger(this));
         trigger.setDaemon(true);
@@ -30,33 +31,29 @@
     }
 
     public void sweep() throws ACIDException {
-        victimsWObjs.clear();
-        lockMgr.sweepForTimeout(); // Initiates the time-out sweeping process
-                                   // from the lockManager
+        victimList.clear();
+        // Initiates the time-out sweeping process
+        // from the lockManager
+        lockMgr.sweepForTimeout();
         notifyVictims();
     }
 
-    public boolean isVictim(TxrInfo txrInfo) {
-        long sWTime = txrInfo.getStartWaitTime();
-        int status = txrInfo.getContext().getStatus();
-        return (status != TransactionContext.TIMED_OUT_SATUS && sWTime != TransactionContext.INVALID_TIME && (System
-                .currentTimeMillis() - sWTime) >= TIME_OUT_THRESHOLD);
-    }
-
-    public void addToVictimsList(WaitEntry wEntry) {
-        victimsWObjs.add(wEntry);
+    public void checkAndSetVictim(LockWaiter waiterObj) {
+        if (System.currentTimeMillis() - waiterObj.getBeginWaitTime() >= TIME_OUT_THRESHOLD) {
+            waiterObj.setVictim(true);
+            waiterObj.setWait(false);
+            victimList.add(waiterObj);
+        }
     }
 
     private void notifyVictims() {
-        for (WaitEntry w : victimsWObjs) {
-            synchronized (w) {
-                w.wakeUp();
-                w.notifyAll();
+        for (LockWaiter waiterObj : victimList) {
+            synchronized (waiterObj) {
+                waiterObj.notifyAll();
             }
         }
-        victimsWObjs.clear();
+        victimList.clear();
     }
-
 }
 
 class TimeoutTrigger implements Runnable {
@@ -79,9 +76,6 @@
             } catch (ACIDException e) {
                 throw new IllegalStateException(e);
             }
-
         }
-
     }
-
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TxrInfo.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TxrInfo.java
deleted file mode 100644
index 097c3ec..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/TxrInfo.java
+++ /dev/null
@@ -1,130 +0,0 @@
-package edu.uci.ics.asterix.transaction.management.service.locking;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Iterator;
-
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-
-/**
- * @author pouria An instance shows information about all the locks a specific
- *         transaction is holding and/or is waiting on (whether for conversion
- *         or as a regular waiter) (Each TInfo instance in the infoList captures
- *         information about one lock on one resource)
- *         If the transaction is waiting for a lock on a specific resource, the
- *         ID of that resource is captured in waitingOnRid
- */
-
-public class TxrInfo {
-    public static final int NOT_FOUND = -2;
-    public static final int NOT_KNOWN_IX = -3;
-
-    private ArrayList<TInfo> infoList;
-    private byte[] waitingOnRid;
-    private TransactionContext context;
-
-    public TxrInfo(TransactionContext context) {
-        this.context = context;
-        this.infoList = new ArrayList<TInfo>();
-        this.waitingOnRid = null;
-    }
-
-    public TInfo getTxrInfo(byte[] resourceId, int lMode, int eix) {
-        if (eix == NOT_KNOWN_IX) {
-            eix = findInList(resourceId, lMode);
-        }
-
-        if (eix != NOT_FOUND) {
-            return infoList.get(eix);
-        }
-        return null;
-    }
-
-    public void addGrantedLock(byte[] resourceId, int lMode) {
-        int eix = findInList(resourceId, lMode);
-        if (eix == NOT_FOUND) { // We do not add a redundant lock here
-            infoList.add(new TInfo(resourceId, lMode));
-        }
-    }
-
-    public void removeLock(byte[] resourceId, int lMode, int eix) {
-        if (eix == NOT_KNOWN_IX) {
-            eix = findInList(resourceId, lMode);
-        }
-        if (eix != NOT_FOUND) {
-            infoList.remove(eix);
-        }
-    }
-
-    public TransactionContext getContext() {
-        return context;
-    }
-
-    public void setWaitOnRid(byte[] resourceId) {
-        this.waitingOnRid = null;
-        if (resourceId != null) {
-            this.waitingOnRid = Arrays.copyOf(resourceId, resourceId.length);
-        }
-
-    }
-
-    public byte[] getWaitOnRid() {
-        return this.waitingOnRid;
-    }
-
-    public long getStartWaitTime() {
-        return this.context.getStartWaitTime();
-    }
-
-    public int getSize() {
-        return infoList.size();
-    }
-
-    public int findInList(byte[] resourceId, int lMode) {
-        for (int i = 0; i < infoList.size(); i++) {
-            TInfo ti = infoList.get(i);
-            if (((lMode == LockInfo.ANY_LOCK_MODE) || (lMode == ti.getMode()))
-                    && Arrays.equals(ti.getResourceId(), resourceId)) {
-                return i;
-            }
-        }
-        return NOT_FOUND;
-    }
-
-    public Iterator<TInfo> getIterator() { // TODO change the direct way of
-        // accessing
-        return infoList.iterator();
-    }
-}
-
-class TInfo {
-    private byte[] resourceId; // The resource on which the lock is held or is
-                               // waiting to be held
-    private int lockMode; // The granted/waiting-for lockMode
-
-    public TInfo(byte[] rId, int lMode) {
-        this.resourceId = rId;
-        this.lockMode = lMode;
-    }
-
-    public byte[] getResourceId() {
-        return this.resourceId;
-    }
-
-    public int getMode() {
-        return lockMode;
-    }
-
-    public void setMode(int mode) {
-        lockMode = mode;
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (o == null || !(o instanceof TInfo)) {
-            return false;
-        }
-        TInfo t = (TInfo) o;
-        return ((t.lockMode == lockMode) && (Arrays.equals(t.resourceId, resourceId)));
-    }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
index 41a2a52..4b8d62f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
@@ -47,4 +47,17 @@
         return bytes;
     }
 
+    public static byte[] longToByteArray(long value) {
+        byte[] bytes = new byte[8];
+        bytes[0] = (byte) ((value >>> 56) & 0xFF);
+        bytes[1] = (byte) ((value >>> 48) & 0xFF);
+        bytes[2] = (byte) ((value >>> 40) & 0xFF);
+        bytes[3] = (byte) ((value >>> 32) & 0xFF);
+        bytes[4] = (byte) ((value >>> 24) & 0xFF);
+        bytes[5] = (byte) ((value >>> 16) & 0xFF);
+        bytes[6] = (byte) ((value >>> 8) & 0xFF);
+        bytes[7] = (byte) ((value >>> 0) & 0xFF);
+        return bytes;
+    }
+
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
index e5cf1af..7575b45 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
@@ -30,10 +30,16 @@
     private final int BEGIN_ACTION_TYPE_POS = 9;
     private final int BEGIN_TIMESTAMP_POS = 10;
     private final int BEGIN_TRANSACTION_ID_POS = 18;
+    /*
     private final int BEGIN_RESOURCE_MGR_ID_POS = 26;
     private final int BEGIN_PAGE_ID_POS = 27;
     private final int BEGIN_PREV_LSN_POS = 35;
+    */
+    private final int BEGIN_RESOURCE_MGR_ID_POS = 22;
+    private final int BEGIN_PAGE_ID_POS = 23;
+    private final int BEGIN_PREV_LSN_POS = 31;
 
+    
     private ILogManager logManager;
 
     public LogRecordHelper(ILogManager logManager) {
@@ -62,7 +68,7 @@
     }
 
     public long getLogTransactionId(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS);
+        return (logicalLogLocator.getBuffer()).readInt(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS);
     }
 
     public byte getResourceMgrId(LogicalLogLocator logicalLogLocator) {
@@ -144,8 +150,8 @@
         (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TIMESTAMP_POS, timestamp);
 
         /* transaction id */
-        (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS,
-                context.getTransactionID());
+        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS,
+                context.getJobId().getId());
 
         /* resource Mgr id */
         (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_RESOURCE_MGR_ID_POS,
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
index 3b923dd..b742ada 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
@@ -25,7 +25,7 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 
 /**
  * Represents a utility class for generating log records corresponding to
@@ -58,10 +58,10 @@
 class TxnThreadState {
 
     private ITupleReference tuple;
-    private IndexOp indexOperation;
+    private IndexOperation indexOperation;
     private LogicalLogLocator logicalLogLocator;
 
-    public TxnThreadState(LogicalLogLocator logicalLogLocator, IndexOp indexOperation, ITupleReference tupleReference) {
+    public TxnThreadState(LogicalLogLocator logicalLogLocator, IndexOperation indexOperation, ITupleReference tupleReference) {
         this.tuple = tupleReference;
         this.indexOperation = indexOperation;
         this.logicalLogLocator = logicalLogLocator;
@@ -75,11 +75,11 @@
         this.tuple = tuple;
     }
 
-    public synchronized IndexOp getIndexOperation() {
+    public synchronized IndexOperation getIndexOperation() {
         return indexOperation;
     }
 
-    public synchronized void setIndexOperation(IndexOp indexOperation) {
+    public synchronized void setIndexOperation(IndexOperation indexOperation) {
         this.indexOperation = indexOperation;
     }
 
@@ -122,32 +122,32 @@
     }
 
     public synchronized void close(TransactionContext context) {
-        TransactionState txnState = (TransactionState) arguments.get(context.getTransactionID());
+        TransactionState txnState = (TransactionState) arguments.get(context.getJobId());
         txnState.remove(Thread.currentThread().getId());
-        arguments.remove(context.getTransactionID());
+        arguments.remove(context.getJobId());
     }
 
-    public void generateLogRecord(TransactionProvider provider, TransactionContext context, IndexOp operation,
+    public void generateLogRecord(TransactionProvider provider, TransactionContext context, IndexOperation operation,
             ITupleReference tuple) throws ACIDException {
         context.addCloseableResource(this); // the close method would be called
         // on this TreeLogger instance at
         // the time of transaction
         // commit/abort.
-        if (operation != IndexOp.INSERT && operation != IndexOp.DELETE) {
+        if (operation != IndexOperation.INSERT && operation != IndexOperation.DELETE) {
             throw new ACIDException("Loging for Operation " + operation + " not supported");
 
         }
 
         TxnThreadState txnThreadState = null;
         TransactionState txnState;
-        txnState = (TransactionState) arguments.get(context.getTransactionID());
+        txnState = (TransactionState) arguments.get(context.getJobId());
         if (txnState == null) {
             synchronized (context) { // threads belonging to different
                 // transaction do not need to
                 // synchronize amongst them.
                 if (txnState == null) {
                     txnState = new TransactionState();
-                    arguments.put(context.getTransactionID(), txnState);
+                    arguments.put(context.getJobId(), txnState);
                 }
             }
         }
@@ -172,7 +172,7 @@
     @Override
     public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logRecordSize,
             Map<Object, Object> loggerArguments) throws ACIDException {
-        TransactionState txnState = (TransactionState) loggerArguments.get(context.getTransactionID());
+        TransactionState txnState = (TransactionState) loggerArguments.get(context.getJobId());
         TxnThreadState state = (TxnThreadState) txnState.getTransactionThreadState(Thread.currentThread().getId());
         int count = 0;
         byte[] logBuffer = logicalLogLocator.getBuffer().getArray();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
index 7c6d05a..4e6ad80 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
@@ -20,6 +20,7 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
 
 public class TreeResourceManager implements IResourceManager {
 
@@ -36,7 +37,6 @@
     }
 
     public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
-
         int logContentBeginPos = logRecordHelper.getLogContentBeginPos(logLocator);
         byte[] logBufferContent = logLocator.getBuffer().getArray();
         // read the length of resource id byte array
@@ -58,7 +58,8 @@
         tupleReference.setFieldCount(tupleReference.getFieldCount());
         tupleReference.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), tupleBeginPos);
         byte operation = logBufferContent[operationOffset];
-        IIndexAccessor treeIndexAccessor = treeIndex.createAccessor();
+        IIndexAccessor treeIndexAccessor = treeIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
         try {
             switch (operation) {
                 case TreeLogger.BTreeOperationCodes.INSERT:
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index 9c78b95..e96cc73 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -166,7 +166,7 @@
     /**
      * Rollback a transaction (non-Javadoc)
      * 
-     * @see edu.uci.ics.transaction.management.service.recovery.IRecoveryManager# rollbackTransaction (edu.uci.ics.transaction.management.service.transaction .TransactionContext)
+     * @see edu.uci.ics.transaction.management.service.recovery.IRecoveryManager# rollbackTransaction (edu.uci.ics.TransactionContext.management.service.transaction .TransactionContext)
      */
     @Override
     public void rollbackTransaction(TransactionContext txnContext) throws ACIDException {
@@ -183,7 +183,7 @@
         if (lsn.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info(" no need to roll back as there were no operations by the transaction "
-                        + txnContext.getTransactionID());
+                        + txnContext.getJobId());
             }
             return;
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java
new file mode 100644
index 0000000..9aded2a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetId.java
@@ -0,0 +1,32 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import java.io.Serializable;
+
+public class DatasetId implements Serializable {
+    int id;
+
+    public DatasetId(int id) {
+        this.id = id;
+    }
+
+    public void setId(int id) {
+        this.id = id;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if ((o == null) || !(o instanceof DatasetId)) {
+            return false;
+        }
+        return ((DatasetId) o).id == this.id;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetIdFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetIdFactory.java
new file mode 100644
index 0000000..65512ec
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/DatasetIdFactory.java
@@ -0,0 +1,15 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class DatasetIdFactory {
+    private static AtomicInteger id = new AtomicInteger();
+    
+    public static void initialize(int initialId) {
+    	id.set(initialId);
+    }
+
+    public static int generateDatasetId() {
+        return id.incrementAndGet();
+    }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
index d7078bd..a876bb2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionManager.java
@@ -40,24 +40,24 @@
      * Begins a transaction identified by a transaction id and returns the
      * associated transaction context.
      * 
-     * @param transactionId
+     * @param jobId
      *            a unique value for the transaction id.
      * @return the transaction context associated with the initiated transaction
      * @see TransactionContext
      * @throws ACIDException
      */
-    public TransactionContext beginTransaction(long transactionId) throws ACIDException;
+    public TransactionContext beginTransaction(JobId jobId) throws ACIDException;
 
     /**
      * Returns the transaction context of an active transaction given the
      * transaction id.
      * 
-     * @param transactionId
+     * @param jobId
      *            a unique value for the transaction id.
      * @return
      * @throws ACIDException
      */
-    public TransactionContext getTransactionContext(long transactionId) throws ACIDException;
+    public TransactionContext getTransactionContext(JobId jobId) throws ACIDException;
 
     /**
      * Commits a transaction.
@@ -65,7 +65,7 @@
      * @param txnContext
      *            the transaction context associated with the transaction
      * @throws ACIDException
-     * @see transactionContext
+     * @see transactionContextimport edu.uci.ics.hyracks.api.job.JobId;
      * @see ACIDException
      */
     public void commitTransaction(TransactionContext txnContext) throws ACIDException;
@@ -96,21 +96,6 @@
     public void completedTransaction(TransactionContext txnContext, boolean success) throws ACIDException;
 
     /**
-     * Associates a resource manager with a transaction. In a distributed
-     * transaction multiple resource managers can join a transaction and
-     * participate in a two phase commit protocol. This method is not used
-     * currently as we do not support distributed transactions.
-     * 
-     * @param txnContext
-     *            the transaction context associated with the transaction
-     * @param resourceMgrID
-     *            a unique identifier for the resource manager.
-     * @see IResourceManager
-     * @throws ACIDException
-     */
-    public void joinTransaction(TransactionContext txnContext, byte[] resourceMgrID) throws ACIDException;
-
-    /**
      * Returns the Transaction Provider for the transaction eco-system. A
      * transaction eco-system consists of a Log Manager, a Recovery Manager, a
      * Transaction Manager and a Lock Manager.
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java
new file mode 100644
index 0000000..d306670
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobId.java
@@ -0,0 +1,42 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+import java.io.Serializable;
+
+public class JobId implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    private int id;
+
+    public JobId(int id) {
+        this.id = id;
+    }
+
+    public int getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof JobId)) {
+            return false;
+        }
+        return ((JobId) o).id == id;
+    }
+
+    @Override
+    public String toString() {
+        return "JID:" + id;
+    }
+
+	public void setId(int jobId) {
+		id = jobId;
+	}
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
similarity index 77%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java
rename to asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
index e5475f4..da86199 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/JobIdFactory.java
@@ -14,15 +14,15 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Represents a factory to generate unique transaction IDs.
  */
-public class TransactionIDFactory {
-    private static final AtomicLong ID = new AtomicLong();
+public class JobIdFactory {
+    private static final AtomicInteger Id = new AtomicInteger();
 
-    public static long generateTransactionId() {
-        return ID.incrementAndGet();
+    public static JobId generateJobId() {
+        return new JobId(Id.incrementAndGet());
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index e37f892..587f358 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -36,7 +36,7 @@
     public static final long INVALID_TIME = -1l; // used for showing a
     // transaction is not waiting.
     public static final int ACTIVE_STATUS = 0;
-    public static final int TIMED_OUT_SATUS = 1;
+    public static final int TIMED_OUT_STATUS = 1;
 
     public enum TransactionType {
         READ,
@@ -45,13 +45,13 @@
 
     private static final long serialVersionUID = -6105616785783310111L;
     private TransactionProvider transactionProvider;
-    private long transactionID;
     private LogicalLogLocator lastLogLocator;
     private TransactionState txnState;
     private long startWaitTime;
     private int status;
     private Set<ICloseable> resources = new HashSet<ICloseable>();
     private TransactionType transactionType = TransactionType.READ;
+    private JobId jobId;
 
     public void setTransactionType(TransactionType transactionType) {
         this.transactionType = transactionType;
@@ -65,8 +65,8 @@
         resources.add(resource);
     }
 
-    public TransactionContext(long transactionId, TransactionProvider transactionProvider) throws ACIDException {
-        this.transactionID = transactionId;
+    public TransactionContext(JobId jobId, TransactionProvider transactionProvider) throws ACIDException {
+        this.jobId = jobId;
         this.transactionProvider = transactionProvider;
         init();
     }
@@ -86,8 +86,8 @@
         this.lastLogLocator = lastLogLocator;
     }
 
-    public long getTransactionID() {
-        return transactionID;
+    public JobId getJobId() {
+        return jobId;
     }
 
     public void setStartWaitTime(long time) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
index ad65973..cb71e9f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManagementConstants.java
@@ -41,8 +41,10 @@
         public static final int[] LOCK_CONVERT_MATRIX = new int[] { 2, 0 };
 
         public static class LockMode {
-            public static final int SHARED = 0;
-            public static final int EXCLUSIVE = 1;
+            public static final byte S = 0;
+            public static final byte X = 1;
+            public static final byte IS = 2;
+            public static final byte IX = 3;
         }
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
index 8e7f26a..63deb13 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -30,7 +30,7 @@
 public class TransactionManager implements ITransactionManager {
     private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
     private final TransactionProvider transactionProvider;
-    private Map<Long, TransactionContext> transactionContextRepository = new HashMap<Long, TransactionContext>();
+    private Map<JobId, TransactionContext> transactionContextRepository = new HashMap<JobId, TransactionContext>();
 
     public TransactionManager(TransactionProvider provider) {
         this.transactionProvider = provider;
@@ -54,29 +54,29 @@
             } finally {
                 txnContext.releaseResources();
                 transactionProvider.getLockManager().releaseLocks(txnContext);
-                transactionContextRepository.remove(txnContext.getTransactionID());
+                transactionContextRepository.remove(txnContext.getJobId());
                 txnContext.setTxnState(TransactionState.ABORTED);
             }
         }
     }
 
     @Override
-    public TransactionContext beginTransaction(long transactionId) throws ACIDException {
-        TransactionContext txnContext = new TransactionContext(transactionId, transactionProvider);
+    public TransactionContext beginTransaction(JobId jobId) throws ACIDException {
+        TransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
         synchronized (this) {
-            transactionContextRepository.put(transactionId, txnContext);
+            transactionContextRepository.put(jobId, txnContext);
         }
         return txnContext;
     }
 
     @Override
-    public TransactionContext getTransactionContext(long transactionId) throws ACIDException {
+    public TransactionContext getTransactionContext(JobId jobId) throws ACIDException {
         synchronized (transactionContextRepository) {
-            TransactionContext context = transactionContextRepository.get(transactionId);
+            TransactionContext context = transactionContextRepository.get(jobId);
             if (context == null) {
-                context = transactionContextRepository.get(transactionId);
-                context = new TransactionContext(transactionId, transactionProvider);
-                transactionContextRepository.put(transactionId, context);
+                context = transactionContextRepository.get(jobId);
+                context = new TransactionContext(jobId, transactionProvider);
+                transactionContextRepository.put(jobId, context);
             }
             return context;
         }
@@ -100,27 +100,19 @@
                 }
             } catch (ACIDException ae) {
                 if (LOGGER.isLoggable(Level.SEVERE)) {
-                    LOGGER.severe(" caused exception in commit !" + txnContext.getTransactionID());
+                    LOGGER.severe(" caused exception in commit !" + txnContext.getJobId());
                 }
                 throw ae;
             } finally {
                 txnContext.releaseResources();
                 transactionProvider.getLockManager().releaseLocks(txnContext); // release
-                transactionContextRepository.remove(txnContext.getTransactionID());
+                transactionContextRepository.remove(txnContext.getJobId());
                 txnContext.setTxnState(TransactionState.COMMITTED);
             }
         }
     }
 
     @Override
-    public void joinTransaction(TransactionContext txnContext, byte[] resourceMgrID) throws ACIDException {
-        throw new UnsupportedOperationException();
-        // TODO this method will be implemented as part of support for
-        // distributed transactions
-
-    }
-
-    @Override
     public void completedTransaction(TransactionContext txnContext, boolean success) throws ACIDException {
         if (!success) {
             abortTransaction(txnContext);
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
index 496260d..cc266fe 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
@@ -20,7 +20,6 @@
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.logging.BasicLogger;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
 import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
@@ -28,9 +27,10 @@
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogUtil;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
 
 public class TransactionWorkloadSimulator {
@@ -99,7 +99,7 @@
 
     public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
         if (context == null) {
-            context = new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+            context = new TransactionContext(JobIdFactory.generateJobId(), provider);
         }
         return context;
     }
@@ -108,7 +108,7 @@
 class MultipleTransactionContextFactory {
 
     public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
-        return new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+        return new TransactionContext(JobIdFactory.generateJobId(), provider);
     }
 }
 
@@ -121,10 +121,12 @@
     LogicalLogLocator memLSN;
     String name;
     TransactionContext context;
-    private byte[] resourceID = new byte[1];
+    //private byte[] resourceID = new byte[1];
+    private int resourceID;
     private int myLogCount = 0;
     private TransactionProvider transactionProvider;
     private ILogManager logManager;
+    private DatasetId tempDatasetId = new DatasetId(-1);
 
     public Transaction(TransactionProvider provider, String name, boolean singleTransaction) throws ACIDException {
         this.name = name;
@@ -157,12 +159,12 @@
         }
         if (TransactionWorkloadSimulator.workload.singleResource) {
             int choice = random.nextInt(2);
-            resourceID[0] = (byte) (choice % 2);
+            resourceID = (byte) (choice % 2);
         } else {
-            random.nextBytes(resourceID);
+            random.nextInt(resourceID);
         }
         boolean retry = false;
-        int lockMode = -1;
+        byte lockMode = -1;
         try {
             for (int i = 0; i < numLogs - 1; i++) {
                 int logSize = TransactionWorkloadSimulator.workload.minLogSize
@@ -174,13 +176,10 @@
                 byte logActionType = LogActionType.REDO_UNDO;
                 long pageId = 0;
                 if (!retry) {
-                    lockMode = random.nextInt(2);
+                    lockMode = (byte)(random.nextInt(2));
                 }
-                boolean lockGranted = TransactionWorkloadSimulator.lockManager.lock(context, resourceID, lockMode);
-                if (!lockGranted) {
-                    retry = true;
-                    continue;
-                }
+                tempDatasetId.setId(resourceID);
+                TransactionWorkloadSimulator.lockManager.lock(tempDatasetId, -1, lockMode, context);
                 TransactionWorkloadSimulator.logManager.log(memLSN, context, ResourceMgrInfo.BTreeResourceMgrId,
                         pageId, logType, logActionType, logSize, logger, null);
                 retry = false;
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
index ffa5af6..4476955 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
@@ -31,8 +31,9 @@
 import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager.SystemState;
 import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionIDFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
 
 public class TransactionSimulator {
@@ -63,8 +64,8 @@
     }
 
     public TransactionContext beginTransaction() throws ACIDException {
-        long transactionId = TransactionIDFactory.generateTransactionId();
-        return transactionManager.beginTransaction(transactionId);
+        JobId jobId = JobIdFactory.generateJobId();
+        return transactionManager.beginTransaction(jobId);
     }
 
     public void executeTransactionOperation(TransactionContext txnContext, FileResource.CounterOperation operation)
diff --git a/pom.xml b/pom.xml
index 95f8103..c5a3959 100644
--- a/pom.xml
+++ b/pom.xml
@@ -74,7 +74,6 @@
 		<module>asterix-runtime</module>
 		<module>asterix-om</module>
 		<module>asterix-aql</module>
-                <module>asterix-hyracks-glue</module>
                 <module>asterix-external-data</module>
                 <module>asterix-metadata</module>
                 <module>asterix-dist</module>