Creating a new branch for API related cleanup.

git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_stabilization_api_cleanup@1489 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/CommitOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/CommitOperator.java
new file mode 100644
index 0000000..98774d2
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/CommitOperator.java
@@ -0,0 +1,66 @@
+/*
+ * 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.algebra.operators;
+
+import java.util.Collection;
+import java.util.List;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractExtensibleLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorExtension;
+import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+
+public class CommitOperator extends AbstractExtensibleLogicalOperator {
+
+    private final List<LogicalVariable> primaryKeyLogicalVars;
+
+    public CommitOperator(List<LogicalVariable> primaryKeyLogicalVars) {
+        this.primaryKeyLogicalVars = primaryKeyLogicalVars;
+    }
+
+    @Override
+    public boolean isMap() {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public IOperatorExtension newInstance() {
+        return new CommitOperator(primaryKeyLogicalVars);
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform transform) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return "commit";
+    }
+
+    @Override
+    public void getUsedVariables(Collection<LogicalVariable> usedVars) {
+        usedVars.addAll(primaryKeyLogicalVars);
+    }
+
+    @Override
+    public void getProducedVariables(Collection<LogicalVariable> producedVars) {
+        // No produced variables.
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index f0880ec..efe01a5 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -47,9 +47,11 @@
     private final List<LogicalVariable> highKeyVarList;
     private final boolean isPrimaryIndex;
     private final boolean isEqCondition;
+    private Object implConfig;
 
     public BTreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast,
-            boolean isPrimaryIndex, boolean isEqCondition, List<LogicalVariable> lowKeyVarList, List<LogicalVariable> highKeyVarList) {
+            boolean isPrimaryIndex, boolean isEqCondition, List<LogicalVariable> lowKeyVarList,
+            List<LogicalVariable> highKeyVarList) {
         super(idx, requiresBroadcast);
         this.isPrimaryIndex = isPrimaryIndex;
         this.isEqCondition = isEqCondition;
@@ -57,6 +59,14 @@
         this.highKeyVarList = highKeyVarList;
     }
 
+    public void setImplConfig(Object implConfig) {
+        this.implConfig = implConfig;
+    }
+
+    public Object getImplConfig() {
+        return implConfig;
+    }
+
     @Override
     public PhysicalOperatorTag getOperatorTag() {
         return PhysicalOperatorTag.BTREE_SEARCH;
@@ -89,16 +99,16 @@
             VariableUtilities.getLiveVariables(unnestMap, outputVars);
         }
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
-                builder.getJobSpec(), outputVars, opSchema, typeEnv,  context, jobGenParams.getRetainInput(),
-                dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
-                jobGenParams.isHighKeyInclusive());
+                builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), dataset,
+                jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
+                jobGenParams.isHighKeyInclusive(), implConfig);
         builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
 
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
-    
+
     public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
             IPhysicalPropertiesVector reqdByParent) {
         if (requiresBroadcast) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java
new file mode 100644
index 0000000..ece3db6
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitPOperator.java
@@ -0,0 +1,93 @@
+/*
+ * 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.algebra.operators.physical;
+
+import java.util.List;
+
+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.core.algebra.base.IHyracksJobBuilder;
+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.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
+
+public class CommitPOperator extends AbstractPhysicalOperator {
+
+    private final List<LogicalVariable> primaryKeyLogicalVars;
+    private final JobId jobId;
+    private final int datasetId;
+    private final boolean isWriteTransaction;
+
+    public CommitPOperator(JobId jobId, int datasetId, List<LogicalVariable> primaryKeyLogicalVars,
+            boolean isWriteTransaction) {
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.primaryKeyLogicalVars = primaryKeyLogicalVars;
+        this.isWriteTransaction = isWriteTransaction;
+    }
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.EXTENSION_OPERATOR;
+    }
+
+    @Override
+    public String toString() {
+        return "COMMIT";
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent) {
+        return emptyUnaryRequirements();
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        deliveredProperties = (StructuralPropertiesVector) op2.getDeliveredPhysicalProperties().clone();
+    }
+
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+
+        RecordDescriptor recDesc = JobGenHelper.mkRecordDescriptor(context.getTypeEnvironment(op), propagatedSchema,
+                context);
+        int[] primaryKeyFields = JobGenHelper.variablesToFieldIndexes(primaryKeyLogicalVars, inputSchemas[0]);
+        CommitRuntimeFactory runtime = new CommitRuntimeFactory(jobId, datasetId, primaryKeyFields, isWriteTransaction);
+        builder.contributeMicroOperator(op, runtime, recDesc);
+        ILogicalOperator src = op.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(src, 0, op, 0);
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return true;
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
new file mode 100644
index 0000000..8533e74
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -0,0 +1,121 @@
+/*
+ * 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.algebra.operators.physical;
+
+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.service.transaction.DatasetId;
+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.algebricks.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+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.data.accessors.FrameTupleReference;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
+
+public class CommitRuntime implements IPushRuntime {
+    
+    private final static long SEED = 0L;
+
+    private final IHyracksTaskContext hyracksTaskCtx;
+    private final ITransactionManager transactionManager;
+    private final JobId jobId;
+    private final DatasetId datasetId;
+    private final int[] primaryKeyFields;
+    private final boolean isWriteTransaction;
+    private final long[] longHashes; 
+
+    private TransactionContext transactionContext;
+    private RecordDescriptor inputRecordDesc;
+    private FrameTupleAccessor frameTupleAccessor;
+    private FrameTupleReference frameTupleReference;
+
+    public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
+            boolean isWriteTransaction) {
+        this.hyracksTaskCtx = ctx;
+        AsterixAppRuntimeContext runtimeCtx = (AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+                .getApplicationObject();
+        this.transactionManager = runtimeCtx.getTransactionSubsystem().getTransactionManager();
+        this.jobId = jobId;
+        this.datasetId = new DatasetId(datasetId);
+        this.primaryKeyFields = primaryKeyFields;
+        this.frameTupleReference = new FrameTupleReference();
+        this.isWriteTransaction = isWriteTransaction;
+        this.longHashes= new long[2];
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        try {
+            transactionContext = transactionManager.getTransactionContext(jobId);
+            transactionContext.setTransactionType(isWriteTransaction ? TransactionType.READ_WRITE
+                    : TransactionType.READ);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        int pkHash = 0;
+        frameTupleAccessor.reset(buffer);
+        int nTuple = frameTupleAccessor.getTupleCount();
+        for (int t = 0; t < nTuple; t++) {
+            frameTupleReference.reset(frameTupleAccessor, t);
+            pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
+            try {
+                transactionManager.commitTransaction(transactionContext, datasetId, pkHash);
+            } catch (ACIDException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+    
+    private int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {
+        MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
+        return Math.abs((int) longHashes[0]); 
+    }
+
+    @Override
+    public void fail() throws HyracksDataException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        // TODO Auto-generated method stub
+    }
+
+    @Override
+    public void setFrameWriter(int index, IFrameWriter writer, RecordDescriptor recordDesc) {
+        throw new IllegalStateException();
+    }
+
+    @Override
+    public void setInputRecordDescriptor(int index, RecordDescriptor recordDescriptor) {
+        this.inputRecordDesc = recordDescriptor;
+        this.frameTupleAccessor = new FrameTupleAccessor(hyracksTaskCtx.getFrameSize(), recordDescriptor);
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java
new file mode 100644
index 0000000..f3b6526
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntimeFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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.algebra.operators.physical;
+
+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.runtime.base.IPushRuntime;
+import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+public class CommitRuntimeFactory implements IPushRuntimeFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final JobId jobId;
+    private final int datasetId;
+    private final int[] primaryKeyFields;
+    private final boolean isWriteTransaction;
+
+    public CommitRuntimeFactory(JobId jobId, int datasetId, int[] primaryKeyFields, boolean isWriteTransaction) {
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.primaryKeyFields = primaryKeyFields;
+        this.isWriteTransaction = isWriteTransaction;
+    }
+
+    @Override
+    public String toString() {
+        return "commit";
+    }
+
+    @Override
+    public IPushRuntime createPushRuntime(IHyracksTaskContext ctx) throws AlgebricksException {
+        return new CommitRuntime(ctx, jobId, datasetId, primaryKeyFields, isWriteTransaction);
+    }
+}
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 510aa4c..49ca5ba 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,8 @@
 import java.util.ArrayList;
 import java.util.List;
 
+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.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -17,6 +19,7 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 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.optimizer.rules.am.InvertedIndexAccessMethod;
 import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
 import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexJobGenParams;
@@ -43,25 +46,37 @@
 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.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 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.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+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.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
  * Contributes the runtime operator for an unnest-map representing an
  * inverted-index search.
  */
 public class InvertedIndexPOperator extends IndexSearchPOperator {
-    public InvertedIndexPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
+    private final boolean isPartitioned;
+
+    public InvertedIndexPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast,
+            boolean isPartitioned) {
         super(idx, requiresBroadcast);
+        this.isPartitioned = isPartitioned;
     }
 
     @Override
     public PhysicalOperatorTag getOperatorTag() {
-        return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
+        if (isPartitioned) {
+            return PhysicalOperatorTag.FUZZY_INVERTED_INDEX_SEARCH;
+        } else {
+            return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
+        }
     }
 
     @Override
@@ -138,17 +153,21 @@
             }
 
             // TODO: For now we assume the type of the generated tokens is the
-            // same
-            // as the indexed field.
+            // same as the indexed field.
             // We need a better way of expressing this because tokens may be
-            // hashed,
-            // or an inverted-index may index a list type, etc.
-            ITypeTraits[] tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
-            IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
+            // hashed, or an inverted-index may index a list type, etc.
+            int numTokenKeys = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
+            ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenKeys];
+            IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numTokenKeys];
             for (int i = 0; i < numSecondaryKeys; i++) {
-                tokenComparatorFactories[i] = InvertedIndexAccessMethod
-                        .getTokenBinaryComparatorFactory(secondaryKeyType);
-                tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
+                tokenComparatorFactories[i] = NonTaggedFormatUtil.getTokenBinaryComparatorFactory(secondaryKeyType);
+                tokenTypeTraits[i] = NonTaggedFormatUtil.getTokenTypeTrait(secondaryKeyType);
+            }
+            if (isPartitioned) {
+                // The partitioning field is hardcoded to be a short *without* an Asterix type tag.
+                tokenComparatorFactories[numSecondaryKeys] = PointableBinaryComparatorFactory
+                        .of(ShortPointable.FACTORY);
+                tokenTypeTraits[numSecondaryKeys] = ShortPointable.TYPE_TRAITS;
             }
 
             IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
@@ -169,9 +188,6 @@
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
                     .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
                             datasetName, indexName);
-            Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders = metadataProvider
-                    .getInvertedIndexFileSplitProviders(secondarySplitsAndConstraint.first);
-
             // TODO: Here we assume there is only one search key field.
             int queryField = keyFields[0];
             // Get tokenizer and search modifier factories.
@@ -179,12 +195,27 @@
                     .getSearchModifierFactory(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);
+            IIndexDataflowHelperFactory dataflowHelperFactory;
+            if (!isPartitioned) {
+                dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+            } else {
+                dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+            }
+            LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
+                    jobSpec, queryField, appContext.getStorageManagerInterface(), secondarySplitsAndConstraint.first,
+                    appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
+                    invListsTypeTraits, invListsComparatorFactories, dataflowHelperFactory, queryTokenizerFactory,
+                    searchModifierFactory, outputRecDesc, retainInput, NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp,
                     secondarySplitsAndConstraint.second);
         } catch (MetadataException e) {
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 9d9de08..3f434b7 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
@@ -30,6 +30,7 @@
 import edu.uci.ics.asterix.optimizer.rules.InlineUnnestFunctionRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceInstantLockSearchCallbackRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
 import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
@@ -43,6 +44,7 @@
 import edu.uci.ics.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
 import edu.uci.ics.asterix.optimizer.rules.RemoveRedundantListifyRule;
 import edu.uci.ics.asterix.optimizer.rules.RemoveUnusedOneToOneEquiJoinRule;
+import edu.uci.ics.asterix.optimizer.rules.ReplaceSinkOpWithCommitOpRule;
 import edu.uci.ics.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
 import edu.uci.ics.asterix.optimizer.rules.SetClosedRecordConstructorsRule;
 import edu.uci.ics.asterix.optimizer.rules.SimilarityCheckRule;
@@ -57,6 +59,7 @@
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateAssignsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceOrderByAfterSubplan;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonExpressionsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
@@ -105,6 +108,7 @@
     public final static List<IAlgebraicRewriteRule> buildNormalizationRuleCollection() {
         List<IAlgebraicRewriteRule> normalization = new LinkedList<IAlgebraicRewriteRule>();
         normalization.add(new EliminateSubplanRule());
+        normalization.add(new EnforceOrderByAfterSubplan());
         normalization.add(new PushAggFuncIntoStandaloneAggregateRule());
         normalization.add(new BreakSelectIntoConjunctsRule());
         normalization.add(new ExtractGbyExpressionsRule());
@@ -223,8 +227,12 @@
     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 ReplaceSinkOpWithCommitOpRule());
         physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
+        physicalRewritesAllLevels.add(new IntroduceInstantLockSearchCallbackRule());
         physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
         physicalRewritesAllLevels.add(new IntroHashPartitionMergeExchange());
         physicalRewritesAllLevels.add(new SetClosedRecordConstructorsRule());
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 8936425..420c713 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
@@ -267,7 +267,7 @@
         // The translator will compile metadata internally. Run this compilation
         // under the same transaction id as the "outer" compilation.
         AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(
-                metadataProvider.getJobTxnId(), metadataProvider, counter, null, null);
+                metadataProvider.getJobId(), metadataProvider, counter, null, null);
 
         LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
new file mode 100644
index 0000000..62cca6c
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceInstantLockSearchCallbackRule.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2009-2013 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 java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.algebra.operators.CommitOperator;
+import edu.uci.ics.asterix.algebra.operators.physical.BTreeSearchPOperator;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataImplConfig;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+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.IPhysicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.DataSourceScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class IntroduceInstantLockSearchCallbackRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    private void extractDataSourcesInfo(AbstractLogicalOperator op,
+            Map<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> dataSourcesMap) {
+
+        for (int i = 0; i < op.getInputs().size(); ++i) {
+            AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
+
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
+                UnnestMapOperator unnestMapOp = (UnnestMapOperator) descendantOp;
+                ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
+                if (unnestExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) unnestExpr;
+                    FunctionIdentifier fid = f.getFunctionIdentifier();
+                    if (!fid.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
+                        throw new IllegalStateException();
+                    }
+                    AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
+                    jobGenParams.readFromFuncArgs(f.getArguments());
+                    boolean isPrimaryIndex = jobGenParams.isPrimaryIndex();
+                    String indexName = jobGenParams.getIndexName();
+                    if (isPrimaryIndex) {
+                        if (dataSourcesMap.containsKey(indexName)) {
+                            ++(dataSourcesMap.get(indexName).first);
+                        } else {
+                            dataSourcesMap.put(indexName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
+                                    LogicalOperatorTag.UNNEST_MAP, unnestMapOp.getPhysicalOperator()));
+                        }
+                    }
+                }
+            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
+                DataSourceScanOperator dataSourceScanOp = (DataSourceScanOperator) descendantOp;
+                String datasetName = ((AqlDataSource) dataSourceScanOp.getDataSource()).getDataset().getDatasetName();
+                if (dataSourcesMap.containsKey(datasetName)) {
+                    ++(dataSourcesMap.get(datasetName).first);
+                } else {
+                    dataSourcesMap.put(datasetName, new Triple<Integer, LogicalOperatorTag, IPhysicalOperator>(1,
+                            LogicalOperatorTag.DATASOURCESCAN, dataSourceScanOp.getPhysicalOperator()));
+                }
+            }
+            extractDataSourcesInfo(descendantOp, dataSourcesMap);
+        }
+
+    }
+
+    private boolean checkIfRuleIsApplicable(AbstractLogicalOperator op) {
+        if (op.getPhysicalOperator() == null) {
+            return false;
+        }
+        if (op.getOperatorTag() == LogicalOperatorTag.EXTENSION_OPERATOR) {
+            ExtensionOperator extensionOp = (ExtensionOperator) op;
+            if (extensionOp.getDelegate() instanceof CommitOperator) {
+                return true;
+            }
+        }
+        if (op.getOperatorTag() == LogicalOperatorTag.DISTRIBUTE_RESULT
+                || op.getOperatorTag() == LogicalOperatorTag.WRITE_RESULT) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
+        if (!checkIfRuleIsApplicable(op)) {
+            return false;
+        }
+        Map<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> dataSourcesMap = new HashMap<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>>();
+        extractDataSourcesInfo(op, dataSourcesMap);
+
+        boolean introducedInstantLock = false;
+
+        Iterator<Map.Entry<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>>> it = dataSourcesMap
+                .entrySet().iterator();
+        while (it.hasNext()) {
+            Entry<String, Triple<Integer, LogicalOperatorTag, IPhysicalOperator>> entry = it.next();
+            Triple<Integer, LogicalOperatorTag, IPhysicalOperator> triple = entry.getValue();
+            if (triple.first == 1) {
+                AqlMetadataImplConfig aqlMetadataImplConfig = new AqlMetadataImplConfig(true);
+                if (triple.second == LogicalOperatorTag.UNNEST_MAP) {
+                    BTreeSearchPOperator pOperator = (BTreeSearchPOperator) triple.third;
+                    pOperator.setImplConfig(aqlMetadataImplConfig);
+                    introducedInstantLock = true;
+                } else {
+                    DataSourceScanPOperator pOperator = (DataSourceScanPOperator) triple.third;
+                    pOperator.setImplConfig(aqlMetadataImplConfig);
+                    introducedInstantLock = true;
+                }
+            }
+
+        }
+        return introducedInstantLock;
+    }
+}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index 3310f4d..c99e4bc 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -64,27 +64,34 @@
         }
 
         FunctionIdentifier fid = null;
-        AbstractLogicalOperator op2 = op1;
+        /** op2 is the assign operator which extract primary keys from the record variable */
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
         List<LogicalVariable> recordVar = new ArrayList<LogicalVariable>();
-        // Find assign op that creates record to be inserted/deleted.
-        while (fid != AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR) {
-            if (op2.getInputs().size() == 0) {
-                return false;
+        VariableUtilities.getUsedVariables(op2, recordVar);
+        if (recordVar.size() == 0) {
+            /**
+             * For the case primary key-assignment expressions are constant expressions,
+             * find assign op that creates record to be inserted/deleted.
+             */
+            while (fid != AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR) {
+                if (op2.getInputs().size() == 0) {
+                    return false;
+                }
+                op2 = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
+                if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+                    continue;
+                }
+                AssignOperator assignOp = (AssignOperator) op2;
+                ILogicalExpression assignExpr = assignOp.getExpressions().get(0).getValue();
+                if (assignExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions()
+                            .get(0).getValue();
+                    fid = funcExpr.getFunctionIdentifier();
+                }
             }
-            op2 = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
-            if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-                continue;
-            }
-            AssignOperator assignOp = (AssignOperator) op2;
-            ILogicalExpression assignExpr = assignOp.getExpressions().get(0).getValue();
-            if (assignExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions().get(0)
-                        .getValue();
-                fid = funcExpr.getFunctionIdentifier();
-            }
+            AssignOperator assignOp2 = (AssignOperator) op2;
+            recordVar.addAll(assignOp2.getVariables());
         }
-        AssignOperator assignOp2 = (AssignOperator) op2;
-        recordVar.addAll(assignOp2.getVariables());
         InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
         AqlDataSource datasetSource = (AqlDataSource) insertOp.getDataSource();
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
@@ -146,7 +153,10 @@
             project.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
             context.computeAndSetTypeEnvironmentForOperator(project);
             context.computeAndSetTypeEnvironmentForOperator(assign);
-            if (index.getIndexType() == IndexType.BTREE) {
+            if (index.getIndexType() == IndexType.BTREE || index.getIndexType() == IndexType.WORD_INVIX
+                    || index.getIndexType() == IndexType.NGRAM_INVIX
+                    || index.getIndexType() == IndexType.FUZZY_WORD_INVIX
+                    || index.getIndexType() == IndexType.FUZZY_NGRAM_INVIX) {
                 for (LogicalVariable secondaryKeyVar : secondaryKeyVars) {
                     secondaryExpressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
                             secondaryKeyVar)));
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/ReplaceSinkOpWithCommitOpRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
new file mode 100644
index 0000000..30ed8e7
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ReplaceSinkOpWithCommitOpRule.java
@@ -0,0 +1,94 @@
+package edu.uci.ics.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.algebra.operators.CommitOperator;
+import edu.uci.ics.asterix.algebra.operators.physical.CommitPOperator;
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+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.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.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SinkOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+public class ReplaceSinkOpWithCommitOpRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.SINK) {
+            return false;
+        }
+        SinkOperator sinkOperator = (SinkOperator) op;
+
+        List<Mutable<ILogicalExpression>> primaryKeyExprs = null;
+        int datasetId = 0;
+        AbstractLogicalOperator descendantOp = (AbstractLogicalOperator) sinkOperator.getInputs().get(0).getValue();
+        while (descendantOp != null) {
+            if (descendantOp.getOperatorTag() == LogicalOperatorTag.INDEX_INSERT_DELETE) {
+                IndexInsertDeleteOperator indexInsertDeleteOperator = (IndexInsertDeleteOperator) descendantOp;
+                primaryKeyExprs = indexInsertDeleteOperator.getPrimaryKeyExpressions();
+                datasetId = ((AqlDataSource) indexInsertDeleteOperator.getDataSourceIndex().getDataSource()).getDataset().getDatasetId();
+                break;
+            } else if (descendantOp.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE) {
+                InsertDeleteOperator insertDeleteOperator = (InsertDeleteOperator) descendantOp;
+                primaryKeyExprs = insertDeleteOperator.getPrimaryKeyExpressions();
+                datasetId = ((AqlDataSource) insertDeleteOperator.getDataSource()).getDataset().getDatasetId();
+                break;
+            }
+            descendantOp = (AbstractLogicalOperator) descendantOp.getInputs().get(0).getValue();
+        }
+
+        if (primaryKeyExprs != null) {
+
+            //copy primaryKeyExprs
+            List<LogicalVariable> primaryKeyLogicalVars = new ArrayList<LogicalVariable>();
+            for (Mutable<ILogicalExpression> expr : primaryKeyExprs) {
+                VariableReferenceExpression varRefExpr = (VariableReferenceExpression)expr.getValue();
+                primaryKeyLogicalVars.add(new LogicalVariable(varRefExpr.getVariableReference().getId()));
+            }
+
+            //get JobId(TransactorId)
+            AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
+            JobId jobId = mp.getJobId();
+
+            //create the logical and physical operator
+            CommitOperator commitOperator = new CommitOperator(primaryKeyLogicalVars);
+            CommitPOperator commitPOperator = new CommitPOperator(jobId, datasetId, primaryKeyLogicalVars, mp.isWriteTransaction());
+            commitOperator.setPhysicalOperator(commitPOperator);
+
+            //create ExtensionOperator and put the commitOperator in it.
+            ExtensionOperator extensionOperator = new ExtensionOperator(commitOperator);
+            extensionOperator.setPhysicalOperator(commitPOperator);
+            
+            //update plan link
+            extensionOperator.getInputs().add(sinkOperator.getInputs().get(0));
+            context.computeAndSetTypeEnvironmentForOperator(extensionOperator);
+            opRef.setValue(extensionOperator);
+        }
+
+        return true;
+    }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index e19466e..9c7594c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -173,12 +173,14 @@
                                 op.setPhysicalOperator(new RTreeSearchPOperator(dsi, requiresBroadcast));
                                 break;
                             }
-                            case WORD_INVIX: {
-                                op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast));
+                            case WORD_INVIX:
+                            case NGRAM_INVIX: {
+                                op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast, false));
                                 break;
                             }
-                            case NGRAM_INVIX: {
-                                op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast));
+                            case FUZZY_WORD_INVIX:
+                            case FUZZY_NGRAM_INVIX: {
+                                op.setPhysicalOperator(new InvertedIndexPOperator(dsi, requiresBroadcast, true));
                                 break;
                             }
                             default: {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index e3a9e91..84e152a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -59,7 +59,6 @@
         retainInput = AccessMethodUtils.getBooleanConstant(funcArgs.get(4));
         requiresBroadcast = AccessMethodUtils.getBooleanConstant(funcArgs.get(5));
         isPrimaryIndex = datasetName.equals(indexName);
-        isPrimaryIndex = datasetName.equals(indexName);
     }
 
     public String getIndexName() {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
index ffd447c..7084cd7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -149,7 +149,9 @@
         switch (index.getIndexType()) {
             case BTREE:
             case WORD_INVIX:
-            case NGRAM_INVIX: {
+            case NGRAM_INVIX: 
+            case FUZZY_WORD_INVIX:
+            case FUZZY_NGRAM_INVIX: {
                 return index.getKeyFieldNames().size();
             }
             case RTREE: {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
index 3b547e7..ddcf768 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -17,6 +17,7 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.BitSet;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -341,12 +342,26 @@
             return null;
         }
 
+        // If the select condition contains mixed open/closed intervals on multiple keys, then we make all intervals closed to obtain a superset of answers and leave the original selection in place.
+        boolean primaryIndexPostProccessingIsNeeded = false;
+        for (int i = 1; i < numSecondaryKeys; ++i) {
+            if (lowKeyInclusive[i] != lowKeyInclusive[0]) {
+                Arrays.fill(lowKeyInclusive, true);
+                primaryIndexPostProccessingIsNeeded = true;
+                break;
+            }
+        }
+        for (int i = 1; i < numSecondaryKeys; ++i) {
+            if (highKeyInclusive[i] != highKeyInclusive[0]) {
+                Arrays.fill(highKeyInclusive, true);
+                primaryIndexPostProccessingIsNeeded = true;
+                break;
+            }
+        }
+
         // Rule out the cases unsupported by the current btree search
         // implementation.
         for (int i = 1; i < numSecondaryKeys; i++) {
-            if (lowKeyInclusive[i] != lowKeyInclusive[0] || highKeyInclusive[i] != highKeyInclusive[0]) {
-                return null;
-            }
             if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null
                     && lowKeyLimits[i] == null) {
                 return null;
@@ -404,6 +419,10 @@
         if (!isPrimaryIndex) {
             primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
                     secondaryIndexUnnestOp, context, true, retainInput, false);
+
+            // Replace the datasource scan with the new plan rooted at
+            // primaryIndexUnnestMap.
+            indexSubTree.dataSourceScanRef.setValue(primaryIndexUnnestOp); //kisskys
         } else {
             List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
             try {
@@ -414,16 +433,18 @@
             primaryIndexUnnestOp = new UnnestMapOperator(dataSourceScan.getVariables(),
                     secondaryIndexUnnestOp.getExpressionRef(), primaryIndexOutputTypes, retainInput);
             primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
-        }
 
-        List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<Mutable<ILogicalExpression>>();
-        getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
-        // Generate new condition.
-        if (!remainingFuncExprs.isEmpty()) {
-            ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
-            conditionRef.setValue(pulledCond);
-        } else {
-            conditionRef.setValue(null);
+            if (!primaryIndexPostProccessingIsNeeded) {
+                List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<Mutable<ILogicalExpression>>();
+                getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
+                // Generate new condition.
+                if (!remainingFuncExprs.isEmpty()) {
+                    ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
+                    conditionRef.setValue(pulledCond);
+                } else {
+                    conditionRef.setValue(null);
+                }
+            }
         }
         return primaryIndexUnnestOp;
     }
@@ -561,4 +582,4 @@
         // No additional analysis required for BTrees.
         return true;
     }
-}
+}
\ No newline at end of file
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 84a86b1..ff260f1 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
@@ -12,10 +12,7 @@
 import edu.uci.ics.asterix.algebra.base.LogicalOperatorDeepCopyVisitor;
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.dataflow.data.common.ListEditDistanceSearchModifierFactory;
-import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryTokenizerFactoryProvider;
-import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.base.AFloat;
@@ -26,11 +23,8 @@
 import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.types.AOrderedListType;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.AUnorderedListType;
-import edu.uci.ics.asterix.om.types.AbstractCollectionType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
@@ -58,13 +52,12 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 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.search.ListEditDistanceSearchModifierFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
  * Class for helping rewrite rules to choose and apply inverted indexes.
@@ -519,37 +512,37 @@
             joinInputSubTreeVarMap.put(optFuncExpr.getLogicalVar(i), context.newVar());
             newProbeSubTreeVarMap.put(optFuncExpr.getLogicalVar(i), optFuncExpr.getLogicalVar(i));
         }
-        for (int i = 0; i < originalSubTreePKs.size(); i++) {            
+        for (int i = 0; i < originalSubTreePKs.size(); i++) {
             LogicalVariable newPKVar = context.newVar();
             surrogateSubTreePKs.add(newPKVar);
             joinInputSubTreeVarMap.put(originalSubTreePKs.get(i), newPKVar);
             newProbeSubTreeVarMap.put(originalSubTreePKs.get(i), originalSubTreePKs.get(i));
         }
-        
+
         // Create first copy.
         Counter firstCounter = new Counter(context.getVarCounter());
         LogicalOperatorDeepCopyVisitor firstDeepCopyVisitor = new LogicalOperatorDeepCopyVisitor(firstCounter,
                 newProbeSubTreeVarMap);
         ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.root, null);
         inferTypes(newProbeSubTree, context);
-        Mutable<ILogicalOperator> newProbeSubTreeRootRef = new MutableObject<ILogicalOperator>(newProbeSubTree);        
+        Mutable<ILogicalOperator> newProbeSubTreeRootRef = new MutableObject<ILogicalOperator>(newProbeSubTree);
         context.setVarCounter(firstCounter.get());
         // Create second copy.
         Counter secondCounter = new Counter(context.getVarCounter());
         LogicalOperatorDeepCopyVisitor secondDeepCopyVisitor = new LogicalOperatorDeepCopyVisitor(secondCounter,
-                joinInputSubTreeVarMap);        
+                joinInputSubTreeVarMap);
         ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.root, null);
         inferTypes(joinInputSubTree, context);
         probeSubTree.rootRef.setValue(joinInputSubTree);
         context.setVarCounter(secondCounter.get());
-        
+
         // Remember the original probe subtree reference so we can return it.
         Mutable<ILogicalOperator> originalProbeSubTreeRootRef = probeSubTree.rootRef;
 
         // Replace the original probe subtree with its copy.
         Dataset origDataset = probeSubTree.dataset;
         ARecordType origRecordType = probeSubTree.recordType;
-        probeSubTree.initFromSubTree(newProbeSubTreeRootRef);        
+        probeSubTree.initFromSubTree(newProbeSubTreeRootRef);
         probeSubTree.dataset = origDataset;
         probeSubTree.recordType = origRecordType;
 
@@ -571,7 +564,7 @@
         for (int i = 0; i < numPKVars; i++) {
             List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
             args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(surrogateSubTreePKs.get(i))));
-            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(originalSubTreePKs.get(i))));                   
+            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(originalSubTreePKs.get(i))));
             ILogicalExpression eqFunc = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.EQ), args);
             eqExprs.add(new MutableObject<ILogicalExpression>(eqFunc));
@@ -773,7 +766,8 @@
             AInt32 edThresh = (AInt32) intObj;
             int mergeThreshold = 0;
             // We can only optimize edit distance on strings using an ngram index.
-            if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING && index.getIndexType() == IndexType.NGRAM_INVIX) {
+            if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING
+                    && (index.getIndexType() == IndexType.NGRAM_INVIX || index.getIndexType() == IndexType.FUZZY_NGRAM_INVIX)) {
                 AString astr = (AString) listOrStrObj;
                 // Compute merge threshold.
                 mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
@@ -781,7 +775,7 @@
             }
             // We can only optimize edit distance on lists using a word index.
             if ((listOrStrObj.getType().getTypeTag() == ATypeTag.ORDEREDLIST || listOrStrObj.getType().getTypeTag() == ATypeTag.UNORDEREDLIST)
-                    && index.getIndexType() == IndexType.WORD_INVIX) {
+                    && (index.getIndexType() == IndexType.WORD_INVIX || index.getIndexType() == IndexType.FUZZY_WORD_INVIX)) {
                 IACollection alist = (IACollection) listOrStrObj;
                 // Compute merge threshold.
                 mergeThreshold = alist.size() - edThresh.getIntegerValue();
@@ -807,11 +801,11 @@
                 AbstractFunctionCallExpression nonConstfuncExpr = (AbstractFunctionCallExpression) nonConstArg;
                 // We can use this index if the tokenization function matches the index type.
                 if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS
-                        && index.getIndexType() == IndexType.WORD_INVIX) {
+                        && (index.getIndexType() == IndexType.WORD_INVIX || index.getIndexType() == IndexType.FUZZY_WORD_INVIX)) {
                     return true;
                 }
                 if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS
-                        && index.getIndexType() == IndexType.NGRAM_INVIX) {
+                        && (index.getIndexType() == IndexType.NGRAM_INVIX || index.getIndexType() == IndexType.FUZZY_NGRAM_INVIX)) {
                     return true;
                 }
             }
@@ -824,7 +818,7 @@
         }
         // We can only optimize contains with ngram indexes.
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS
-                && index.getIndexType() == IndexType.NGRAM_INVIX) {
+                && (index.getIndexType() == IndexType.NGRAM_INVIX || index.getIndexType() == IndexType.FUZZY_NGRAM_INVIX)) {
             // Check that the constant search string has at least gramLength characters.
             AsterixConstantValue strConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
             IAObject strObj = strConstVal.getObject();
@@ -838,49 +832,15 @@
         return false;
     }
 
-    public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
-        IAType type = keyType;
-        ATypeTag typeTag = keyType.getTypeTag();
-        // Extract item type from list.
-        if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
-            AbstractCollectionType listType = (AbstractCollectionType) keyType;
-            if (!listType.isTyped()) {
-                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
-            }
-            type = listType.getItemType();
-        }
-        // Ignore case for string types.
-        return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
-    }
-
-    public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
-        IAType type = keyType;
-        ATypeTag typeTag = keyType.getTypeTag();
-        // Extract item type from list.
-        if (typeTag == ATypeTag.UNORDEREDLIST) {
-            AUnorderedListType ulistType = (AUnorderedListType) keyType;
-            if (!ulistType.isTyped()) {
-                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
-            }
-            type = ulistType.getItemType();
-        }
-        if (typeTag == ATypeTag.ORDEREDLIST) {
-            AOrderedListType olistType = (AOrderedListType) keyType;
-            if (!olistType.isTyped()) {
-                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
-            }
-            type = olistType.getItemType();
-        }
-        return AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
-    }
-
     public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType,
             ATypeTag searchKeyType, Index index) throws AlgebricksException {
         switch (index.getIndexType()) {
-            case WORD_INVIX: {
+            case WORD_INVIX:
+            case FUZZY_WORD_INVIX: {
                 return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(searchKeyType, false);
             }
-            case NGRAM_INVIX: {
+            case NGRAM_INVIX:
+            case FUZZY_NGRAM_INVIX: {
                 // Make sure not to use pre- and postfixing for conjunctive searches.
                 boolean prePost = (searchModifierType == SearchModifierType.CONJUNCTIVE) ? false : true;
                 return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(searchKeyType,
@@ -892,22 +852,6 @@
         }
     }
 
-    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
-            int gramLength) throws AlgebricksException {
-        switch (indexType) {
-            case WORD_INVIX: {
-                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(keyType, false);
-            }
-            case NGRAM_INVIX: {
-                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType, gramLength, true,
-                        false);
-            }
-            default: {
-                throw new AlgebricksException("Tokenizer not applicable to index type '" + indexType + "'.");
-            }
-        }
-    }
-
     public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType,
             IAObject simThresh, Index index) throws AlgebricksException {
         switch (searchModifierType) {
@@ -921,11 +865,13 @@
             case EDIT_DISTANCE: {
                 int edThresh = ((AInt32) simThresh).getIntegerValue();
                 switch (index.getIndexType()) {
-                    case NGRAM_INVIX: {
+                    case NGRAM_INVIX:
+                    case FUZZY_NGRAM_INVIX: {
                         // Edit distance on strings, filtered with overlapping grams.
                         return new EditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
                     }
-                    case WORD_INVIX: {
+                    case WORD_INVIX:
+                    case FUZZY_WORD_INVIX: {
                         // Edit distance on two lists. The list-elements are non-overlapping.
                         return new ListEditDistanceSearchModifierFactory(edThresh);
                     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
index 16cfefe..9fe5750 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
@@ -26,6 +26,7 @@
 import edu.uci.ics.asterix.aql.expression.NodeGroupDropStatement;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints;
 import edu.uci.ics.asterix.metadata.entities.AsterixBuiltinTypeMap;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.om.types.BuiltinType;
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 23a7a18..caa16d4 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
@@ -163,7 +163,6 @@
     private final ICompiledDmlStatement stmt;
     private static AtomicLong outputFileID = new AtomicLong(0);
     private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
-
     private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
     public AqlExpressionToPlanTranslator(AqlMetadataProvider metadataProvider, int currentVarCounter,
@@ -186,7 +185,6 @@
                 new EmptyTupleSourceOperator()));
 
         ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
-
         boolean isTransactionalWrite = false;
         ILogicalOperator topOp = p.first;
         ProjectOperator project = (ProjectOperator) topOp;
@@ -209,11 +207,9 @@
 
             AqlDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
                     stmt.getDatasetName());
-
             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(targetDatasource.getDataset());
             for (String keyFieldName : partitionKeys) {
                 IFunctionInfo finfoAccess = AsterixBuiltinFunctions
@@ -238,7 +234,6 @@
 
             switch (stmt.getKind()) {
                 case WRITE_FROM_QUERY_RESULT: {
-
                     leafOperator = new WriteResultOperator(targetDatasource, varRef, varRefsForLoading);
                     leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                     break;
@@ -273,7 +268,6 @@
             }
             topOp = leafOperator;
         }
-
         globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
         ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
         return plan;
@@ -294,7 +288,6 @@
             throw new AlgebricksException("Cannot write output to an external dataset.");
         }
         return dataSource;
-
     }
 
     private FileSplit getDefaultOutputFileLocation() throws MetadataException {
@@ -311,7 +304,6 @@
     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;
@@ -410,7 +402,6 @@
         AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
         a.getInputs().add(p.second);
         return new Pair<ILogicalOperator, LogicalVariable>(a, v);
-
     }
 
     @Override
@@ -515,7 +506,6 @@
     private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
             List<Mutable<ILogicalExpression>> args) {
         AbstractFunctionCallExpression f = null;
-
         FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, functionName, arity);
         AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
         FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
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 be6e2af..3440ce8 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
@@ -94,6 +94,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.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
@@ -190,7 +191,7 @@
         }
     }
 
-    private final long txnId;
+    private final JobId jobId;
    private TranslationContext context;
     private String outputDatasetName;
     private ICompiledDmlStatement stmt;
@@ -202,9 +203,9 @@
 
     
     
-    public AqlPlusExpressionToPlanTranslator(long txnId, AqlMetadataProvider metadataProvider,
+    public AqlPlusExpressionToPlanTranslator(JobId jobId, AqlMetadataProvider metadataProvider,
             Counter currentVarCounter, String outputDatasetName, ICompiledDmlStatement stmt) {
-        this.txnId = txnId;
+        this.jobId = jobId;
         this.metadataProvider = metadataProvider;
         this.context = new TranslationContext(currentVarCounter);
         this.outputDatasetName = outputDatasetName;
diff --git a/asterix-app/data/custord-tiny/customer-tiny-neg.adm b/asterix-app/data/custord-tiny/customer-tiny-neg.adm
new file mode 100644
index 0000000..4e8e9ab
--- /dev/null
+++ b/asterix-app/data/custord-tiny/customer-tiny-neg.adm
@@ -0,0 +1,10 @@
+{  "cid": 4440,  "name": "Renea Lavelett",  "age": 72,  "address": {  "number": 8963,  "street": "Oak St.",  "city": "Mountain View" },  "lastorder": {  "oid": 20520,  "total": 6.198883E-4f } }
+{  "cid": 530,  "name": "Clint Coil",  "age": 79,  "address": {  "number": 4491,  "street": "7th St.",  "city": "San Jose" },  "lastorder": {  "oid": 530,  "total": 7.5879574f } }
+{  "cid": 76,  "name": "Marvella Loud",  "age": 72,  "address": {  "number": 6988,  "street": "7th St.",  "city": "Sunnyvale" },  "lastorder": {  "oid": 76,  "total": 12.811708f } }
+{  "cid": 586,  "name": "Tamie Pollara",  "age": 89,  "address": {  "number": 7424,  "street": "Oak St.",  "city": "Sunnyvale" },  "lastorder": {  "oid": 586,  "total": 22.163845f } }
+{  "cid": 59,  "name": "Lance Pracht",  "age": 27,  "address": {  "number": 342,  "street": "Washington St.",  "city": "Mountain View" },  "lastorder": {  "oid": 59,  "total": 26.975239f } }
+{  "cid": 939,  "name": "Larry Gothier",  "age": 34,  "address": {  "number": 1786,  "street": "Park St.",  "city": "Mountain View" },  "lastorder": {  "oid": 939,  "total": 33.49055f } }
+{  "cid": 996,  "name": "Obdulia Dicosmo",  "age": 14,  "address": {  "number": 9237,  "street": "Cedar St.",  "city": "Los Angeles" },  "lastorder": {  "oid": 996,  "total": 94.23889f } }
+{  "cid": 953,  "name": "Elias Leonardo",  "age": 62,  "address": {  "number": 7831,  "street": "Main St.",  "city": "San Jose" },  "lastorder": {  "oid": 953,  "total": 79.990875f } }
+{  "cid": 74,  "name": "Myrtice Cubias",  "age": 11,  "address": {  "number": 9048,  "street": "Park St.",  "city": "San Jose" },  "lastorder": {  "oid": 74,  "total": 47.675938f } }
+{  "cid": 758,  "name": "Curt Savage",  "age": 81,  "address": {  "number": 5651,  "street": "Main St.",  "city": "Seattle" },  "lastorder": {  "oid": 758,  "total": 45.33596f } }
\ No newline at end of file
diff --git a/asterix-app/data/dblp-small/dblp-small-nulls.adm b/asterix-app/data/dblp-small/dblp-small-nulls.adm
new file mode 100644
index 0000000..d943f54
--- /dev/null
+++ b/asterix-app/data/dblp-small/dblp-small-nulls.adm
@@ -0,0 +1,100 @@
+{ "id": 2, "dblpid": "books/acm/kim95/Blakeley95", "title": "OQL[C++]  Extending C++ with an Object Query Capability.", "authors": "José A. Blakeley", "misc": "2002-01-03 69-88 Modern Database Systems db/books/collections/kim95.html#Blakeley95 1995" }
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 6, "dblpid": "books/acm/kim95/DittrichD95", "title": "Where Object-Oriented DBMSs Should Do Better  A Critique Based on Early Experiences.", "authors": "Angelika Kotz Dittrich Klaus R. Dittrich", "misc": "2002-01-03 238-254 1995 Modern Database Systems db/books/collections/kim95.html#DittrichD95" }
+{ "id": 8, "dblpid": "books/acm/kim95/Goodman95", "title": "An Object-Oriented DBMS War Story  Developing a Genome Mapping Database in C++.", "authors": "Nathan Goodman", "misc": "2002-01-03 216-237 1995 Modern Database Systems db/books/collections/kim95.html#Goodman95" }
+{ "id": 10, "dblpid": "books/acm/kim95/KelleyGKRG95", "title": "Schema Architecture of the UniSQL/M Multidatabase System", "authors": "William Kelley Sunit K. Gala Won Kim Tom C. Reyes Bruce Graham", "misc": "2004-03-08 Modern Database Systems books/acm/Kim95 621-648 1995 db/books/collections/kim95.html#KelleyGKRG95" }
+{ "id": 12, "dblpid": "books/acm/kim95/Kim95", "authors": "Won Kim", "misc": "2002-01-03 5-17 1995 Modern Database Systems db/books/collections/kim95.html#Kim95" }
+{ "id": 14, "dblpid": "books/acm/kim95/Kim95b", "title": "Introduction to Part 2  Technology for Interoperating Legacy Databases.", "authors": "Won Kim", "misc": "2002-01-03 515-520 1995 Modern Database Systems db/books/collections/kim95.html#Kim95b" }
+{ "id": 16, "dblpid": "books/acm/kim95/KimG95", "authors": "Won Kim Jorge F. Garza", "misc": "2002-01-03 203-215 1995 Modern Database Systems db/books/collections/kim95.html#KimG95" }
+{ "id": 18, "dblpid": "books/acm/kim95/Kowalski95", "title": "The POSC Solution to Managing E&P Data.", "authors": "Vincent J. Kowalski", "misc": "2002-01-03 281-301 1995 Modern Database Systems db/books/collections/kim95.html#Kowalski95" }
+{ "id": 20, "dblpid": "books/acm/kim95/Lunt95", "title": "Authorization in Object-Oriented Databases.", "authors": "Teresa F. Lunt", "misc": "2002-01-03 130-145 1995 Modern Database Systems db/books/collections/kim95.html#Lunt95" }
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
+{ "id": 24, "dblpid": "books/acm/kim95/OzsuB95", "authors": "M. Tamer Özsu José A. Blakeley", "misc": "2002-01-03 146-174 1995 Modern Database Systems db/books/collections/kim95.html#OzsuB95" }
+{ "id": 26, "dblpid": "books/acm/kim95/Samet95", "title": "Spatial Data Structures.", "authors": "Hanan Samet", "misc": "2004-03-08 361-385 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#Samet95 1995" }
+{ "id": 28, "dblpid": "books/acm/kim95/ShanADDK95", "title": "Pegasus  A Heterogeneous Information Management System.", "authors": "Ming-Chien Shan Rafi Ahmed Jim Davis Weimin Du William Kent", "misc": "2004-03-08 664-682 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#ShanADDK95 1995" }
+{ "id": 30, "dblpid": "books/acm/kim95/SoleyK95", "title": "The OMG Object Model.", "authors": "Richard Mark Soley William Kent", "misc": "2002-01-03 18-41 1995 Modern Database Systems db/books/collections/kim95.html#SoleyK95" }
+{ "id": 32, "dblpid": "books/acm/kim95/Thompson95", "title": "The Changing Database Standards Landscape.", "authors": "Craig W. Thompson", "misc": "2002-01-03 302-317 1995 Modern Database Systems db/books/collections/kim95.html#Thompson95" }
+{ "id": 34, "dblpid": "books/acm/Kim95", "title": "Modern Database Systems  The Object Model, Interoperability, and Beyond.", "authors": "", "misc": "2004-03-08 Won Kim Modern Database Systems ACM Press and Addison-Wesley 1995 0-201-59098-0 db/books/collections/kim95.html" }
+{ "id": 36, "dblpid": "books/aw/kimL89/BjornerstedtH89", "title": "Version Control in an Object-Oriented Architecture.", "authors": "Anders Björnerstedt Christer Hulten", "misc": "2006-02-24 451-485 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#BjornerstedtH89" }
+{ "id": 38, "dblpid": "books/aw/kimL89/CareyDRS89", "title": "Storage Management in EXODUS.", "authors": "Michael J. Carey David J. DeWitt Joel E. Richardson Eugene J. Shekita", "misc": "2002-01-03 341-369 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#CareyDRS89" }
+{ "id": 40, "dblpid": "books/aw/kimL89/DiederichM89", "title": "Objects, Messages, and Rules in Database Design.", "authors": "Jim Diederich Jack Milton", "misc": "2002-01-03 177-197 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#DiederichM89" }
+{ "id": 42, "dblpid": "books/aw/kimL89/FishmanABCCDHHKLLMNRSW89", "title": "Overview of the Iris DBMS.", "authors": "Daniel H. Fishman Jurgen Annevelink David Beech E. C. Chow Tim Connors J. W. Davis Waqar Hasan C. G. Hoch William Kent S. Leichner Peter Lyngbæk Brom Mahbod Marie-Anne Neimat Tore Risch Ming-Chien Shan W. Kevin Wilkinson", "misc": "2002-01-03 219-250 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#FishmanABCCDHHKLLMNRSW89" }
+{ "id": 44, "dblpid": "books/aw/kimL89/KimKD89", "title": "Indexing Techniques for Object-Oriented Databases.", "authors": "Won Kim Kyung-Chang Kim Alfred G. Dale", "misc": "2002-01-03 371-394 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimKD89" }
+{ "id": 46, "dblpid": "books/aw/kimL89/Maier89", "title": "Making Database Systems Fast Enough for CAD Applications.", "authors": "David Maier", "misc": "2002-01-03 573-582 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Maier89" }
+{ "id": 48, "dblpid": "books/aw/kimL89/Moon89", "title": "The Common List Object-Oriented Programming Language Standard.", "authors": "David A. Moon", "misc": "2002-01-03 49-78 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Moon89" }
+{ "id": 50, "dblpid": "books/aw/kimL89/Nierstrasz89", "title": "A Survey of Object-Oriented Concepts.", "authors": "Oscar Nierstrasz", "misc": "2002-01-03 3-21 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Nierstrasz89" }
+{ "id": 52, "dblpid": "books/aw/kimL89/Russinoff89", "title": "Proteus  A Frame-Based Nonmonotonic Inference System.", "authors": "David M. Russinoff", "misc": "2002-01-03 127-150 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#Russinoff89" }
+{ "id": 54, "dblpid": "books/aw/kimL89/SteinLU89", "authors": "Lynn Andrea Stein Henry Lieberman David Ungar", "misc": "2002-01-03 31-48 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#SteinLU89" }
+{ "id": 56, "dblpid": "books/aw/kimL89/TomlinsonS89", "title": "Concurrent Object-Oriented Programming Languages.", "authors": "Chris Tomlinson Mark Scheevel", "misc": "2002-01-03 79-124 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TomlinsonS89" }
+{ "id": 58, "dblpid": "books/aw/kimL89/Wand89", "title": "A Proposal for a Formal Model of Objects.", "authors": "Yair Wand", "misc": "2002-01-03 537-559 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Wand89" }
+{ "id": 60, "dblpid": "books/aw/stonebraker86/RoweS86", "authors": "Lawrence A. Rowe Michael Stonebraker", "misc": "2002-01-03 63-82 1986 The INGRES Papers db/books/collections/Stonebraker86.html#RoweS86 db/books/collections/Stonebraker86/RoweS86.html ingres/P063.pdf" }
+{ "id": 62, "dblpid": "books/aw/stonebraker86/Stonebraker86a", "title": "Supporting Studies on Relational Systems (Introduction to Section 2).", "authors": "Michael Stonebraker", "misc": "2002-01-03 83-85 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86a db/books/collections/Stonebraker86/Stonebraker86a.html ingres/P083.pdf" }
+{ "id": 64, "dblpid": "books/aw/stonebraker86/Stonebraker86c", "authors": "Michael Stonebraker", "misc": "2002-01-03 187-196 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86c db/books/collections/Stonebraker86/Stonebraker86c.html ingres/P187.pdf" }
+{ "id": 66, "dblpid": "books/aw/stonebraker86/Stonebraker86e", "title": "Extended Semantics for the Relational Model (Introduction to Section 5).", "authors": "Michael Stonebraker", "misc": "2002-01-03 313-316 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86e db/books/collections/Stonebraker86/Stonebraker86e.html ingres/P313.pdf" }
+{ "id": 68, "dblpid": "books/aw/stonebraker86/X86", "title": "Title, Preface, Contents.", "authors": "", "misc": "2002-01-03 1986 The INGRES Papers db/books/collections/Stonebraker86.html#X86 db/books/collections/Stonebraker86/X86.html ingres/frontmatter.pdf" }
+{ "id": 70, "dblpid": "books/aw/Knuth86a", "title": "TeX  The Program", "authors": "Donald E. Knuth", "misc": "2002-01-03 Addison-Wesley 1986 0-201-13437-3" }
+{ "id": 72, "dblpid": "books/aw/Lamport86", "title": "LaTeX  User's Guide & Reference Manual", "authors": "Leslie Lamport", "misc": "2002-01-03 Addison-Wesley 1986 0-201-15790-X" }
+{ "id": 74, "dblpid": "books/aw/Lamport2002", "title": "Specifying Systems, The TLA+ Language and Tools for Hardware and Software Engineers", "authors": "Leslie Lamport", "misc": "2005-07-28 Addison-Wesley 2002 0-3211-4306-X http //research.microsoft.com/users/lamport/tla/book.html" }
+{ "id": 76, "dblpid": "books/aw/LewisBK01", "title": "Databases and Transaction Processing  An Application-Oriented Approach", "authors": "Philip M. Lewis Arthur J. Bernstein Michael Kifer", "misc": "2002-01-03 Addison-Wesley 2001 0-201-70872-8" }
+{ "id": 78, "dblpid": "books/aw/LindholmY97", "title": "The Java Virtual Machine Specification", "authors": "Tim Lindholm Frank Yellin", "misc": "2002-01-28 Addison-Wesley 1997 0-201-63452-X" }
+{ "id": 80, "dblpid": "books/aw/Sedgewick83", "title": "Algorithms", "authors": "Robert Sedgewick", "misc": "2002-01-03 Addison-Wesley 1983 0-201-06672-6" }
+{ "id": 82, "dblpid": "conf/focs/AspnesW92", "title": "Randomized Consensus in Expected O(n log ^2 n) Operations Per Processor", "authors": "James Aspnes Orli Waarts", "misc": "2006-04-25 137-146 conf/focs/FOCS33 1992 FOCS db/conf/focs/focs92.html#AspnesW92" }
+{ "id": 84, "dblpid": "conf/stoc/Bloniarz80", "title": "A Shortest-Path Algorithm with Expected Time O(n^2 log n log ^* n)", "authors": "Peter A. Bloniarz", "misc": "2006-04-25 378-384 conf/stoc/STOC12 1980 STOC db/conf/stoc/stoc80.html#Bloniarz80" }
+{ "id": 86, "dblpid": "conf/focs/Megiddo82", "title": "Linear-Time Algorithms for Linear Programming in R^3 and Related Problems", "authors": "Nimrod Megiddo", "misc": "2006-04-25 329-338 conf/focs/FOCS23 1982 FOCS db/conf/focs/focs82.html#Megiddo82" }
+{ "id": 88, "dblpid": "conf/focs/MoffatT85", "title": "An All Pairs Shortest Path Algorithm with Expected Running Time O(n^2 log n)", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2006-04-25 101-105 conf/focs/FOCS26 1985 FOCS db/conf/focs/focs85.html#MoffatT85" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
+{ "id": 92, "dblpid": "conf/stacs/Laue08", "title": "Geometric Set Cover and Hitting Sets for Polytopes in R³.", "authors": "Sören Laue", "misc": "2008-03-04 2008 STACS 479-490 http //drops.dagstuhl.de/opus/volltexte/2008/1367 conf/stacs/2008 db/conf/stacs/stacs2008.html#Laue08" }
+{ "id": 94, "dblpid": "conf/awoc/IbarraJRC88", "title": "On Some Languages in NC.", "authors": "Oscar H. Ibarra Tao Jiang Bala Ravikumar Jik H. Chang", "misc": "2002-08-06 64-73 1988 conf/awoc/1988 AWOC db/conf/awoc/awoc88.html#IbarraJRC88" }
+{ "id": 96, "dblpid": "conf/focs/GalilHLSW82", "title": "An O(n^3 log n) Deterministic and an O(n^3) Probabilistic Isomorphism Test for Trivalent Graphs", "authors": "Zvi Galil Christoph M. Hoffmann Eugene M. Luks Claus-Peter Schnorr Andreas Weber", "misc": "2006-04-25 118-125 conf/focs/FOCS23 1982 FOCS db/conf/focs/focs82.html#GalilHLSW82" }
+{ "id": 98, "dblpid": "conf/focs/GalilT86", "title": "An O(n^2 (m + n log n) log n) Min-Cost Flow Algorithm", "authors": "Zvi Galil Éva Tardos", "misc": "2006-04-25 1-9 conf/focs/FOCS27 1986 FOCS db/conf/focs/focs86.html#GalilT86" }
+{ "id": 100, "dblpid": "series/synthesis/2009Brozos", "title": "The Geometry of Walker Manifolds", "authors": "Miguel Brozos-Vázquez Eduardo García-Río Peter Gilkey Stana Nikcevic Rámon Vázquez-Lorenzo", "misc": "2009-09-06 The Geometry of Walker Manifolds http //dx.doi.org/10.2200/S00197ED1V01Y200906MAS005 http //dx.doi.org/10.2200/S00197ED1V01Y200906MAS005 2009 Synthesis Lectures on Mathematics & Statistics Morgan & Claypool Publishers" }
+{ "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }
+{ "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }
+{ "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }
+{ "id": 7, "dblpid": "books/acm/kim95/Garcia-MolinaH95", "title": "Distributed Databases.", "authors": "Hector Garcia-Molina Meichun Hsu", "misc": "2002-01-03 477-493 1995 Modern Database Systems db/books/collections/kim95.html#Garcia-MolinaH95" }
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
+{ "id": 11, "dblpid": "books/acm/kim95/KemperM95", "title": "Physical Object Management.", "authors": "Alfons Kemper Guido Moerkotte", "misc": "2002-01-03 175-202 1995 Modern Database Systems db/books/collections/kim95.html#KemperM95" }
+{ "id": 13, "dblpid": "books/acm/kim95/Kim95a", "title": "Object-Oriented Database Systems  Promises, Reality, and Future.", "authors": "Won Kim", "misc": "2002-01-03 255-280 1995 Modern Database Systems db/books/collections/kim95.html#Kim95a" }
+{ "id": 15, "dblpid": "books/acm/kim95/KimCGS95", "title": "On Resolving Schematic Heterogeneity in Multidatabase Systems.", "authors": "Won Kim Injun Choi Sunit K. Gala Mark Scheevel", "misc": "2002-01-03 521-550 1995 Modern Database Systems db/books/collections/kim95.html#KimCGS95" }
+{ "id": 17, "dblpid": "books/acm/kim95/KimK95", "title": "On View Support in Object-Oriented Databases Systems.", "authors": "Won Kim William Kelley", "misc": "2002-01-03 108-129 1995 Modern Database Systems db/books/collections/kim95.html#KimK95" }
+{ "id": 19, "dblpid": "books/acm/kim95/KriegerA95", "title": "C++ Bindings to an Object Database.", "authors": "David Krieger Tim Andrews", "misc": "2002-01-03 89-107 1995 Modern Database Systems db/books/collections/kim95.html#KriegerA95" }
+{ "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }
+{ "id": 23, "dblpid": "books/acm/kim95/Omiecinski95", "title": "Parallel Relational Database Systems.", "authors": "Edward Omiecinski", "misc": "2002-01-03 494-512 1995 Modern Database Systems db/books/collections/kim95.html#Omiecinski95" }
+{ "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }
+{ "id": 27, "dblpid": "books/acm/kim95/SametA95", "title": "Spatial Data Models and Query Processing.", "authors": "Hanan Samet Walid G. Aref", "misc": "2002-01-03 338-360 1995 Modern Database Systems db/books/collections/kim95.html#SametA95" }
+{ "id": 29, "dblpid": "books/acm/kim95/Snodgrass95", "title": "Temporal Object-Oriented Databases  A Critical Comparison.", "authors": "Richard T. Snodgrass", "misc": "2002-01-03 386-408 1995 Modern Database Systems db/books/collections/kim95.html#Snodgrass95" }
+{ "id": 31, "dblpid": "books/acm/kim95/Stout95", "title": "EDA/SQL.", "authors": "Ralph L. Stout", "misc": "2004-03-08 649-663 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#Stout95 1995" }
+{ "id": 33, "dblpid": "books/acm/kim95/BreitbartR95", "title": "Overview of the ADDS System.", "authors": "Yuri Breitbart Tom C. Reyes", "misc": "2009-06-12 683-701 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartR95 1995" }
+{ "id": 35, "dblpid": "books/ap/MarshallO79", "title": "Inequalities  Theory of Majorization and Its Application.", "authors": "Albert W. Marshall Ingram Olkin", "misc": "2002-01-03 Academic Press 1979 0-12-473750-1" }
+{ "id": 37, "dblpid": "books/aw/kimL89/BretlMOPSSWW89", "title": "The GemStone Data Management System.", "authors": "Robert Bretl David Maier Allen Otis D. Jason Penney Bruce Schuchardt Jacob Stein E. Harold Williams Monty Williams", "misc": "2002-01-03 283-308 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#BretlMOPSSWW89" }
+{ "id": 39, "dblpid": "books/aw/kimL89/Decouchant89", "title": "A Distributed Object Manager for the Smalltalk-80 System.", "authors": "Dominique Decouchant", "misc": "2002-01-03 487-520 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Decouchant89" }
+{ "id": 41, "dblpid": "books/aw/kimL89/EllisG89", "title": "Active Objects  Ealities and Possibilities.", "authors": "Clarence A. Ellis Simon J. Gibbs", "misc": "2002-01-03 561-572 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#EllisG89" }
+{ "id": 43, "dblpid": "books/aw/kimL89/KimBCGW89", "title": "Features of the ORION Object-Oriented Database System.", "authors": "Won Kim Nat Ballou Hong-Tai Chou Jorge F. Garza Darrell Woelk", "misc": "2002-01-03 251-282 Object-Oriented Concepts, Databases, and Applications ACM Press and Addison-Wesley 1989 db/books/collections/kim89.html#KimBCGW89" }
+{ "id": 45, "dblpid": "books/aw/kimL89/King89", "title": "My Cat Is Object-Oriented.", "authors": "Roger King", "misc": "2002-01-03 23-30 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#King89" }
+{ "id": 47, "dblpid": "books/aw/kimL89/MellenderRS89", "authors": "Fred Mellender Steve Riegel Andrew Straw", "misc": "2002-01-03 423-450 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#MellenderRS89" }
+{ "id": 49, "dblpid": "books/aw/kimL89/Moss89", "title": "Object Orientation as Catalyst for Language-Database Inegration.", "authors": "J. Eliot B. Moss", "misc": "2002-01-03 583-592 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#Moss89" }
+{ "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }
+{ "id": 53, "dblpid": "books/aw/kimL89/SkarraZ89", "title": "Concurrency Control and Object-Oriented Databases.", "authors": "Andrea H. Skarra Stanley B. Zdonik", "misc": "2002-01-03 395-421 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#SkarraZ89" }
+{ "id": 55, "dblpid": "books/aw/kimL89/TarltonT89", "title": "Pogo  A Declarative Representation System for Graphics.", "authors": "Mark A. Tarlton P. Nong Tarlton", "misc": "2002-01-03 151-176 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TarltonT89" }
+{ "id": 57, "dblpid": "books/aw/kimL89/TsichritzisN89", "title": "Directions in Object-Oriented Research.", "authors": "Dennis Tsichritzis Oscar Nierstrasz", "misc": "2002-01-03 523-536 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#TsichritzisN89" }
+{ "id": 59, "dblpid": "books/aw/kimL89/WeiserL89", "title": "OZ+  An Object-Oriented Database System.", "authors": "Stephen P. Weiser Frederick H. Lochovsky", "misc": "2002-01-03 309-337 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#WeiserL89" }
+{ "id": 61, "dblpid": "books/aw/stonebraker86/Stonebraker86", "title": "Design of Relational Systems (Introduction to Section 1).", "authors": "Michael Stonebraker", "misc": "2002-01-03 1-3 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86 db/books/collections/Stonebraker86/Stonebraker86.html ingres/P001.pdf" }
+{ "id": 63, "dblpid": "books/aw/stonebraker86/Stonebraker86b", "title": "Distributed Database Systems (Introduction to Section 3).", "authors": "Michael Stonebraker", "misc": "2002-01-03 183-186 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86b db/books/collections/Stonebraker86/Stonebraker86b.html ingres/P183.pdf" }
+{ "id": 65, "dblpid": "books/aw/stonebraker86/Stonebraker86d", "title": "User Interfaces for Database Systems (Introduction to Section 4).", "authors": "Michael Stonebraker", "misc": "2002-01-03 243-245 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86d db/books/collections/Stonebraker86/Stonebraker86d.html ingres/P243.pdf" }
+{ "id": 67, "dblpid": "books/aw/stonebraker86/Stonebraker86f", "authors": "Michael Stonebraker", "misc": "2002-01-03 393-394 1986 The INGRES Papers db/books/collections/Stonebraker86.html#Stonebraker86f db/books/collections/Stonebraker86/Stonebraker86f.html ingres/P393.pdf" }
+{ "id": 69, "dblpid": "books/aw/stonebraker86/X86a", "title": "References.", "authors": "", "misc": "2002-01-03 429-444 1986 The INGRES Papers db/books/collections/Stonebraker86.html#X86a db/books/collections/Stonebraker86/X86a.html ingres/P429.pdf" }
+{ "id": 71, "dblpid": "books/aw/AbiteboulHV95", "title": "Foundations of Databases.", "authors": "Serge Abiteboul Richard Hull Victor Vianu", "misc": "2002-01-03 Addison-Wesley 1995 0-201-53771-0 AHV/Toc.pdf ... ... journals/tods/AstrahanBCEGGKLMMPTWW76 books/bc/AtzeniA93 journals/tcs/AtzeniABM82 journals/jcss/AbiteboulB86 journals/csur/AtkinsonB87 conf/pods/AtzeniB87 journals/vldb/AbiteboulB95 conf/sigmod/AbiteboulB91 conf/dood/AtkinsonBDDMZ89 conf/vldb/AlbanoBGO93 ... conf/icdt/Abiteboul88 journals/ipl/Abiteboul89 conf/ds/Abrial74 journals/tods/AhoBU79 books/mk/minker88/AptBW88 conf/vldb/AroraC78 conf/stoc/AfratiC89 journals/tods/AlbanoCO85 conf/pods/AfratiCY91 conf/pods/AusielloDM85 conf/vldb/AbiteboulG85 journals/jacm/AjtaiG87 conf/focs/AjtaiG89 journals/tods/AbiteboulG91 ... ... journals/tods/AbiteboulH87 conf/sigmod/AbiteboulH88 ... conf/sigmod/AbiteboulK89 journals/tcs/AbiteboulKG91 journals/jcss/AbiteboulKRW95 conf/sigmod/AbiteboulLUW93 conf/pods/AtzeniP82 conf/pods/AfratiP87 conf/pods/AptP87 conf/wg/AndriesP91 conf/pods/AfratiPPRSU86 books/el/leeuwen90/Apt90 conf/ifip/Armstrong74 journals/siamcomp/AhoSSU81 journals/tods/AhoSU79 journals/siamcomp/AhoSU79 conf/pods/AbiteboulSV90 journals/is/AtzeniT93 conf/popl/AhoU79 conf/pods/AbiteboulV87 conf/jcdkb/AbiteboulV88 journals/jacm/AbiteboulV88 conf/pods/AbiteboulV88 journals/jacm/AbiteboulV89 journals/jcss/AbiteboulV90 journals/jcss/AbiteboulV91 conf/stoc/AbiteboulV91 journals/amai/AbiteboulV91 journals/jcss/AbiteboulV95 journals/jacm/AptE82 conf/coco/AbiteboulVV92 conf/iclp/AptB88 conf/oopsla/BobrowKKMSZ86 journals/tse/BatoryBGSTTW88 conf/mfcs/Bancilhon78 ... conf/db-workshops/Bancilhon85 books/el/leeuwen90/Barendregt90 ... journals/tods/BeeriB79 books/el/leeuwen90/BerstelB90 conf/icdt/BeneventanoB92 conf/vldb/BernsteinBC80 conf/vldb/BeeriBG78 conf/sigmod/BorgidaBMR89 journals/tods/BunemanC79 journals/jacm/BernsteinC81 conf/dbpl/BancilhonCD89 books/bc/tanselCGSS93/BaudinetCW93 conf/sigmod/BiskupDB79 journals/jacm/BeeriDFS84 books/mk/BancilhonDK92 conf/edbt/BryDM88 conf/pods/BunemanDW88 journals/jcss/BunemanDW91 journals/tods/Beeri80 journals/dke/Beeri90 ... journals/tods/Bernstein76 conf/lics/BidoitF87 journals/iandc/BidoitF91 conf/sigmod/BeeriFH77 conf/stoc/BeeriFMMUY81 journals/jacm/BeeriFMY83 journals/tods/BunemanFN82 journals/siamcomp/BernsteinG81 journals/iandc/BlassGK85 conf/ijcai/BrachmanGL85 journals/tods/BernsteinGWRR81 books/aw/BernsteinHG87 ... journals/tcs/Bidoit91 journals/tcs/Biskup80 conf/adbt/Biskup79 journals/tods/Biskup83 journals/tcs/BunemanJO91 journals/tods/BeeriK86 conf/pods/BeeriKBR87 conf/icdt/BidoitL90 journals/csur/BatiniL86 conf/sigmod/BlakeleyLT86 conf/vldb/BeeriM91 conf/sigmod/BlakeleyMG93 journals/siamcomp/BeeriMSU81 conf/pods/BancilhonMSU86 conf/pods/BeeriNRST87 journals/software/Borgida85 conf/icalp/BraP83 conf/fgcs/BalbinMR88 ... conf/pods/BeeriR87 journals/jlp/BalbinR87 conf/sigmod/BancilhonR86 books/mk/minker88/BancilhonR88 journals/jlp/BeeriR91 conf/vldb/BancilhonRS82 conf/pods/BeeriRSS92 conf/dood/Bry89 journals/tods/BancilhonS81 journals/cogsci/BrachmanS85 journals/tods/BergamaschiS92 conf/sigmod/BernsteinST75 conf/dbpl/TannenBN91 conf/icdt/TannenBW92 ... journals/jacm/BeeriV84 conf/icalp/BeeriV81 conf/adbt/BeeriV79 journals/siamcomp/BeeriV84 journals/iandc/BeeriV84 journals/jacm/BeeriV84 journals/tcs/BeeriV85 journals/ibmrd/ChamberlinAEGLMRW76 ... journals/iandc/Cardelli88 books/mk/Cattell94 conf/sigmod/CacaceCCTZ90 conf/vldb/CastilhoCF82 conf/adbt/CasanovaF82 conf/focs/CaiFI89 journals/jcss/CasanovaFP84 conf/stoc/CosmadakisGKV88 conf/dood/CorciuloGP93 books/sp/CeriGT90 conf/focs/ChandraH80 journals/jcss/ChandraH80 journals/jcss/ChandraH82 journals/jlp/ChandraH85 conf/popl/Chandra81 conf/adbt/Chang79 conf/pods/Chandra88 ... journals/tods/Chen76 conf/ride/ChenHM94 conf/icde/Chomicki92 conf/pods/Chomicki92 ... ... ... conf/stoc/CosmadakisK85 journals/acr/CosmadakisK86 ... journals/jcss/CosmadakisKS86 journals/jacm/CosmadakisKV90 ... conf/pods/CalvaneseL94 conf/adbt/Clark77 conf/stoc/ChandraLM81 conf/stoc/ChandraM77 conf/pods/ConsensM90 conf/sigmod/ConsensM93 conf/icdt/ConsensM90 journals/cacm/Codd70 conf/sigmod/Codd71a persons/Codd71a persons/Codd72 conf/ifip/Codd74 ... conf/sigmod/Codd79 journals/cacm/Codd82 ... conf/sigmod/Cohen89 journals/cacm/Cohen90 ... journals/jcss/Cook74 conf/pods/Cosmadakis83 conf/focs/Cosmadakis87 books/el/leeuwen90/Courcelle90a journals/jacm/CosmadakisP84 conf/edbt/CeriCGLLTZ88 ... conf/vldb/CeriT87 conf/vldb/CasanovaTF88 ... conf/pods/CasanovaV83 journals/siamcomp/ChandraV85 conf/pods/ChaudhuriV92 conf/pods/ChaudhuriV93 conf/pods/ChaudhuriV94 journals/csur/CardelliW85 conf/pods/ChenW89 conf/pods/CohenW89 conf/vldb/CeriW90 conf/vldb/CeriW91 conf/iclp/ChenW92 conf/vldb/CeriW93 ... conf/birthday/Dahlhaus87 conf/vldb/Date81 books/aw/Date86 ... conf/dbpl/Dayal89 journals/tods/DayalB82 journals/ibmrd/DelobelC73 conf/icde/DelcambreD89 ... journals/tods/Delobel78 journals/jacm/Demolombe92 journals/tods/DateF92 ... conf/vldb/DayalHL91 journals/jacm/Paola69a conf/caap/DahlhausM86 journals/acr/DAtriM86 journals/iandc/DahlhausM92 conf/sigmod/DerrMP93 conf/vldb/MaindrevilleS88 conf/pods/Dong92 conf/adbt/BraP82 ... conf/dbpl/DongS91 journals/iandc/DongS95 conf/dbpl/DongS93 conf/dbpl/DongS93 conf/icdt/DongT92 conf/vldb/DenninghoffV91 conf/pods/DenninghoffV93 ... ... books/acm/kim95/DayalHW95 ... conf/pods/EiterGM94 conf/pods/Escobar-MolanoHJ93 ... books/el/leeuwen90/Emerson90 books/bc/ElmasriN89 ... conf/icse/Eswaran76 conf/sigmod/EpsteinSW78 ... ... conf/vldb/Fagin77 journals/tods/Fagin77 conf/sigmod/Fagin79 journals/tods/Fagin81 journals/ipl/FaginV83 journals/jacm/Fagin82 journals/jacm/Fagin83 journals/tcs/Fagin93 books/sp/kimrb85/FurtadoC85 ... journals/jlp/Fitting85a journals/tcs/FischerJT83 journals/acr/FaginKUV86 conf/icdt/FernandezM92 journals/tods/FaginMU82 conf/vldb/FaloutsosNS91 ... journals/ai/Forgy82 ... conf/sigmod/Freytag87 ... journals/siamcomp/FischerT83 journals/siamcomp/FaginMUY83 conf/pods/FaginUV83 conf/icalp/FaginV84 ... ... ... ... conf/sigmod/GraefeD87 conf/ride/GatziuD94 conf/sigmod/GardarinM86 conf/sigmod/GyssensG88 journals/tcs/GinsburgH83a journals/jacm/GinsburgH86 ... books/bc/tanselCGSS93/Ginsburg93 books/fm/GareyJ79 journals/jacm/GrantJ82 conf/vldb/GehaniJ91 conf/vldb/GhandeharizadehHJCELLTZ93 journals/tods/GhandeharizadehHJ96 conf/vldb/GehaniJS92 ... conf/sigmod/GehaniJS92 ... conf/deductive/GuptaKM92 conf/pods/GurevichL82 conf/iclp/GelfondL88 conf/adbt/77 journals/csur/GallaireMN84 conf/pods/GrahneMR92 conf/sigmod/GuptaMS93 conf/lics/GaifmanMSV87 journals/jacm/GaifmanMSV93 journals/jacm/GrahamMV86 conf/csl/GradelO92 ... conf/pods/Gottlob87 conf/pods/GyssensPG90 conf/dood/GiannottiPSZ91 books/aw/GoldbergR83 journals/acr/GrahneR86 journals/ipl/Grant77 ... journals/iandc/Grandjean83 conf/vldb/Grahne84 ... journals/csur/Graefe93 books/sp/Greibach75 journals/tods/GoodmanS82 journals/jcss/GoodmanS84 conf/focs/GurevichS85 ... conf/pods/GrumbachS94 conf/sigmod/GangulyST90 ... journals/tcs/Gunter92 ... ... ... ... conf/pods/GrahamV84 conf/pods/GrumbachV91 conf/icde/GardarinV92 conf/sigmod/GraefeW89 ... journals/jacm/GinsburgZ82 conf/vldb/GottlobZ88 ... ... journals/sigmod/Hanson89 ... journals/cacm/Harel80 journals/tkde/HaasCLMWLLPCS90 conf/lics/Hella92 journals/iandc/Herrmann95 conf/pods/HirstH93 conf/vldb/HullJ91 conf/ewdw/HullJ90 journals/csur/HullK87 journals/tods/HudsonK89 conf/lics/HillebrandKM93 conf/nato/HillebrandKR93 conf/jcdkb/HsuLM88 journals/ipl/HoneymanLY80 journals/tods/HammerM81 conf/adbt/HenschenMN82 ... journals/jacm/HenschenN84 journals/jacm/Honeyman82 conf/sigmod/HullS89 conf/pods/HullS89 journals/acta/HullS94 journals/jcss/HullS93 conf/fodo/HullTY89 journals/jcss/Hull83 journals/jacm/Hull84 journals/tcs/Hull85 journals/siamcomp/Hull86 ... conf/vldb/Hulin89 ... journals/jacm/HullY84 conf/vldb/HullY90 conf/pods/HullY91 conf/sigmod/IoannidisK90 journals/jcss/ImielinskiL84 conf/adbt/Imielinski82 journals/jcss/Immerman82 journals/iandc/Immerman86 ... journals/siamcomp/Immerman87 conf/pods/ImielinskiN88 conf/vldb/IoannidisNSS92 conf/sigmod/ImielinskiNV91 conf/dood/ImielinskiNV91 conf/vldb/Ioannidis85 journals/jacm/Jacobs82 conf/dbpl/JacobsH91 journals/csur/JarkeK84 journals/jcss/JohnsonK84 conf/popl/JaffarL87 books/el/leeuwen90/Johnson90 journals/jacm/Joyner76 conf/pods/JaeschkeS82 ... books/mk/minker88/Kanellakis88 books/el/leeuwen90/Kanellakis90 conf/oopsla/KhoshafianC86 conf/edbt/KotzDM88 conf/jcdkb/Keller82 conf/pods/Keller85 journals/computer/Keller86 ... journals/tods/Kent79 ... journals/ngc/RohmerLK86 conf/tacs/KanellakisG94 conf/jcdkb/Kifer88 conf/pods/KanellakisKR90 conf/sigmod/KiferKS92 ... conf/icdt/KiferL86 books/aw/KimL89 ... journals/tods/Klug80 journals/jacm/Klug82 journals/jacm/Klug88 journals/jacm/KiferLW95 conf/kr/KatsunoM91 journals/ai/KatsunoM92 conf/jcdkb/KrishnamurthyN88 journals/csur/Knight89 ... journals/iandc/Kolaitis91 journals/ai/Konolige88 conf/ifip/Kowalski74 journals/jacm/Kowalski75 conf/bncod/Kowalski84 conf/vldb/KoenigP81 journals/tods/KlugP82 ... conf/pods/KolaitisP88 conf/pods/KiferRS88 conf/sigmod/KrishnamurthyRS88 books/mg/SilberschatzK91 conf/iclp/KempT88 conf/sigmod/KellerU84 conf/dood/Kuchenhoff91 ... journals/jlp/Kunen87 conf/iclp/Kunen88 conf/pods/Kuper87 conf/pods/Kuper88 conf/ppcp/Kuper93 conf/pods/KuperV84 conf/stoc/KolaitisV87 journals/tcs/KarabegV90 journals/iandc/KolaitisV90 conf/pods/KolaitisV90 journals/tods/KarabegV91 journals/iandc/KolaitisV92 journals/tcs/KuperV93 journals/tods/KuperV93 journals/tse/KellerW85 conf/pods/KiferW89 conf/jcdkb/Lang88 books/el/Leeuwen90 ... journals/jcss/Leivant89 ... journals/iandc/Leivant90 ... conf/db-workshops/Levesque82 journals/ai/Levesque84 conf/mfdbs/Libkin91 conf/er/Lien79 journals/jacm/Lien82 books/mk/minker88/Lifschitz88 ... journals/tcs/Lindell91 journals/tods/Lipski79 journals/jacm/Lipski81 journals/tcs/LeratL86 journals/cj/LeveneL90 books/sp/Lloyd87 conf/pods/LakshmananM89 conf/tlca/LeivantM93 conf/sigmod/LaverMG83 conf/pods/LiptonN90 journals/jcss/LucchesiO78 conf/sigmod/Lohman88 ... conf/ijcai/Lozinskii85 books/ph/LewisP81 ... conf/sigmod/LecluseRV88 journals/is/LipeckS87 journals/jlp/LloydST87 journals/tods/LingTK81 conf/sigmod/LyngbaekV87 conf/dood/LefebvreV89 conf/pods/LibkinW93 conf/dbpl/LibkinW93 journals/jacm/Maier80 books/cs/Maier83 ... conf/vldb/Makinouchi77 conf/icalp/Makowsky81 ... conf/icdt/Malvestuto86 conf/aaai/MacGregorB92 journals/tods/MylopoulosBW80 conf/sigmod/McCarthyD89 journals/csur/MishraE92 conf/sigmod/MumickFPR90 books/mk/Minker88 journals/jlp/Minker88 conf/vldb/MillerIR93 journals/is/MillerIR94 journals/iandc/Mitchell83 conf/pods/Mitchell83 conf/vldb/MendelzonM79 journals/tods/MaierMS79 journals/jcss/MaierMSU80 conf/pods/MendelzonMW94 journals/debu/MorrisNSUG87 journals/ai/Moore85 conf/vldb/Morgenstern83 conf/pods/Morris88 ... conf/pods/MannilaR85 ... journals/jlp/MinkerR90 books/aw/MannilaR92 journals/acr/MaierRW86 ... journals/tods/MarkowitzS92 conf/pods/Marchetti-SpaccamelaPS87 journals/jacm/MaierSY81 conf/iclp/MorrisUG86 journals/tods/MaierUV84 conf/iclp/MorrisUG86 journals/acta/MakowskyV86 books/bc/MaierW88 books/mk/minker88/ManchandraW88 conf/pods/Naughton86 conf/sigmod/NgFS91 ... conf/vldb/Nejdl87 conf/adbt/NicolasM77 conf/sigmod/Nicolas78 journals/acta/Nicolas82 conf/ds/76 conf/pods/NaqviK88 journals/tods/NegriPS91 conf/vldb/NaughtonRSU89 conf/pods/NaughtonS87 ... ... conf/vldb/Osborn79 ... journals/tods/OzsoyogluY87 conf/adbt/Paige82 ... books/cs/Papadimitriou86 ... journals/ipl/Paredaens78 ... books/sp/ParedaensBGG89 journals/ai/Andersen91 books/el/leeuwen90/Perrin90 journals/ins/Petrov89 conf/pods/ParedaensG88 conf/pods/PatnaikI94 conf/adbt/ParedaensJ79 journals/csur/PeckhamM88 ... ... conf/sigmod/ParkerP80 ... conf/iclp/Przymusinski88 conf/pods/Przymusinski89 ... conf/vldb/ParkerSV92 conf/aaai/PearlV87 journals/ai/PereiraW80a conf/pods/PapadimitriouY92 journals/tkde/QianW91 ... journals/jlp/Ramakrishnan91 conf/pods/RamakrishnanBS87 ... conf/adbt/Reiter77 journals/ai/Reiter80 conf/db-workshops/Reiter82 journals/jacm/Reiter86 journals/tods/Rissanen77 conf/mfcs/Rissanen78 conf/pods/Rissanen82 ... journals/ngc/RohmerLK86 journals/jacm/Robinson65 ... conf/pods/Ross89 ... ... conf/sigmod/RoweS79 conf/sigmod/RichardsonS91 journals/debu/RamamohanaraoSBPNTZD87 conf/vldb/RamakrishnanSS92 conf/sigmod/RamakrishnanSSS93 conf/pods/RamakrishnanSUV89 journals/jcss/RamakrishnanSUV93 journals/jlp/RamakrishnanU95 conf/sigmod/SelingerACLP79 conf/sigmod/Sagiv81 journals/tods/Sagiv83 books/mk/minker88/Sagiv88 conf/slp/Sagiv90 conf/sigmod/Sciore81 journals/jacm/Sciore82 conf/pods/Sciore83 journals/acr/Sciore86 journals/jacm/SagivDPF81 conf/pods/X89 ... journals/ai/SmithG85 books/mk/minker88/Shepherdson88 journals/tods/Shipman81 conf/pods/Shmueli87 conf/iclp/SekiI88 conf/sigmod/ShmueliI84 journals/tc/Sickel76 journals/jsc/Siekmann89 conf/sigmod/StonebrakerJGP90 conf/vldb/SimonKM92 journals/csur/ShethL90 conf/pods/SeibL91 conf/sigmod/SuLRD93 conf/adbt/SilvaM79 journals/sigmod/Snodgrass90 journals/sigmod/Soo91 conf/pods/SuciuP94 conf/sigmod/StonebrakerR86 conf/slp/SudarshanR93 conf/pods/SagivS86 journals/cacm/Stonebraker81 books/mk/Stonebraker88 journals/tkde/Stonebraker92 books/aw/Stroustrup91 journals/jacm/SadriU82 conf/vldb/Su91 conf/pods/SagivV89 journals/jacm/SagivW82 journals/tods/StonebrakerWKH76 journals/jacm/SagivY80 conf/pods/SaccaZ86 journals/tcs/SaccaZ88 ... conf/pods/SaccaZ90 ... ... books/bc/TanselCGJSS93 ... journals/acr/ThomasF86 ... ... ... ... journals/tcs/Topor87 ... books/mk/minker88/ToporS88 ... journals/siamcomp/TarjanY84 journals/csur/TeoreyYF86 journals/algorithmica/UllmanG88 conf/pods/Ullman82 books/cs/Ullman82 journals/tods/Ullman85 books/cs/Ullman88 conf/pods/Ullman89 books/cs/Ullman89 conf/sigmod/Gelder86 ... conf/pods/BusscheG92 conf/focs/BusscheGAG92 conf/pods/BusscheP91 conf/slp/Gelder86 conf/pods/Gelder89 conf/pods/GelderRS88 journals/jacm/GelderRS91 journals/tods/GelderT91 journals/ipl/Vardi81 conf/stoc/Vardi82 conf/focs/Vardi82 journals/acta/Vardi83 journals/jcss/Vardi84 conf/pods/Vardi85 conf/pods/Vardi86 journals/jcss/Vardi86 ... conf/pods/Vardi88 conf/sigmod/Vassiliou79 ... ... journals/jacm/EmdenK76 conf/nf2/SchollABBGPRV87 journals/jacm/Vianu87 journals/acta/Vianu87 conf/eds/Vieille86 conf/iclp/Vieille87 ... conf/eds/Vieille88 journals/tcs/Vieille89 ... journals/tcs/VianuV92 conf/sigmod/WidomF90 conf/icde/WangH92 conf/pos/WidjojoHW90 journals/computer/Wiederhold92 conf/pods/Wilkins86 conf/pods/Winslett88 conf/sigmod/WolfsonO90 conf/pods/Wong93 conf/sigmod/WolfsonS88 journals/ibmrd/WangW75 journals/tods/WongY76 conf/vldb/Yannakakis81 journals/csur/YuC84 ... journals/jcss/YannakakisP82 ... journals/tods/Zaniolo82 journals/jcss/Zaniolo84 ... conf/edbt/ZhouH90 journals/ibmsj/Zloof77 books/mk/ZdonikM90 db/books/dbtext/abiteboul95.html" }
+{ "id": 73, "dblpid": "books/aw/AhoHU74", "title": "The Design and Analysis of Computer Algorithms.", "authors": "Alfred V. Aho John E. Hopcroft Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1974 0-201-00029-6" }
+{ "id": 75, "dblpid": "books/aw/AhoHU83", "title": "Data Structures and Algorithms.", "authors": "Alfred V. Aho John E. Hopcroft Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1983 0-201-00023-7" }
+{ "id": 77, "dblpid": "books/aw/AhoKW88", "title": "The AWK Programming Language", "authors": "Alfred V. Aho Brian W. Kernighan Peter J. Weinberger", "misc": "2002-01-03 Addison-Wesley 1988" }
+{ "id": 79, "dblpid": "books/aw/AhoSU86", "title": "Compilers  Princiles, Techniques, and Tools.", "authors": "Alfred V. Aho Ravi Sethi Jeffrey D. Ullman", "misc": "2002-01-03 Addison-Wesley 1986 0-201-10088-6" }
+{ "id": 81, "dblpid": "journals/siamcomp/AspnesW96", "authors": "James Aspnes Orli Waarts", "misc": "2002-01-03 1024-1044 1996 25 SIAM J. Comput. 5 db/journals/siamcomp/siamcomp25.html#AspnesW96" }
+{ "id": 83, "dblpid": "journals/siamcomp/Bloniarz83", "title": "A Shortest-Path Algorithm with Expected Time O(n² log n log* n).", "authors": "Peter A. Bloniarz", "misc": "2002-01-03 588-600 1983 12 SIAM J. Comput. 3 db/journals/siamcomp/siamcomp12.html#Bloniarz83" }
+{ "id": 85, "dblpid": "journals/siamcomp/Megiddo83a", "title": "Linear-Time Algorithms for Linear Programming in R³ and Related Problems.", "authors": "Nimrod Megiddo", "misc": "2002-01-03 759-776 1983 12 SIAM J. Comput. 4 db/journals/siamcomp/siamcomp12.html#Megiddo83a" }
+{ "id": 87, "dblpid": "journals/siamcomp/MoffatT87", "title": "An All Pairs Shortest Path Algorithm with Expected Time O(n² log n).", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2002-01-03 1023-1031 1987 16 SIAM J. Comput. 6 db/journals/siamcomp/siamcomp16.html#MoffatT87" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 91, "dblpid": "journals/corr/abs-0802-2861", "title": "Geometric Set Cover and Hitting Sets for Polytopes in $R^3$", "authors": "Sören Laue", "misc": "2008-03-03 http //arxiv.org/abs/0802.2861 2008 CoRR abs/0802.2861 db/journals/corr/corr0802.html#abs-0802-2861 informal publication" }
+{ "id": 93, "dblpid": "journals/iandc/IbarraJCR91", "authors": "Oscar H. Ibarra Tao Jiang Jik H. Chang Bala Ravikumar", "misc": "2006-04-25 86-106 Inf. Comput. January 1991 90 1 db/journals/iandc/iandc90.html#IbarraJCR91" }
+{ "id": 95, "dblpid": "journals/jacm/GalilHLSW87", "title": "An O(n³log n) deterministic and an O(n³) Las Vegs isomorphism test for trivalent graphs.", "authors": "Zvi Galil Christoph M. Hoffmann Eugene M. Luks Claus-Peter Schnorr Andreas Weber", "misc": "2003-11-20 513-531 1987 34 J. ACM 3 http //doi.acm.org/10.1145/28869.28870 db/journals/jacm/jacm34.html#GalilHLSW87" }
+{ "id": 97, "dblpid": "journals/jacm/GalilT88", "title": "An O(n²(m + n log n)log n) min-cost flow algorithm.", "authors": "Zvi Galil Éva Tardos", "misc": "2003-11-20 374-386 1988 35 J. ACM 2 http //doi.acm.org/10.1145/42282.214090 db/journals/jacm/jacm35.html#GalilT88" }
+{ "id": 99, "dblpid": "series/synthesis/2009Weintraub", "title": "Jordan Canonical Form  Theory and Practice", "authors": "Steven H. Weintraub", "misc": "2009-09-06 Jordan Canonical Form  Theory and Practice http //dx.doi.org/10.2200/S00218ED1V01Y200908MAS006 http //dx.doi.org/10.2200/S00218ED1V01Y200908MAS006 2009 Synthesis Lectures on Mathematics & Statistics Morgan & Claypool Publishers" }
diff --git a/asterix-app/data/spatial/spatialData.json b/asterix-app/data/spatial/spatialData.json
index e2dd151..9c78064 100644
--- a/asterix-app/data/spatial/spatialData.json
+++ b/asterix-app/data/spatial/spatialData.json
@@ -16,4 +16,6 @@
 {"id": 16, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,35.0 10.0")}
 {"id": 17, "point": point("4.1,7.0"), "kwds": "sign ahead", "line1": line("4.0,7.0 9.0,7.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("0.0,6.0 0.0,0.0 3.0,0.0 4.0,1.0 6.0,1.0 8.0,0.0 12.0,0.0 13.0,2.0 8.0,2.0 8.0,4.0 11.0,4.0 11.0,6.0 6.0,6.0 4.0,3.0 2.0,6.0"), "poly2": polygon("5.0,1.0 5.0,4.0 7.0,4.0 7.0,1.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,51.0 10.0")}
 {"id": 18, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "poly2": polygon("6.0,3.0 7.0,5.0 6.0,7.0 5.0,5.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("43.0,45.0 12.0")}
-{"id": 19, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "poly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("65.0,2.0 13.0")}
\ No newline at end of file
+{"id": 19, "point": point("-2.0,3.0"), "kwds": "sign ahead", "line1": line("1.0,2.0 3.0,4.0"), "line2": line("5.0,8.0 5.0,1.0"), "poly1": polygon("5.0,1.0 7.0,1.0 7.0,4.0 6.0,2.0 5.0,4.0"), "poly2": polygon("6.0,1.0 7.0,5.0 6.0,7.0 5.0,5.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("65.0,2.0 13.0")}
+{"id": 20, "point": point("4.0,3.0"), "kwds": "sign ahead", "line1": line("20.0,20.0 30.0,40.0"), "line2": line("5.0,8.0 0.0,1.0"), "poly1": polygon("4.0,1.0 4.0,4.0 12.0,4.0 12.0,1.0"), "poly2": polygon("50.0,10.0 50.0,40.0 70.0,40.0 70.0,10.0"), "rec": rectangle("0.0,0.0 4.0,4.0"), "circle" : circle("1.0,23.0 12.0")}
+{"id": 21, "point": point("0.0,5.0"), "kwds": "sign ahead", "line1": line("0.0,5.0 0.0,40.0"), "line2": line("5.0,8.0 0.0,1.0"), "poly1": polygon("5.1,5.1 14.0,14.0 22.0,14.0 22.0,10.0"), "poly2": polygon("50.0,10.0 50.0,40.0 70.0,40.0 70.0,10.0"), "rec": rectangle("0.0,0.0 5.1,5.1"), "circle" : circle("1.0,23.0 12.0")}
\ No newline at end of file
diff --git a/asterix-app/pom.xml b/asterix-app/pom.xml
index 5c0a1a6..e55231f 100644
--- a/asterix-app/pom.xml
+++ b/asterix-app/pom.xml
@@ -6,9 +6,7 @@
 		<groupId>edu.uci.ics.asterix</groupId>
 		<version>0.0.4-SNAPSHOT</version>
 	</parent>
-	<groupId>edu.uci.ics.asterix</groupId>
 	<artifactId>asterix-app</artifactId>
-	<version>0.0.4-SNAPSHOT</version>
 
 	<build>
 		<plugins>
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 ea4fc67..862c701 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
@@ -43,6 +43,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.JobIdFactory;
 import edu.uci.ics.asterix.translator.AqlExpressionToPlanTranslator;
 import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -68,6 +69,7 @@
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.IJobletEventListenerFactory;
 import edu.uci.ics.hyracks.api.job.JobId;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 
@@ -108,6 +110,8 @@
                 RuleCollections.buildAccessMethodRuleCollection()));
         defaultLogicalRewrites.add(new Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>(seqCtrlNoDfs,
                 RuleCollections.buildPlanCleanupRuleCollection()));
+
+        //put TXnRuleCollection!
         return defaultLogicalRewrites;
     }
 
@@ -153,6 +157,7 @@
     public static Pair<Query, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
             AqlMetadataProvider metadataProvider, Query q, SessionConfig pc, PrintWriter out, DisplayFormat pdf)
             throws AsterixException {
+
         if (!pc.isPrintPhysicalOpsOnly() && pc.isPrintExprParam()) {
             out.println();
             switch (pdf) {
@@ -215,6 +220,9 @@
 
         }
 
+        edu.uci.ics.asterix.transaction.management.service.transaction.JobId asterixJobId = JobIdFactory
+                .generateJobId();
+        queryMetadataProvider.setJobId(asterixJobId);
         AqlExpressionToPlanTranslator t = new AqlExpressionToPlanTranslator(queryMetadataProvider, varCounter,
                 outputDatasetName, statement);
 
@@ -332,10 +340,10 @@
         builder.setTypeTraitProvider(format.getTypeTraitProvider());
         builder.setNormalizedKeyComputerFactoryProvider(format.getNormalizedKeyComputerFactoryProvider());
 
-        JobSpecification spec = compiler.createJob(AsterixAppContextInfoImpl.getInstance());
-        // set the job event listener
-        spec.setJobletEventListenerFactory(new JobEventListenerFactory(queryMetadataProvider.getJobTxnId(),
-                isWriteTransaction));
+        IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(asterixJobId,
+                isWriteTransaction);
+        JobSpecification spec = compiler.createJob(AsterixAppContextInfoImpl.getInstance(), jobEventListenerFactory);
+
         if (pc.isPrintJob()) {
             switch (pdf) {
                 case HTML: {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 40b1848..c8b0321 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -32,7 +32,6 @@
 import edu.uci.ics.asterix.api.common.Job;
 import edu.uci.ics.asterix.api.common.SessionConfig;
 import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.base.Statement.Kind;
 import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CreateDataverseStatement;
@@ -72,6 +71,7 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
@@ -88,7 +88,7 @@
 import edu.uci.ics.asterix.result.ResultReader;
 import edu.uci.ics.asterix.result.ResultUtils;
 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.DatasetIdFactory;
 import edu.uci.ics.asterix.translator.AbstractAqlTranslator;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledBeginFeedStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledControlFeedStatement;
@@ -172,8 +172,7 @@
 
         for (Statement stmt : aqlStatements) {
             validateOperation(activeDefaultDataverse, stmt);
-            MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-            AqlMetadataProvider metadataProvider = new AqlMetadataProvider(mdTxnCtx, activeDefaultDataverse);
+            AqlMetadataProvider metadataProvider = new AqlMetadataProvider(activeDefaultDataverse);
             metadataProvider.setWriterFactory(writerFactory);
             metadataProvider.setResultSerializerFactoryProvider(resultSerializerFactoryProvider);
             metadataProvider.setOutputFile(outputFile);
@@ -182,100 +181,99 @@
             try {
                 switch (stmt.getKind()) {
                     case SET: {
-                        handleSetStatement(metadataProvider, stmt, config, jobsToExecute);
+                        handleSetStatement(metadataProvider, stmt, config);
                         break;
                     }
                     case DATAVERSE_DECL: {
-                        activeDefaultDataverse = handleUseDataverseStatement(metadataProvider, stmt, jobsToExecute);
+                        activeDefaultDataverse = handleUseDataverseStatement(metadataProvider, stmt);
                         break;
                     }
                     case CREATE_DATAVERSE: {
-                        handleCreateDataverseStatement(metadataProvider, stmt, jobsToExecute);
+                        handleCreateDataverseStatement(metadataProvider, stmt);
                         break;
                     }
                     case DATASET_DECL: {
-                        handleCreateDatasetStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleCreateDatasetStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case CREATE_INDEX: {
-                        handleCreateIndexStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleCreateIndexStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case TYPE_DECL: {
-                        handleCreateTypeStatement(metadataProvider, stmt, jobsToExecute);
+                        handleCreateTypeStatement(metadataProvider, stmt);
                         break;
                     }
                     case NODEGROUP_DECL: {
-                        handleCreateNodeGroupStatement(metadataProvider, stmt, jobsToExecute);
+                        handleCreateNodeGroupStatement(metadataProvider, stmt);
                         break;
                     }
                     case DATAVERSE_DROP: {
-                        handleDataverseDropStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleDataverseDropStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case DATASET_DROP: {
-                        handleDatasetDropStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleDatasetDropStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case INDEX_DROP: {
-                        handleIndexDropStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleIndexDropStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case TYPE_DROP: {
-                        handleTypeDropStatement(metadataProvider, stmt, jobsToExecute);
+                        handleTypeDropStatement(metadataProvider, stmt);
                         break;
                     }
                     case NODEGROUP_DROP: {
-                        handleNodegroupDropStatement(metadataProvider, stmt, jobsToExecute);
+                        handleNodegroupDropStatement(metadataProvider, stmt);
                         break;
                     }
 
                     case CREATE_FUNCTION: {
-                        handleCreateFunctionStatement(metadataProvider, stmt, jobsToExecute);
+                        handleCreateFunctionStatement(metadataProvider, stmt);
                         break;
                     }
 
                     case FUNCTION_DROP: {
-                        handleFunctionDropStatement(metadataProvider, stmt, jobsToExecute);
+                        handleFunctionDropStatement(metadataProvider, stmt);
                         break;
                     }
 
                     case LOAD_FROM_FILE: {
-                        handleLoadFromFileStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleLoadFromFileStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case WRITE_FROM_QUERY_RESULT: {
-                        handleWriteFromQueryResultStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleWriteFromQueryResultStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case INSERT: {
-                        handleInsertStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleInsertStatement(metadataProvider, stmt, hcc);
                         break;
                     }
                     case DELETE: {
-                        handleDeleteStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleDeleteStatement(metadataProvider, stmt, hcc);
                         break;
                     }
 
                     case BEGIN_FEED: {
-                        handleBeginFeedStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleBeginFeedStatement(metadataProvider, stmt, hcc);
                         break;
                     }
 
                     case CONTROL_FEED: {
-                        handleControlFeedStatement(metadataProvider, stmt, hcc, jobsToExecute);
+                        handleControlFeedStatement(metadataProvider, stmt, hcc);
                         break;
                     }
 
                     case QUERY: {
                         metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
-                        executionResult.add(handleQuery(metadataProvider, (Query) stmt, hcc, jobsToExecute));
+                        executionResult.add(handleQuery(metadataProvider, (Query) stmt, hcc, hdc, asyncResults));
                         break;
                     }
 
                     case WRITE: {
-                        Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(metadataProvider, stmt,
-                                jobsToExecute);
+                        Pair<IAWriterFactory, FileSplit> result = handleWriteStatement(metadataProvider, stmt);
                         if (result.first != null) {
                             writerFactory = result.first;
                         }
@@ -284,65 +282,23 @@
                     }
 
                 }
-                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             } catch (Exception e) {
-                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
                 throw new AlgebricksException(e);
             }
-            // Following jobs are run under a separate transaction, that is
-            // committed/aborted by the JobEventListener
-            for (JobSpecification jobspec : jobsToExecute) {
-                JobId jobId = runJob(hcc, jobspec);
-                if (stmt.getKind() == Kind.QUERY) {
-                    JSONObject response = new JSONObject();
-
-                    if (asyncResults) {
-                        JSONArray handle = new JSONArray();
-                        handle.put(jobId.getId());
-                        handle.put(metadataProvider.getResultSetId().getId());
-                        response.put("handle", handle);
-                    } else {
-                        ByteBuffer buffer = ByteBuffer.allocate(ResultReader.FRAME_SIZE);
-                        ResultReader resultReader = new ResultReader(hcc, hdc);
-                        resultReader.open(jobId, metadataProvider.getResultSetId());
-                        buffer.clear();
-                        JSONArray results = new JSONArray();
-                        while (resultReader.read(buffer) > 0) {
-                            results.put(ResultUtils.getJSONFromBuffer(buffer, resultReader.getFrameTupleAccessor()));
-                            buffer.clear();
-                        }
-                        response.put("results", results);
-                    }
-                    switch (pdf) {
-                        case HTML:
-                            out.println("<h3>Result:</h3>");
-                            out.println("<pre>");
-                            ResultUtils.prettyPrintHTML(out, response);
-                            out.println("</pre>");
-                            break;
-                        case TEXT:
-                        case JSON:
-                            out.print(response);
-                            break;
-                    }
-                }
-                hcc.waitForCompletion(jobId);
-            }
         }
         return executionResult;
     }
 
-    private void handleSetStatement(AqlMetadataProvider metadataProvider, Statement stmt, Map<String, String> config,
-            List<JobSpecification> jobsToExecute) throws RemoteException, ACIDException {
+    private void handleSetStatement(AqlMetadataProvider metadataProvider, Statement stmt, Map<String, String> config)
+            throws RemoteException, ACIDException {
         SetStatement ss = (SetStatement) stmt;
         String pname = ss.getPropName();
         String pvalue = ss.getPropValue();
         config.put(pname, pvalue);
     }
 
-    private Pair<IAWriterFactory, FileSplit> handleWriteStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws InstantiationException, IllegalAccessException,
-            ClassNotFoundException {
+    private Pair<IAWriterFactory, FileSplit> handleWriteStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws InstantiationException, IllegalAccessException, ClassNotFoundException {
         WriteStatement ws = (WriteStatement) stmt;
         File f = new File(ws.getFileName());
         FileSplit outputFile = new FileSplit(ws.getNcName().getValue(), new FileReference(f));
@@ -353,403 +309,930 @@
         return new Pair<IAWriterFactory, FileSplit>(writerFactory, outputFile);
     }
 
-    private Dataverse handleUseDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws MetadataException, RemoteException, ACIDException {
-        DataverseDecl dvd = (DataverseDecl) stmt;
-        String dvName = dvd.getDataverseName().getValue();
-        Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
-        if (dv == null) {
-            throw new MetadataException("Unknown dataverse " + dvName);
+    private Dataverse handleUseDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws MetadataException, RemoteException, ACIDException {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
+
+        try {
+            DataverseDecl dvd = (DataverseDecl) stmt;
+            String dvName = dvd.getDataverseName().getValue();
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
+            if (dv == null) {
+                throw new MetadataException("Unknown dataverse " + dvName);
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            return dv;
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new MetadataException(e);
+        } finally {
+            releaseReadLatch();
         }
-        return dv;
     }
 
-    private void handleCreateDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws MetadataException, AlgebricksException, RemoteException,
-            ACIDException {
-        CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
-        String dvName = stmtCreateDataverse.getDataverseName().getValue();
-        Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
-        if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
-            throw new AlgebricksException("A dataverse with this name " + dvName + " already exists.");
+    private void handleCreateDataverseStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws MetadataException, AlgebricksException, RemoteException, ACIDException {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            CreateDataverseStatement stmtCreateDataverse = (CreateDataverseStatement) stmt;
+            String dvName = stmtCreateDataverse.getDataverseName().getValue();
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(), dvName);
+            if (dv != null && !stmtCreateDataverse.getIfNotExists()) {
+                throw new AlgebricksException("A dataverse with this name " + dvName + " already exists.");
+            }
+            MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(), new Dataverse(dvName,
+                    stmtCreateDataverse.getFormat(), IMetadataEntity.PENDING_NO_OP));
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
-        MetadataManager.INSTANCE.addDataverse(metadataProvider.getMetadataTxnContext(), new Dataverse(dvName,
-                stmtCreateDataverse.getFormat()));
     }
 
     private void handleCreateDatasetStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws AsterixException, Exception {
-        DatasetDecl dd = (DatasetDecl) stmt;
-        String dataverseName = dd.getDataverse() != null ? dd.getDataverse().getValue()
-                : activeDefaultDataverse != null ? activeDefaultDataverse.getDataverseName() : null;
-        if (dataverseName == null) {
-            throw new AlgebricksException(" dataverse not specified ");
-        }
-        String datasetName = dd.getName().getValue();
-        DatasetType dsType = dd.getDatasetType();
-        String itemTypeName = dd.getItemTypeName().getValue();
+            IHyracksClientConnection hcc) throws AsterixException, Exception {
 
-        IDatasetDetails datasetDetails = null;
-        Dataset ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
-                datasetName);
-        if (ds != null) {
-            if (dd.getIfNotExists()) {
-                return;
-            } else {
-                throw new AlgebricksException("A dataset with this name " + datasetName + " already exists.");
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        String dataverseName = null;
+        String datasetName = null;
+        try {
+            DatasetDecl dd = (DatasetDecl) stmt;
+            dataverseName = dd.getDataverse() != null ? dd.getDataverse().getValue()
+                    : activeDefaultDataverse != null ? activeDefaultDataverse.getDataverseName() : null;
+            if (dataverseName == null) {
+                throw new AlgebricksException(" dataverse not specified ");
             }
-        }
-        Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(), dataverseName,
-                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.");
+            datasetName = dd.getName().getValue();
+
+            DatasetType dsType = dd.getDatasetType();
+            String itemTypeName = dd.getItemTypeName().getValue();
+
+            IDatasetDetails datasetDetails = null;
+            Dataset ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    datasetName);
+            if (ds != null) {
+                if (dd.getIfNotExists()) {
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return;
+                } else {
+                    throw new AlgebricksException("A dataset with this name " + datasetName + " already exists.");
                 }
-                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;
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    itemTypeName);
+            if (dt == null) {
+                throw new AlgebricksException(": type " + itemTypeName + " could not be found.");
             }
-            case FEED: {
-                IAType itemType = dt.getDatatype();
-                if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                    throw new AlgebricksException("Can only partition ARecord's.");
+            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;
                 }
-                List<String> partitioningExprs = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getPartitioningExprs();
-                String ngName = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
-                String adapter = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getAdapterFactoryClassname();
-                Map<String, String> configuration = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getConfiguration();
-                FunctionSignature signature = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getFunctionSignature();
-                datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
-                        InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs, ngName,
-                        adapter, configuration, signature, FeedDatasetDetails.FeedState.INACTIVE.toString());
-                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()).getAdapterFactoryClassname();
+                    Map<String, String> configuration = ((FeedDetailsDecl) dd.getDatasetDetailsDecl())
+                            .getConfiguration();
+                    FunctionSignature signature = ((FeedDetailsDecl) dd.getDatasetDetailsDecl()).getFunctionSignature();
+                    datasetDetails = new FeedDatasetDetails(InternalDatasetDetails.FileStructure.BTREE,
+                            InternalDatasetDetails.PartitioningStrategy.HASH, partitioningExprs, partitioningExprs,
+                            ngName, adapter, configuration, signature, FeedDatasetDetails.FeedState.INACTIVE.toString());
+                    break;
+                }
             }
-        }
-        MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), new Dataset(dataverseName,
-                datasetName, itemTypeName, datasetDetails, dd.getHints(), dsType));
-        if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
-            Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
-                    dataverseName);
-            JobId jobId = runJob(hcc, DatasetOperations.createDatasetJobSpec(dataverse, datasetName, metadataProvider));
-            hcc.waitForCompletion(jobId);
+
+            //#. add a new dataset with PendingAddOp
+            Dataset dataset = new Dataset(dataverseName, datasetName, itemTypeName, datasetDetails, dd.getHints(),
+                    dsType, DatasetIdFactory.generateDatasetId(), IMetadataEntity.PENDING_ADD_OP);
+            MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), dataset);
+
+            if (dd.getDatasetType() == DatasetType.INTERNAL || dd.getDatasetType() == DatasetType.FEED) {
+                Dataverse dataverse = MetadataManager.INSTANCE.getDataverse(metadataProvider.getMetadataTxnContext(),
+                        dataverseName);
+                JobSpecification jobSpec = DatasetOperations.createDatasetJobSpec(dataverse, datasetName,
+                        metadataProvider);
+
+                //#. make metadataTxn commit before calling runJob.
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                bActiveTxn = false;
+
+                //#. runJob
+                runJob(hcc, jobSpec, true);
+
+                //#. begin new metadataTxn
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            }
+
+            //#. add a new dataset with PendingNoOp after deleting the dataset with PendingAddOp
+            MetadataManager.INSTANCE.dropDataset(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName);
+            MetadataManager.INSTANCE.addDataset(metadataProvider.getMetadataTxnContext(), new Dataset(dataverseName,
+                    datasetName, itemTypeName, datasetDetails, dd.getHints(), dsType, dataset.getDatasetId(),
+                    IMetadataEntity.PENDING_NO_OP));
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+
+            //#. execute compensation operations
+            //   remove the index in NC
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
+            try {
+                JobSpecification jobSpec = DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                bActiveTxn = false;
+
+                runJob(hcc, jobSpec, true);
+            } catch (Exception e3) {
+                if (bActiveTxn) {
+                    MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                }
+                //do no throw exception since still the metadata needs to be compensated. 
+            }
+
+            //   remove the record from the metadata.
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            try {
+                MetadataManager.INSTANCE.dropDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
+                        datasetName);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            } catch (Exception e2) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                throw new AlgebricksException(e2);
+            }
+
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
     private void handleCreateIndexStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
-        String dataverseName = stmtCreateIndex.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : stmtCreateIndex.getDataverseName().getValue();
-        if (dataverseName == null) {
-            throw new AlgebricksException(" dataverse not specified ");
-        }
-        String datasetName = stmtCreateIndex.getDatasetName().getValue();
-        Dataset ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
-                datasetName);
-        if (ds == null) {
-            throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
-                    + dataverseName);
-        }
-        String indexName = stmtCreateIndex.getIndexName().getValue();
-        Index idx = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
-                datasetName, indexName);
-        if (idx != null) {
-            if (!stmtCreateIndex.getIfNotExists()) {
-                throw new AlgebricksException("An index with this name " + indexName + " already exists.");
-            } else {
-                stmtCreateIndex.setNeedToCreate(false);
-            }
-        } else {
-            Index index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
-                    stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), false);
-            MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
-            runCreateIndexJob(hcc, stmtCreateIndex, metadataProvider);
+            IHyracksClientConnection hcc) throws Exception {
 
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        String dataverseName = null;
+        String datasetName = null;
+        String indexName = null;
+        try {
+            CreateIndexStatement stmtCreateIndex = (CreateIndexStatement) stmt;
+            dataverseName = stmtCreateIndex.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : stmtCreateIndex.getDataverseName().getValue();
+            if (dataverseName == null) {
+                throw new AlgebricksException(" dataverse not specified ");
+            }
+            datasetName = stmtCreateIndex.getDatasetName().getValue();
+
+            Dataset ds = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    datasetName);
+            if (ds == null) {
+                throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
+                        + dataverseName);
+            }
+
+            indexName = stmtCreateIndex.getIndexName().getValue();
+            Index idx = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                    datasetName, indexName);
+
+            if (idx != null) {
+                if (!stmtCreateIndex.getIfNotExists()) {
+                    throw new AlgebricksException("An index with this name " + indexName + " already exists.");
+                } else {
+                    stmtCreateIndex.setNeedToCreate(false);
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    return;
+                }
+            }
+
+            //#. add a new index with PendingAddOp
+            Index index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
+                    stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), false,
+                    IMetadataEntity.PENDING_ADD_OP);
+            MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
+
+            //#. create the index artifact in NC.
             CompiledCreateIndexStatement cis = new CompiledCreateIndexStatement(index.getIndexName(), dataverseName,
                     index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(), index.getIndexType());
-            JobSpecification loadIndexJobSpec = IndexOperations
-                    .buildSecondaryIndexLoadingJobSpec(cis, metadataProvider);
-            JobId jobId = runJob(hcc, loadIndexJobSpec);
-            hcc.waitForCompletion(jobId);
+            JobSpecification spec = IndexOperations.buildSecondaryIndexCreationJobSpec(cis, metadataProvider);
+            if (spec == null) {
+                throw new AsterixException("Failed to create job spec for creating index '"
+                        + stmtCreateIndex.getDatasetName() + "." + stmtCreateIndex.getIndexName() + "'");
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            runJob(hcc, spec, true);
+
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+            //#. load data into the index in NC.
+            cis = new CompiledCreateIndexStatement(index.getIndexName(), dataverseName, index.getDatasetName(),
+                    index.getKeyFieldNames(), index.getGramLength(), index.getIndexType());
+            spec = IndexOperations.buildSecondaryIndexLoadingJobSpec(cis, metadataProvider);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            runJob(hcc, spec, true);
+
+            //#. begin new metadataTxn
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+            //#. add another new index with PendingNoOp after deleting the index with PendingAddOp
+            MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName,
+                    indexName);
+            index = new Index(dataverseName, datasetName, indexName, stmtCreateIndex.getIndexType(),
+                    stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), false,
+                    IMetadataEntity.PENDING_NO_OP);
+            MetadataManager.INSTANCE.addIndex(metadataProvider.getMetadataTxnContext(), index);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+
+            //#. execute compensation operations
+            //   remove the index in NC
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
+            try {
+                JobSpecification jobSpec = IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                bActiveTxn = false;
+
+                runJob(hcc, jobSpec, true);
+            } catch (Exception e3) {
+                if (bActiveTxn) {
+                    MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                }
+                //do no throw exception since still the metadata needs to be compensated. 
+            }
+
+            //   remove the record from the metadata.
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            try {
+                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                        datasetName, indexName);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            } catch (Exception e2) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                throw new AlgebricksException(e2);
+            }
+
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
-    private void handleCreateTypeStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws AlgebricksException, RemoteException, ACIDException,
-            MetadataException {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        TypeDecl stmtCreateType = (TypeDecl) stmt;
-        String dataverseName = stmtCreateType.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : stmtCreateType.getDataverseName().getValue();
-        if (dataverseName == null) {
-            throw new AlgebricksException(" dataverse not specified ");
-        }
-        String typeName = stmtCreateType.getIdent().getValue();
-        Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
-        if (dv == null) {
-            throw new AlgebricksException("Unknonw dataverse " + dataverseName);
-        }
-        Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, 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<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx, (TypeDecl) stmt,
-                        dataverseName);
-                TypeSignature typeSignature = new TypeSignature(dataverseName, typeName);
-                IAType type = typeMap.get(typeSignature);
-                MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(dataverseName, typeName, type, false));
+    private void handleCreateTypeStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws AlgebricksException, RemoteException, ACIDException, MetadataException {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            TypeDecl stmtCreateType = (TypeDecl) stmt;
+            String dataverseName = stmtCreateType.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : stmtCreateType.getDataverseName().getValue();
+            if (dataverseName == null) {
+                throw new AlgebricksException(" dataverse not specified ");
             }
+            String typeName = stmtCreateType.getIdent().getValue();
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverseName);
+            if (dv == null) {
+                throw new AlgebricksException("Unknonw dataverse " + dataverseName);
+            }
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, 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<TypeSignature, IAType> typeMap = TypeTranslator.computeTypes(mdTxnCtx, (TypeDecl) stmt,
+                            dataverseName);
+                    TypeSignature typeSignature = new TypeSignature(dataverseName, typeName);
+                    IAType type = typeMap.get(typeSignature);
+                    MetadataManager.INSTANCE.addDatatype(mdTxnCtx, new Datatype(dataverseName, typeName, type, false));
+                }
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
     private void handleDataverseDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
-        String dvName = stmtDelete.getDataverseName().getValue();
+            IHyracksClientConnection hcc) throws Exception {
 
-        Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
-        if (dv == null) {
-            if (!stmtDelete.getIfExists()) {
-                throw new AlgebricksException("There is no dataverse with this name " + dvName + ".");
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        String dvName = null;
+        List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+        try {
+            DataverseDropStatement stmtDelete = (DataverseDropStatement) stmt;
+            dvName = stmtDelete.getDataverseName().getValue();
+
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dvName);
+            if (dv == null) {
+                if (!stmtDelete.getIfExists()) {
+                    throw new AlgebricksException("There is no dataverse with this name " + dvName + ".");
+                }
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return;
             }
-        } else {
+
+            //#. prepare jobs which will drop corresponding datasets with indexes. 
             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, dvName, datasetName, indexes.get(k).getIndexName(),
-                                    metadataProvider);
+                            CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dvName, datasetName,
+                                    indexes.get(k).getIndexName());
+                            jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
                         }
                     }
+
+                    CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dvName, datasetName);
+                    jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
                 }
-                compileDatasetDropStatement(hcc, dvName, datasetName, metadataProvider);
             }
 
+            //#. mark PendingDropOp on the dataverse record by 
+            //   first, deleting the dataverse record from the DATAVERSE_DATASET
+            //   second, inserting the dataverse record with the PendingDropOp value into the DATAVERSE_DATASET
+            MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
+            MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(dvName, dv.getDataFormat(),
+                    IMetadataEntity.PENDING_DROP_OP));
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            for (JobSpecification jobSpec : jobsToExecute) {
+                runJob(hcc, jobSpec, true);
+            }
+
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            bActiveTxn = true;
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+            //#. finally, delete the dataverse.
             MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dvName);
             if (activeDefaultDataverse != null && activeDefaultDataverse.getDataverseName() == dvName) {
                 activeDefaultDataverse = null;
             }
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+
+            //#. execute compensation operations
+            //   remove the all indexes in NC
+            for (JobSpecification jobSpec : jobsToExecute) {
+                runJob(hcc, jobSpec, true);
+            }
+
+            //   remove the record from the metadata.
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            try {
+                MetadataManager.INSTANCE.dropDataverse(metadataProvider.getMetadataTxnContext(), dvName);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            } catch (Exception e2) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                throw new AlgebricksException(e2);
+            }
+
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
     private void handleDatasetDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        DropStatement stmtDelete = (DropStatement) stmt;
-        String dataverseName = stmtDelete.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : stmtDelete.getDataverseName().getValue();
-        if (dataverseName == null) {
-            throw new AlgebricksException(" dataverse not specified ");
-        }
-        String datasetName = stmtDelete.getDatasetName().getValue();
-        Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
-        if (ds == null) {
-            if (!stmtDelete.getIfExists())
-                throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
-                        + dataverseName + ".");
-        } else {
+            IHyracksClientConnection hcc) throws Exception {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        String dataverseName = null;
+        String datasetName = null;
+        List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+        try {
+            DropStatement stmtDelete = (DropStatement) stmt;
+            dataverseName = stmtDelete.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : stmtDelete.getDataverseName().getValue();
+            if (dataverseName == null) {
+                throw new AlgebricksException(" dataverse not specified ");
+            }
+            datasetName = stmtDelete.getDatasetName().getValue();
+
+            Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
+            if (ds == null) {
+                if (!stmtDelete.getIfExists()) {
+                    throw new AlgebricksException("There is no dataset with this name " + datasetName
+                            + " in dataverse " + dataverseName + ".");
+                }
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                return;
+            }
+
             if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+
+                //#. prepare jobs to drop the datatset and the indexes in NC
                 List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
                 for (int j = 0; j < indexes.size(); j++) {
-                    if (indexes.get(j).isPrimaryIndex()) {
-                        compileIndexDropStatement(hcc, dataverseName, datasetName, indexes.get(j).getIndexName(),
-                                metadataProvider);
+                    if (indexes.get(j).isSecondaryIndex()) {
+                        CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+                                indexes.get(j).getIndexName());
+                        jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
                     }
                 }
+                CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
+                jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
+
+                //#. mark the existing dataset as PendingDropOp
+                MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
+                MetadataManager.INSTANCE.addDataset(
+                        mdTxnCtx,
+                        new Dataset(dataverseName, datasetName, ds.getItemTypeName(), ds.getDatasetDetails(), ds
+                                .getHints(), ds.getDatasetType(), ds.getDatasetId(), IMetadataEntity.PENDING_DROP_OP));
+
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                bActiveTxn = false;
+
+                //#. run the jobs
+                for (JobSpecification jobSpec : jobsToExecute) {
+                    runJob(hcc, jobSpec, true);
+                }
+
+                mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
             }
-            compileDatasetDropStatement(hcc, dataverseName, datasetName, metadataProvider);
+
+            //#. finally, delete the dataset.
+            MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+
+            //#. execute compensation operations
+            //   remove the all indexes in NC
+            for (JobSpecification jobSpec : jobsToExecute) {
+                runJob(hcc, jobSpec, true);
+            }
+
+            //   remove the record from the metadata.
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            try {
+                MetadataManager.INSTANCE.dropDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
+                        datasetName);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            } catch (Exception e2) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                throw new AlgebricksException(e2);
+            }
+
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
     private void handleIndexDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
-        String datasetName = stmtIndexDrop.getDatasetName().getValue();
-        String dataverseName = stmtIndexDrop.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : stmtIndexDrop.getDataverseName().getValue();
-        if (dataverseName == null) {
-            throw new AlgebricksException(" dataverse not specified ");
-        }
-        Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
-        if (ds == null)
-            throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
-                    + dataverseName);
-        if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
-            String indexName = stmtIndexDrop.getIndexName().getValue();
-            Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
-            if (idx == null) {
-                if (!stmtIndexDrop.getIfExists())
-                    throw new AlgebricksException("There is no index with this name " + indexName + ".");
-            } else
-                compileIndexDropStatement(hcc, dataverseName, datasetName, indexName, metadataProvider);
-        } else {
-            throw new AlgebricksException(datasetName
-                    + " is an external dataset. Indexes are not maintained for external datasets.");
+            IHyracksClientConnection hcc) throws Exception {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        String dataverseName = null;
+        String datasetName = null;
+        String indexName = null;
+        List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+        try {
+            IndexDropStatement stmtIndexDrop = (IndexDropStatement) stmt;
+            datasetName = stmtIndexDrop.getDatasetName().getValue();
+            dataverseName = stmtIndexDrop.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : stmtIndexDrop.getDataverseName().getValue();
+            if (dataverseName == null) {
+                throw new AlgebricksException(" dataverse not specified ");
+            }
+
+            Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
+            if (ds == null) {
+                throw new AlgebricksException("There is no dataset with this name " + datasetName + " in dataverse "
+                        + dataverseName);
+            }
+
+            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+                indexName = stmtIndexDrop.getIndexName().getValue();
+                Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+                if (index == null) {
+                    if (!stmtIndexDrop.getIfExists()) {
+                        throw new AlgebricksException("There is no index with this name " + indexName + ".");
+                    }
+                } else {
+                    //#. prepare a job to drop the index in NC.
+                    CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName,
+                            indexName);
+                    jobsToExecute.add(IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
+
+                    //#. mark PendingDropOp on the existing index
+                    MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+                    MetadataManager.INSTANCE.addIndex(
+                            mdTxnCtx,
+                            new Index(dataverseName, datasetName, indexName, index.getIndexType(), index
+                                    .getKeyFieldNames(), index.isPrimaryIndex(), IMetadataEntity.PENDING_DROP_OP));
+
+                    //#. commit the existing transaction before calling runJob. 
+                    MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                    bActiveTxn = false;
+
+                    for (JobSpecification jobSpec : jobsToExecute) {
+                        runJob(hcc, jobSpec, true);
+                    }
+
+                    //#. begin a new transaction
+                    mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+                    bActiveTxn = true;
+                    metadataProvider.setMetadataTxnContext(mdTxnCtx);
+
+                    //#. finally, delete the existing index
+                    MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+                }
+            } else {
+                throw new AlgebricksException(datasetName
+                        + " is an external dataset. Indexes are not maintained for external datasets.");
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+
+            //#. execute compensation operations
+            //   remove the all indexes in NC
+            for (JobSpecification jobSpec : jobsToExecute) {
+                runJob(hcc, jobSpec, true);
+            }
+
+            //   remove the record from the metadata.
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            metadataProvider.setMetadataTxnContext(mdTxnCtx);
+            try {
+                MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName,
+                        datasetName, indexName);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            } catch (Exception e2) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+                throw new AlgebricksException(e2);
+            }
+
+            throw new AlgebricksException(e);
+
+        } finally {
+            releaseWriteLatch();
         }
     }
 
-    private void handleTypeDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws AlgebricksException, MetadataException, RemoteException,
-            ACIDException {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
-        String dataverseName = stmtTypeDrop.getDataverseName() == null ? (activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName()) : stmtTypeDrop.getDataverseName().getValue();
-        if (dataverseName == null) {
-            throw new AlgebricksException(" dataverse not specified ");
-        }
-        String typeName = stmtTypeDrop.getTypeName().getValue();
-        Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
-        if (dt == null) {
-            if (!stmtTypeDrop.getIfExists())
-                throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
-        } else {
-            MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, dataverseName, typeName);
+    private void handleTypeDropStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws AlgebricksException, MetadataException, RemoteException, ACIDException {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            TypeDropStatement stmtTypeDrop = (TypeDropStatement) stmt;
+            String dataverseName = stmtTypeDrop.getDataverseName() == null ? (activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName()) : stmtTypeDrop.getDataverseName().getValue();
+            if (dataverseName == null) {
+                throw new AlgebricksException(" dataverse not specified ");
+            }
+            String typeName = stmtTypeDrop.getTypeName().getValue();
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, typeName);
+            if (dt == null) {
+                if (!stmtTypeDrop.getIfExists())
+                    throw new AlgebricksException("There is no datatype with this name " + typeName + ".");
+            } else {
+                MetadataManager.INSTANCE.dropDatatype(mdTxnCtx, dataverseName, typeName);
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
-    private void handleNodegroupDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws MetadataException, AlgebricksException, RemoteException,
-            ACIDException {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
-        String nodegroupName = stmtDelete.getNodeGroupName().getValue();
-        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);
+    private void handleNodegroupDropStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws MetadataException, AlgebricksException, RemoteException, ACIDException {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            NodeGroupDropStatement stmtDelete = (NodeGroupDropStatement) stmt;
+            String nodegroupName = stmtDelete.getNodeGroupName().getValue();
+            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);
+            }
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
-    private void handleCreateFunctionStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws AlgebricksException, MetadataException, RemoteException,
-            ACIDException {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
-        String dataverse = cfs.getSignature().getNamespace() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : cfs.getSignature().getNamespace();
-        if (dataverse == null) {
-            throw new AlgebricksException(" dataverse not specified ");
+    private void handleCreateFunctionStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws AlgebricksException, MetadataException, RemoteException, ACIDException {
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            CreateFunctionStatement cfs = (CreateFunctionStatement) stmt;
+            String dataverse = cfs.getSignature().getNamespace() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : cfs.getSignature().getNamespace();
+            if (dataverse == null) {
+                throw new AlgebricksException(" dataverse not specified ");
+            }
+            Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
+            if (dv == null) {
+                throw new AlgebricksException("There is no dataverse with this name " + dataverse + ".");
+            }
+            Function function = new Function(dataverse, cfs.getaAterixFunction().getName(), cfs.getaAterixFunction()
+                    .getArity(), cfs.getParamList(), Function.RETURNTYPE_VOID, cfs.getFunctionBody(),
+                    Function.LANGUAGE_AQL, FunctionKind.SCALAR.toString());
+            MetadataManager.INSTANCE.addFunction(mdTxnCtx, function);
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
-        Dataverse dv = MetadataManager.INSTANCE.getDataverse(mdTxnCtx, dataverse);
-        if (dv == null) {
-            throw new AlgebricksException("There is no dataverse with this name " + dataverse + ".");
-        }
-        Function function = new Function(dataverse, cfs.getaAterixFunction().getName(), cfs.getaAterixFunction()
-                .getArity(), cfs.getParamList(), Function.RETURNTYPE_VOID, cfs.getFunctionBody(),
-                Function.LANGUAGE_AQL, FunctionKind.SCALAR.toString());
-        MetadataManager.INSTANCE.addFunction(mdTxnCtx, function);
     }
 
-    private void handleFunctionDropStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws MetadataException, RemoteException, ACIDException,
-            AlgebricksException {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
-        FunctionSignature signature = stmtDropFunction.getFunctionSignature();
-        Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
-        if (function == null) {
-            if (!stmtDropFunction.getIfExists())
-                throw new AlgebricksException("Unknonw function " + signature);
-        } else {
-            MetadataManager.INSTANCE.dropFunction(mdTxnCtx, signature);
+    private void handleFunctionDropStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws MetadataException, RemoteException, ACIDException, AlgebricksException {
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            FunctionDropStatement stmtDropFunction = (FunctionDropStatement) stmt;
+            FunctionSignature signature = stmtDropFunction.getFunctionSignature();
+            Function function = MetadataManager.INSTANCE.getFunction(mdTxnCtx, signature);
+            if (function == null) {
+                if (!stmtDropFunction.getIfExists())
+                    throw new AlgebricksException("Unknonw function " + signature);
+            } else {
+                MetadataManager.INSTANCE.dropFunction(mdTxnCtx, signature);
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
         }
     }
 
     private void handleLoadFromFileStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        LoadFromFileStatement loadStmt = (LoadFromFileStatement) stmt;
-        String dataverseName = loadStmt.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : loadStmt.getDataverseName().getValue();
-        CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName()
-                .getValue(), loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
+            IHyracksClientConnection hcc) throws Exception {
 
-        IDataFormat format = getDataFormat(metadataProvider.getMetadataTxnContext(), dataverseName);
-        Job job = DatasetOperations.createLoadDatasetJobSpec(metadataProvider, cls, format);
-        jobsToExecute.add(job.getJobSpec());
-        // Also load the dataset's secondary indexes.
-        List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, loadStmt
-                .getDatasetName().getValue());
-        for (Index index : datasetIndexes) {
-            if (!index.isSecondaryIndex()) {
-                continue;
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
+        List<JobSpecification> jobsToExecute = new ArrayList<JobSpecification>();
+        try {
+            LoadFromFileStatement loadStmt = (LoadFromFileStatement) stmt;
+            String dataverseName = loadStmt.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : loadStmt.getDataverseName().getValue();
+            CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName, loadStmt
+                    .getDatasetName().getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
+                    loadStmt.dataIsAlreadySorted());
+
+            IDataFormat format = getDataFormat(metadataProvider.getMetadataTxnContext(), dataverseName);
+            Job job = DatasetOperations.createLoadDatasetJobSpec(metadataProvider, cls, format);
+            jobsToExecute.add(job.getJobSpec());
+            // Also load the dataset's secondary indexes.
+            List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, loadStmt
+                    .getDatasetName().getValue());
+            for (Index index : datasetIndexes) {
+                if (!index.isSecondaryIndex()) {
+                    continue;
+                }
+                // Create CompiledCreateIndexStatement from metadata entity 'index'.
+                CompiledCreateIndexStatement cis = new CompiledCreateIndexStatement(index.getIndexName(),
+                        dataverseName, index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(),
+                        index.getIndexType());
+                jobsToExecute.add(IndexOperations.buildSecondaryIndexLoadingJobSpec(cis, metadataProvider));
             }
-            // Create CompiledCreateIndexStatement from metadata entity 'index'.
-            CompiledCreateIndexStatement cis = new CompiledCreateIndexStatement(index.getIndexName(), dataverseName,
-                    index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(), index.getIndexType());
-            jobsToExecute.add(IndexOperations.buildSecondaryIndexLoadingJobSpec(cis, metadataProvider));
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            for (JobSpecification jobspec : jobsToExecute) {
+                runJob(hcc, jobspec, true);
+            }
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+
+            throw new AlgebricksException(e);
+        } finally {
+            releaseReadLatch();
         }
     }
 
     private void handleWriteFromQueryResultStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        metadataProvider.setWriteTransaction(true);
-        WriteFromQueryResultStatement st1 = (WriteFromQueryResultStatement) stmt;
-        String dataverseName = st1.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : st1.getDataverseName().getValue();
-        CompiledWriteFromQueryResultStatement clfrqs = new CompiledWriteFromQueryResultStatement(dataverseName, st1
-                .getDatasetName().getValue(), st1.getQuery(), st1.getVarCounter());
+            IHyracksClientConnection hcc) throws Exception {
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
 
-        JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
-        if (compiled != null) {
-            jobsToExecute.add(compiled);
+        try {
+            metadataProvider.setWriteTransaction(true);
+            WriteFromQueryResultStatement st1 = (WriteFromQueryResultStatement) stmt;
+            String dataverseName = st1.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : st1.getDataverseName().getValue();
+            CompiledWriteFromQueryResultStatement clfrqs = new CompiledWriteFromQueryResultStatement(dataverseName, st1
+                    .getDatasetName().getValue(), st1.getQuery(), st1.getVarCounter());
+
+            JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+            if (compiled != null) {
+                runJob(hcc, compiled, true);
+            }
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+            throw new AlgebricksException(e);
+        } finally {
+            releaseReadLatch();
         }
     }
 
     private void handleInsertStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        metadataProvider.setWriteTransaction(true);
-        InsertStatement stmtInsert = (InsertStatement) stmt;
-        String dataverseName = stmtInsert.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : stmtInsert.getDataverseName().getValue();
-        CompiledInsertStatement clfrqs = new CompiledInsertStatement(dataverseName, stmtInsert.getDatasetName()
-                .getValue(), stmtInsert.getQuery(), stmtInsert.getVarCounter());
+            IHyracksClientConnection hcc) throws Exception {
 
-        JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
-        if (compiled != null) {
-            jobsToExecute.add(compiled);
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
+
+        try {
+            metadataProvider.setWriteTransaction(true);
+            InsertStatement stmtInsert = (InsertStatement) stmt;
+            String dataverseName = stmtInsert.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : stmtInsert.getDataverseName().getValue();
+            CompiledInsertStatement clfrqs = new CompiledInsertStatement(dataverseName, stmtInsert.getDatasetName()
+                    .getValue(), stmtInsert.getQuery(), stmtInsert.getVarCounter());
+            JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            if (compiled != null) {
+                runJob(hcc, compiled, true);
+            }
+
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+            throw new AlgebricksException(e);
+        } finally {
+            releaseReadLatch();
         }
     }
 
     private void handleDeleteStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        metadataProvider.setWriteTransaction(true);
-        DeleteStatement stmtDelete = (DeleteStatement) stmt;
-        String dataverseName = stmtDelete.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : stmtDelete.getDataverseName().getValue();
-        CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
-                stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getDieClause(),
-                stmtDelete.getVarCounter(), metadataProvider);
-        JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
-        if (compiled != null) {
-            jobsToExecute.add(compiled);
+            IHyracksClientConnection hcc) throws Exception {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
+
+        try {
+            metadataProvider.setWriteTransaction(true);
+            DeleteStatement stmtDelete = (DeleteStatement) stmt;
+            String dataverseName = stmtDelete.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : stmtDelete.getDataverseName().getValue();
+            CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
+                    stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getDieClause(),
+                    stmtDelete.getVarCounter(), metadataProvider);
+            JobSpecification compiled = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            if (compiled != null) {
+                runJob(hcc, compiled, true);
+            }
+
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+            throw new AlgebricksException(e);
+        } finally {
+            releaseReadLatch();
         }
     }
 
@@ -763,9 +1246,6 @@
 
         // Query Compilation (happens under the same ongoing metadata
         // transaction)
-        if (metadataProvider.isWriteTransaction()) {
-            metadataProvider.setJobTxnId(TransactionIDFactory.generateTransactionId());
-        }
         JobSpecification spec = APIFramework.compileQuery(declaredFunctions, metadataProvider, query,
                 reWrittenQuery.second, stmt == null ? null : stmt.getDatasetName(), sessionConfig, out, pdf, stmt);
 
@@ -774,130 +1254,194 @@
     }
 
     private void handleBeginFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        BeginFeedStatement bfs = (BeginFeedStatement) stmt;
-        String dataverseName = bfs.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : bfs.getDataverseName().getValue();
+            IHyracksClientConnection hcc) throws Exception {
 
-        CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(dataverseName,
-                bfs.getDatasetName().getValue(), bfs.getQuery(), bfs.getVarCounter());
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
 
-        Dataset dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName,
-                bfs.getDatasetName().getValue());
-        if (dataset == null) {
-            throw new AsterixException("Unknown dataset :" + bfs.getDatasetName().getValue());
-        }
-        IDatasetDetails datasetDetails = dataset.getDatasetDetails();
-        if (datasetDetails.getDatasetType() != DatasetType.FEED) {
-            throw new IllegalArgumentException("Dataset " + bfs.getDatasetName().getValue() + " is not a feed dataset");
-        }
-        bfs.initialize(metadataProvider.getMetadataTxnContext(), dataset);
-        cbfs.setQuery(bfs.getQuery());
+        try {
+            BeginFeedStatement bfs = (BeginFeedStatement) stmt;
+            String dataverseName = bfs.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : bfs.getDataverseName().getValue();
 
-        JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
-        if (compiled != null) {
-            jobsToExecute.add(compiled);
+            CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(dataverseName, bfs.getDatasetName()
+                    .getValue(), bfs.getQuery(), bfs.getVarCounter());
+
+            Dataset dataset;
+            dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(), dataverseName, bfs
+                    .getDatasetName().getValue());
+            if (dataset == null) {
+                throw new AsterixException("Unknown dataset :" + bfs.getDatasetName().getValue());
+            }
+            IDatasetDetails datasetDetails = dataset.getDatasetDetails();
+            if (datasetDetails.getDatasetType() != DatasetType.FEED) {
+                throw new IllegalArgumentException("Dataset " + bfs.getDatasetName().getValue()
+                        + " is not a feed dataset");
+            }
+            bfs.initialize(metadataProvider.getMetadataTxnContext(), dataset);
+            cbfs.setQuery(bfs.getQuery());
+            JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            if (compiled != null) {
+                runJob(hcc, compiled, true);
+            }
+
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+            throw new AlgebricksException(e);
+        } finally {
+            releaseReadLatch();
         }
     }
 
     private void handleControlFeedStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            IHyracksClientConnection hcc, List<JobSpecification> jobsToExecute) throws Exception {
-        ControlFeedStatement cfs = (ControlFeedStatement) stmt;
-        String dataverseName = cfs.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : cfs.getDataverseName().getValue();
-        CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(), dataverseName,
-                cfs.getDatasetName().getValue(), cfs.getAlterAdapterConfParams());
-        jobsToExecute.add(FeedOperations.buildControlFeedJobSpec(clcfs, metadataProvider));
+            IHyracksClientConnection hcc) throws Exception {
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
+
+        try {
+            ControlFeedStatement cfs = (ControlFeedStatement) stmt;
+            String dataverseName = cfs.getDataverseName() == null ? activeDefaultDataverse == null ? null
+                    : activeDefaultDataverse.getDataverseName() : cfs.getDatasetName().getValue();
+            CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(),
+                    dataverseName, cfs.getDatasetName().getValue(), cfs.getAlterAdapterConfParams());
+            JobSpecification jobSpec = FeedOperations.buildControlFeedJobSpec(clcfs, metadataProvider);
+
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            runJob(hcc, jobSpec, true);
+
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+            throw new AlgebricksException(e);
+        } finally {
+            releaseReadLatch();
+        }
     }
 
     private QueryResult handleQuery(AqlMetadataProvider metadataProvider, Query query, IHyracksClientConnection hcc,
-            List<JobSpecification> jobsToExecute) throws Exception {
-        JobSpecification compiled = rewriteCompileQuery(metadataProvider, query, null);
-        if (compiled != null) {
-            GlobalConfig.ASTERIX_LOGGER.info(compiled.toJSON().toString(1));
-            jobsToExecute.add(compiled);
-        }
+            IHyracksDataset hdc, boolean asyncResults) throws Exception {
 
-        return new QueryResult(query, metadataProvider.getResultSetId());
-    }
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        boolean bActiveTxn = true;
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireReadLatch();
 
-    private void runCreateIndexJob(IHyracksClientConnection hcc, CreateIndexStatement stmtCreateIndex,
-            AqlMetadataProvider metadataProvider) 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.
-        String dataverseName = stmtCreateIndex.getDataverseName() == null ? activeDefaultDataverse == null ? null
-                : activeDefaultDataverse.getDataverseName() : stmtCreateIndex.getDataverseName().getValue();
-        CompiledCreateIndexStatement createIndexStmt = new CompiledCreateIndexStatement(stmtCreateIndex.getIndexName()
-                .getValue(), dataverseName, stmtCreateIndex.getDatasetName().getValue(),
-                stmtCreateIndex.getFieldExprs(), stmtCreateIndex.getGramLength(), stmtCreateIndex.getIndexType());
-        JobSpecification spec = IndexOperations.buildSecondaryIndexCreationJobSpec(createIndexStmt, metadataProvider);
-        if (spec == null) {
-            throw new AsterixException("Failed to create job spec for creating index '"
-                    + stmtCreateIndex.getDatasetName() + "." + stmtCreateIndex.getIndexName() + "'");
-        }
-        JobId jobId = runJob(hcc, spec);
-        hcc.waitForCompletion(jobId);
-    }
+        try {
+            JobSpecification compiled = rewriteCompileQuery(metadataProvider, query, null);
 
-    private void handleCreateNodeGroupStatement(AqlMetadataProvider metadataProvider, Statement stmt,
-            List<JobSpecification> jobsToExecute) throws MetadataException, AlgebricksException, RemoteException,
-            ACIDException {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        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());
+            QueryResult queryResult = new QueryResult(query, metadataProvider.getResultSetId());
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+            bActiveTxn = false;
+
+            if (compiled != null) {
+                GlobalConfig.ASTERIX_LOGGER.info(compiled.toJSON().toString(1));
+                JobId jobId = runJob(hcc, compiled, false);
+
+                JSONObject response = new JSONObject();
+                if (asyncResults) {
+                    JSONArray handle = new JSONArray();
+                    handle.put(jobId.getId());
+                    handle.put(metadataProvider.getResultSetId().getId());
+                    response.put("handle", handle);
+                } else {
+                    ByteBuffer buffer = ByteBuffer.allocate(ResultReader.FRAME_SIZE);
+                    ResultReader resultReader = new ResultReader(hcc, hdc);
+                    resultReader.open(jobId, metadataProvider.getResultSetId());
+                    buffer.clear();
+                    JSONArray results = new JSONArray();
+                    while (resultReader.read(buffer) > 0) {
+                        results.put(ResultUtils.getJSONFromBuffer(buffer, resultReader.getFrameTupleAccessor()));
+                        buffer.clear();
+                    }
+                    response.put("results", results);
+                }
+                switch (pdf) {
+                    case HTML:
+                        out.println("<h3>Result:</h3>");
+                        out.println("<pre>");
+                        ResultUtils.prettyPrintHTML(out, response);
+                        out.println("</pre>");
+                        break;
+                    case TEXT:
+                    case JSON:
+                        out.print(response);
+                        break;
+                }
+                hcc.waitForCompletion(jobId);
             }
-            MetadataManager.INSTANCE.addNodegroup(mdTxnCtx, new NodeGroup(ngName, ncNames));
+
+            return queryResult;
+        } catch (Exception e) {
+            if (bActiveTxn) {
+                MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            }
+            throw new AlgebricksException(e);
+        } finally {
+            releaseReadLatch();
         }
     }
 
-    private JobId runJob(IHyracksClientConnection hcc, JobSpecification spec) throws Exception {
-        JobId[] jobIds = executeJobArray(hcc, new Job[] { new Job(spec) }, out, pdf);
+    private void handleCreateNodeGroupStatement(AqlMetadataProvider metadataProvider, Statement stmt)
+            throws MetadataException, AlgebricksException, RemoteException, ACIDException {
+
+        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        acquireWriteLatch();
+
+        try {
+            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));
+            }
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new AlgebricksException(e);
+        } finally {
+            releaseWriteLatch();
+        }
+    }
+
+    private JobId runJob(IHyracksClientConnection hcc, JobSpecification spec, boolean waitForCompletion)
+            throws Exception {
+        JobId[] jobIds = executeJobArray(hcc, new Job[] { new Job(spec) }, out, pdf, waitForCompletion);
         return jobIds[0];
     }
 
-    private void compileIndexDropStatement(IHyracksClientConnection hcc, String dataverseName, String datasetName,
-            String indexName, AqlMetadataProvider metadataProvider) throws Exception {
-        CompiledIndexDropStatement cds = new CompiledIndexDropStatement(dataverseName, datasetName, indexName);
-        JobId jobId = runJob(hcc, IndexOperations.buildDropSecondaryIndexJobSpec(cds, metadataProvider));
-        hcc.waitForCompletion(jobId);
-        MetadataManager.INSTANCE.dropIndex(metadataProvider.getMetadataTxnContext(), dataverseName, datasetName,
-                indexName);
-    }
-
-    private void compileDatasetDropStatement(IHyracksClientConnection hcc, String dataverseName, String datasetName,
-            AqlMetadataProvider metadataProvider) throws Exception {
-        MetadataTransactionContext mdTxnCtx = metadataProvider.getMetadataTxnContext();
-        CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(dataverseName, datasetName);
-        Dataset ds = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
-        if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
-            JobSpecification[] jobSpecs = DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider);
-            for (JobSpecification spec : jobSpecs) {
-                JobId jobId = runJob(hcc, spec);
-                hcc.waitForCompletion(jobId);
-            }
-        }
-        MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
-    }
-
-    public JobId[] executeJobArray(IHyracksClientConnection hcc, Job[] jobs, PrintWriter out, DisplayFormat pdf)
-            throws Exception {
+    public JobId[] executeJobArray(IHyracksClientConnection hcc, Job[] jobs, PrintWriter out, DisplayFormat pdf,
+            boolean waitForCompletion) throws Exception {
         JobId[] startedJobIds = new JobId[jobs.length];
         for (int i = 0; i < jobs.length; i++) {
             JobSpecification spec = jobs[i].getJobSpec();
             spec.setMaxReattempts(0);
             JobId jobId = hcc.startJob(spec);
             startedJobIds[i] = jobId;
+            if (waitForCompletion) {
+                hcc.waitForCompletion(jobId);
+            }
         }
         return startedJobIds;
     }
@@ -914,4 +1458,19 @@
         return format;
     }
 
+    private void acquireWriteLatch() {
+        MetadataManager.INSTANCE.acquireWriteLatch();
+    }
+
+    private void releaseWriteLatch() {
+        MetadataManager.INSTANCE.releaseWriteLatch();
+    }
+
+    private void acquireReadLatch() {
+        MetadataManager.INSTANCE.acquireReadLatch();
+    }
+
+    private void releaseReadLatch() {
+        MetadataManager.INSTANCE.releaseReadLatch();
+    }
 }
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 075ddd3..d4c8c34 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
@@ -25,21 +25,23 @@
 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.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints.DatasetCardinalityHint;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.Index;
 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.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledDatasetDropStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -72,14 +74,13 @@
 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.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.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 
 public class DatasetOperations {
 
@@ -88,7 +89,7 @@
 
     private static Logger LOGGER = Logger.getLogger(DatasetOperations.class.getName());
 
-    public static JobSpecification[] createDropDatasetJobSpec(CompiledDatasetDropStatement datasetDropStmt,
+    public static JobSpecification createDropDatasetJobSpec(CompiledDatasetDropStatement datasetDropStmt,
             AqlMetadataProvider metadataProvider) throws AlgebricksException, HyracksDataException, RemoteException,
             ACIDException, AsterixException {
 
@@ -104,60 +105,33 @@
 
         LOGGER.info("DROP DATASETPATH: " + datasetPath);
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
         Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
             throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            return new JobSpecification[0];
+            return new JobSpecification();
         }
 
-        List<Index> datasetIndexes = metadataProvider.getDatasetIndexes(dataset.getDataverseName(),
-                dataset.getDatasetName());
-        int numSecondaryIndexes = 0;
-        for (Index index : datasetIndexes) {
-            if (index.isSecondaryIndex()) {
-                numSecondaryIndexes++;
-            }
-        }
-        JobSpecification[] specs;
-        if (numSecondaryIndexes > 0) {
-            specs = new JobSpecification[numSecondaryIndexes + 1];
-            int i = 0;
-            // First, drop secondary indexes.
-            for (Index index : datasetIndexes) {
-                if (index.isSecondaryIndex()) {
-                    specs[i] = new JobSpecification();
-                    Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadataProvider
-                            .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
-                                    datasetName, index.getIndexName());
-                    TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
-                            storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
-                    AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
-                            idxSplitsAndConstraint.second);
-                    i++;
-                }
-            }
-        } else {
-            specs = new JobSpecification[1];
-        }
         JobSpecification specPrimary = new JobSpecification();
-        specs[specs.length - 1] = specPrimary;
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(), datasetName,
                         datasetName);
-        TreeIndexDropOperatorDescriptor primaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specPrimary,
-                storageManager, indexRegistryProvider, splitsAndConstraint.first);
+        IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
 
         specPrimary.addRoot(primaryBtreeDrop);
 
-        return specs;
+        return specPrimary;
     }
 
     public static JobSpecification createDatasetJobSpec(Dataverse dataverse, String datasetName,
@@ -189,6 +163,8 @@
         IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
                 itemType, format.getBinaryComparatorFactoryProvider());
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
+        int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
+
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
         FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
@@ -197,11 +173,23 @@
             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);
+
+        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+        ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
+                comparatorFactories, blooFilterKeyFields, true, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+                localResourceMetadata, LocalResource.LSMBTreeResource);
+
+        TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
+                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), localResourceFactoryProvider,
+                NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
         spec.addRoot(indexCreateOp);
@@ -233,6 +221,7 @@
         IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
                 itemType, format.getBinaryComparatorFactoryProvider());
         ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
+        int[] blooFilterKeyFields = DatasetUtils.createBloomFilterKeyFields(dataset);
 
         ExternalDatasetDetails externalDatasetDetails = new ExternalDatasetDetails(loadStmt.getAdapter(),
                 loadStmt.getProperties());
@@ -272,12 +261,23 @@
         }
         LOGGER.info("LOAD into File Splits: " + sb.toString());
 
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
+        String numElementsHintString = dataset.getHints().get("CARDINALITY");
+        long numElementsHint;
+        if (numElementsHintString == null) {
+            numElementsHint = DatasetCardinalityHint.DEFAULT;
+        } else {
+            numElementsHint = Long.parseLong(dataset.getHints().get("CARDINALITY"));
+        }
+
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                storageManager, indexRegistryProvider, splitsAndConstraint.first, typeTraits, comparatorFactories,
-                fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
-                NoOpOperationCallbackProvider.INSTANCE);
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields, fieldPermutation,
+                GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                 splitsAndConstraint.second);
 
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 a8f18ec..dbbbe44 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
@@ -1,8 +1,8 @@
 package edu.uci.ics.asterix.file;
 
+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.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -15,10 +15,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 {
 
@@ -47,13 +45,17 @@
         String indexName = indexDropStmt.getIndexName();
 
         JobSpecification spec = new JobSpecification();
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
-        TreeIndexDropOperatorDescriptor btreeDrop = new TreeIndexDropOperatorDescriptor(spec, storageManager,
-                indexRegistryProvider, splitsAndConstraint.first);
+        IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES));
         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..2a1349b 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,8 +1,11 @@
 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.config.GlobalConfig;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
@@ -12,12 +15,14 @@
 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.common.impls.NoOpOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 
 public class SecondaryBTreeCreator extends SecondaryIndexCreator {
 
@@ -28,10 +33,24 @@
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
+
+        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+        ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
+                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, false,
+                GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+                localResourceMetadata, LocalResource.LSMBTreeResource);
+
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), localResourceFactoryProvider,
+                NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -63,7 +82,11 @@
 
         // Create secondary BTree bulk load op.
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys,
-                new BTreeDataflowHelperFactory(), BTree.DEFAULT_FILL_FACTOR);
+                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), 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 87d3b08..e7c2e3e 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
@@ -20,14 +20,15 @@
 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.config.GlobalConfig;
+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;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
 import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints.DatasetCardinalityHint;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
@@ -66,11 +67,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.common.impls.NoOpOperationCallbackFactory;
+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, 
@@ -93,9 +94,12 @@
     protected String secondaryIndexName;
     protected boolean anySecondaryKeyIsNullable = false;
 
+    protected long numElementsHint;
     protected IBinaryComparatorFactory[] primaryComparatorFactories;
+    protected int[] primaryBloomFilterKeyFields;
     protected RecordDescriptor primaryRecDesc;
     protected IBinaryComparatorFactory[] secondaryComparatorFactories;
+    protected int[] secondaryBloomFilterKeyFields;
     protected RecordDescriptor secondaryRecDesc;
     protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
 
@@ -118,7 +122,9 @@
                 break;
             }
             case WORD_INVIX:
-            case NGRAM_INVIX: {
+            case NGRAM_INVIX:
+            case FUZZY_WORD_INVIX:
+            case FUZZY_NGRAM_INVIX: {
                 indexCreator = new SecondaryInvertedIndexCreator(physOptConf);
                 break;
             }
@@ -164,6 +170,13 @@
         // Must be called in this order.
         setPrimaryRecDescAndComparators();
         setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
+
+        String numElementsHintString = dataset.getHints().get("CARDINALITY");
+        if (numElementsHintString == null) {
+            numElementsHint = DatasetCardinalityHint.DEFAULT;
+        } else {
+            numElementsHint = Long.parseLong(dataset.getHints().get("CARDINALITY"));
+        }
     }
 
     protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
@@ -172,6 +185,7 @@
         ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
         primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
+        primaryBloomFilterKeyFields = new int[numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
         for (int i = 0; i < numPrimaryKeys; i++) {
             IAType keyType;
@@ -184,6 +198,7 @@
             primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
                     keyType, true);
             primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+            primaryBloomFilterKeyFields[i] = i;
         }
         primaryRecFields[numPrimaryKeys] = payloadSerde;
         primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
@@ -195,6 +210,7 @@
         List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
         secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys];
         secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
+        secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
         ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
         ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
@@ -211,6 +227,7 @@
             secondaryRecFields[i] = keySerde;
             secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
             secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
+            secondaryBloomFilterKeyFields[i] = i;
         }
         // Add serializers and comparators for primary index fields.
         for (int i = 0; i < numPrimaryKeys; i++) {
@@ -250,10 +267,14 @@
         // +Infinity
         int[] highKeyFields = null;
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
-                primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories, lowKeyFields,
-                highKeyFields, true, true, new BTreeDataflowHelperFactory(), false,
-                NoOpOperationCallbackProvider.INSTANCE);
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
+                primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
+                new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), false, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
@@ -309,9 +330,10 @@
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
                         secondaryIndexName);
         TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
+                AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                 secondarySplitsAndConstraint.first, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                fieldPermutation, fillFactor, dataflowHelperFactory, NoOpOperationCallbackProvider.INSTANCE);
+                secondaryBloomFilterKeyFields, fieldPermutation, fillFactor, false, numElementsHint,
+                dataflowHelperFactory, NoOpOperationCallbackFactory.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 dc12814..0bf379d 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,14 +2,18 @@
 
 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.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.LSMInvertedIndexLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -25,17 +29,23 @@
 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.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.ShortSerializerDeserializer;
 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.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.common.dataflow.IIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+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.dataflow.PartitionedLSMInvertedIndexDataflowHelperFactory;
+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.LocalResource;
 
 public class SecondaryInvertedIndexCreator extends SecondaryIndexCreator {
 
@@ -44,11 +54,11 @@
     private IBinaryComparatorFactory[] tokenComparatorFactories;
     private ITypeTraits[] tokenTypeTraits;
     private IBinaryTokenizerFactory tokenizerFactory;
-    private Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders;
     // For tokenization, sorting and loading. Represents <token, primary keys>.
     private int numTokenKeyPairFields;
     private IBinaryComparatorFactory[] tokenKeyPairComparatorFactories;
     private RecordDescriptor tokenKeyPairRecDesc;
+    private boolean isPartitioned;
 
     protected SecondaryInvertedIndexCreator(PhysicalOptimizationConfig physOptConf) {
         super(physOptConf);
@@ -56,8 +66,8 @@
 
     @Override
     @SuppressWarnings("rawtypes")
-    protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadata)
-            throws AlgebricksException, AsterixException {
+    protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+            AqlMetadataProvider metadata) throws AlgebricksException, AsterixException {
         // Sanity checks.
         if (numPrimaryKeys > 1) {
             throw new AsterixException("Cannot create inverted index on dataset with composite primary key.");
@@ -65,7 +75,14 @@
         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.
+        if (createIndexStmt.getIndexType() == IndexType.FUZZY_WORD_INVIX
+                || createIndexStmt.getIndexType() == IndexType.FUZZY_NGRAM_INVIX) {
+            isPartitioned = true;
+        } else {
+            isPartitioned = false;
+        }
+        // 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];
@@ -84,36 +101,47 @@
         }
         secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
         // Comparators and type traits for tokens.
-        tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
-        tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
-        tokenComparatorFactories[0] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
-        tokenTypeTraits[0] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
+        int numTokenFields = (!isPartitioned) ? numSecondaryKeys : numSecondaryKeys + 1;
+        tokenComparatorFactories = new IBinaryComparatorFactory[numTokenFields];
+        tokenTypeTraits = new ITypeTraits[numTokenFields];
+        tokenComparatorFactories[0] = NonTaggedFormatUtil.getTokenBinaryComparatorFactory(secondaryKeyType);
+        tokenTypeTraits[0] = NonTaggedFormatUtil.getTokenTypeTrait(secondaryKeyType);
+        if (isPartitioned) {
+            // The partitioning field is hardcoded to be a short *without* an Asterix type tag.
+            tokenComparatorFactories[1] = PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
+            tokenTypeTraits[1] = ShortPointable.TYPE_TRAITS;
+        }
         // 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(),
+        tokenizerFactory = NonTaggedFormatUtil.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];
         }
-        // Get file split providers for the BTree and inverted-list files.
-        fileSplitProviders = metadata.getInvertedIndexFileSplitProviders(secondaryFileSplitProvider);
         // For tokenization, sorting and loading.
-        // One token + primary keys.
-        numTokenKeyPairFields = 1 + numPrimaryKeys;
+        // One token (+ optional partitioning field) + primary keys.
+        numTokenKeyPairFields = (!isPartitioned) ? 1 + numPrimaryKeys : 2 + numPrimaryKeys;
         ISerializerDeserializer[] tokenKeyPairFields = new ISerializerDeserializer[numTokenKeyPairFields];
         ITypeTraits[] tokenKeyPairTypeTraits = new ITypeTraits[numTokenKeyPairFields];
         tokenKeyPairComparatorFactories = new IBinaryComparatorFactory[numTokenKeyPairFields];
         tokenKeyPairFields[0] = serdeProvider.getSerializerDeserializer(secondaryKeyType);
         tokenKeyPairTypeTraits[0] = tokenTypeTraits[0];
-        tokenKeyPairComparatorFactories[0] = InvertedIndexAccessMethod
-                .getTokenBinaryComparatorFactory(secondaryKeyType);
+        tokenKeyPairComparatorFactories[0] = NonTaggedFormatUtil.getTokenBinaryComparatorFactory(secondaryKeyType);
+        int pkOff = 1;
+        if (isPartitioned) {
+            tokenKeyPairFields[1] = ShortSerializerDeserializer.INSTANCE;
+            tokenKeyPairTypeTraits[1] = tokenTypeTraits[1];
+            tokenKeyPairComparatorFactories[1] = PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
+            pkOff = 2;
+        }
         for (int i = 0; i < numPrimaryKeys; i++) {
-            tokenKeyPairFields[i + 1] = primaryRecDesc.getFields()[i];
-            tokenKeyPairTypeTraits[i + 1] = primaryRecDesc.getTypeTraits()[i];
-            tokenKeyPairComparatorFactories[i + 1] = primaryComparatorFactories[i];
+            tokenKeyPairFields[i + pkOff] = primaryRecDesc.getFields()[i];
+            tokenKeyPairTypeTraits[i + pkOff] = primaryRecDesc.getTypeTraits()[i];
+            tokenKeyPairComparatorFactories[i + pkOff] = primaryComparatorFactories[i];
         }
         tokenKeyPairRecDesc = new RecordDescriptor(tokenKeyPairFields, tokenKeyPairTypeTraits);
     }
@@ -121,11 +149,20 @@
     @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);
+
+        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+        ILocalResourceMetadata localResourceMetadata = new LSMInvertedIndexLocalResourceMetadata(invListsTypeTraits,
+                primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory,
+                GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES, isPartitioned);
+        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+                localResourceMetadata, LocalResource.LSMInvertedIndexResource);
+
+        IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
+        LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+                invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
+                localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(invIndexCreateOp);
@@ -137,7 +174,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.
@@ -146,7 +183,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);
@@ -159,7 +197,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);
@@ -178,16 +216,13 @@
     }
 
     private AbstractOperatorDescriptor createTokenizerOp(JobSpecification spec) throws AlgebricksException {
-        int[] fieldsToTokenize = new int[numSecondaryKeys];
-        for (int i = 0; i < numSecondaryKeys; i++) {
-            fieldsToTokenize[i] = i;
-        }
+        int docField = 0;
         int[] primaryKeyFields = new int[numPrimaryKeys];
         for (int i = 0; i < numPrimaryKeys; i++) {
             primaryKeyFields[i] = numSecondaryKeys + i;
         }
         BinaryTokenizerOperatorDescriptor tokenizerOp = new BinaryTokenizerOperatorDescriptor(spec,
-                tokenKeyPairRecDesc, tokenizerFactory, fieldsToTokenize, primaryKeyFields);
+                tokenKeyPairRecDesc, tokenizerFactory, docField, primaryKeyFields, isPartitioned);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, tokenizerOp,
                 primaryPartitionConstraint);
         return tokenizerOp;
@@ -207,18 +242,37 @@
         return sortOp;
     }
 
-    private InvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
-        int[] fieldPermutation = new int[numSecondaryKeys + numPrimaryKeys];
+    private LSMInvertedIndexBulkLoadOperatorDescriptor createInvertedIndexBulkLoadOp(JobSpecification spec) {
+        int[] fieldPermutation = new int[numTokenKeyPairFields];
         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,
+        IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
+        LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
+                spec, fieldPermutation, false, numElementsHint, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                secondaryFileSplitProvider, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits,
                 tokenComparatorFactories, invListsTypeTraits, primaryComparatorFactories, tokenizerFactory,
-                new BTreeDataflowHelperFactory(), NoOpOperationCallbackProvider.INSTANCE);
+                dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return invIndexBulkLoadOp;
     }
+
+    private IIndexDataflowHelperFactory createDataflowHelperFactory() {
+        if (!isPartitioned) {
+            return new LSMInvertedIndexDataflowHelperFactory(
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+        } else {
+            return new PartitionedLSMInvertedIndexDataflowHelperFactory(
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+        }
+    }
 }
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 ba8f6cd..6f400f9 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,8 +2,8 @@
 
 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.config.GlobalConfig;
+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;
@@ -11,8 +11,12 @@
 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.transaction.management.resource.ILocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -32,14 +36,18 @@
 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.common.impls.NoOpOperationCallbackFactory;
+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.ILocalResourceFactoryProvider;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 
 @SuppressWarnings("rawtypes")
 public class SecondaryRTreeCreator extends SecondaryIndexCreator {
 
     protected IPrimitiveValueProviderFactory[] valueProviderFactories;
     protected int numNestedSecondaryKeyFields;
+    protected ATypeTag keyType;
 
     protected SecondaryRTreeCreator(PhysicalOptimizationConfig physOptConf) {
         super(physOptConf);
@@ -48,10 +56,27 @@
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
+
+        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+        ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(
+                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, primaryComparatorFactories,
+                valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
+                        secondaryComparatorFactories.length), GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
+                GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+                localResourceMetadata, LocalResource.LSMRTreeResource);
+
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                AsterixStorageManagerInterface.INSTANCE, AsterixIndexRegistryProvider.INSTANCE,
-                secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
-                new RTreeDataflowHelperFactory(valueProviderFactories), NoOpOperationCallbackProvider.INSTANCE);
+                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
+                new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+                                keyType, secondaryComparatorFactories.length),
+                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
+                localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -60,8 +85,8 @@
     }
 
     @Override
-    protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadata)
-            throws AlgebricksException, AsterixException {
+    protected void setSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+            AqlMetadataProvider metadata) throws AlgebricksException, AsterixException {
         List<String> secondaryKeyFields = createIndexStmt.getKeyFields();
         int numSecondaryKeys = secondaryKeyFields.size();
         if (numSecondaryKeys != 1) {
@@ -86,6 +111,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);
@@ -124,8 +150,16 @@
         }
 
         // Create secondary RTree bulk load op.
-        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec,
-                numNestedSecondaryKeyFields, new RTreeDataflowHelperFactory(valueProviderFactories),
+        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+                spec,
+                numNestedSecondaryKeyFields,
+                new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+                        primaryComparatorFactories, AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+                                keyType, secondaryComparatorFactories.length),
+                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
                 BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
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
deleted file mode 100644
index 6baa865..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestKeywordIndexJob.java
+++ /dev/null
@@ -1,228 +0,0 @@
-package edu.uci.ics.asterix.file;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.util.ArrayList;
-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.exceptions.AsterixException;
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-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.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-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.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;
-
-public class TestKeywordIndexJob {
-
-    private static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
-    static {
-        typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
-    }
-
-    public static int DEFAULT_INPUT_DATA_COLUMN = 0;
-    public static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
-
-    @SuppressWarnings("unchecked")
-    public JobSpecification createJobSpec() throws AsterixException, HyracksDataException {
-
-        JobSpecification spec = new JobSpecification();
-
-        // ---------- START GENERAL BTREE STUFF
-
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
-        // ---------- END GENERAL BTREE STUFF
-
-        List<String> nodeGroup = new ArrayList<String>();
-        nodeGroup.add("nc1");
-        nodeGroup.add("nc2");
-
-        // ---------- START KEY PROVIDER OP
-
-        // TODO: should actually be empty tuple source
-        // build tuple containing low and high search keys
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(1); // just one dummy field
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        AObjectSerializerDeserializer.INSTANCE.serialize(new AString("Jodi Rotruck"), dos); // dummy
-        // field
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { AObjectSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        String[] keyProviderOpLocationConstraint = new String[nodeGroup.size()];
-        for (int p = 0; p < nodeGroup.size(); p++) {
-            keyProviderOpLocationConstraint[p] = nodeGroup.get(p);
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, keyProviderOpLocationConstraint);
-
-        // ---------- END KEY PROVIDER OP
-
-        // ---------- START SECONRARY INDEX SCAN
-
-        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[2];
-        secondaryTypeTraits[0] = new ITypeTraits() {
-
-            @Override
-            public boolean isFixedLength() {
-                return false;
-            }
-
-            @Override
-            public int getFixedLength() {
-                return -1;
-            }
-        };
-
-        secondaryTypeTraits[1] = new ITypeTraits() {
-
-            @Override
-            public boolean isFixedLength() {
-                return true;
-            }
-
-            @Override
-            public int getFixedLength() {
-                return 5;
-            }
-        };
-
-        ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[2];
-        secondaryRecFields[0] = AObjectSerializerDeserializer.INSTANCE;
-        secondaryRecFields[1] = AObjectSerializerDeserializer.INSTANCE;
-        IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
-        secondaryComparatorFactories[0] = AObjectAscBinaryComparatorFactory.INSTANCE;
-        secondaryComparatorFactories[1] = AObjectAscBinaryComparatorFactory.INSTANCE;
-
-        int[] lowKeyFields = null;
-        int[] highKeyFields = null;
-        RecordDescriptor secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
-        // TODO: change file splits according to mount points in cluster config
-        IFileSplitProvider secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] {
-                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);
-        String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
-        for (int p = 0; p < nodeGroup.size(); p++) {
-            secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp,
-                secondarySearchOpLocationConstraint);
-
-        // ---------- END SECONDARY INDEX SCAN
-
-        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        String[] printerLocationConstraint = new String[nodeGroup.size()];
-        for (int p = 0; p < nodeGroup.size(); p++) {
-            printerLocationConstraint[p] = nodeGroup.get(p);
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, printerLocationConstraint);
-
-        // ---------- START CONNECT THE OPERATORS
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, printer, 0);
-
-        // ---------- END CONNECT THE OPERATORS
-
-        spec.addRoot(printer);
-
-        return spec;
-    }
-
-    public static void main(String[] args) throws Exception {
-        String host;
-        String appName;
-        String ddlFile;
-
-        switch (args.length) {
-            case 0: {
-                host = "127.0.0.1";
-                appName = "asterix";
-                ddlFile = "/home/abehm/workspace/asterix/src/test/resources/demo0927/local/create-index.aql";
-                System.out.println("No arguments specified, using defauls:");
-                System.out.println("HYRACKS HOST: " + host);
-                System.out.println("APPNAME:      " + appName);
-                System.out.println("DDLFILE:      " + ddlFile);
-            }
-                break;
-
-            case 3: {
-                host = args[0];
-                appName = args[1];
-                ddlFile = args[2];
-            }
-                break;
-
-            default: {
-                System.out.println("USAGE:");
-                System.out.println("ARG 1: Hyracks Host (IP or Hostname)");
-                System.out.println("ARG 2: Application Name (e.g., asterix)");
-                System.out.println("ARG 3: DDL File");
-                host = null;
-                appName = null;
-                ddlFile = null;
-                System.exit(0);
-            }
-                break;
-        }
-
-        int port = 1098;
-        IHyracksClientConnection hcc = new HyracksConnection(host, port);
-
-        TestKeywordIndexJob tij = new TestKeywordIndexJob();
-        JobSpecification jobSpec = tij.createJobSpec();
-
-        long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(jobSpec);
-        hcc.waitForCompletion(jobId);
-        long end = System.currentTimeMillis();
-        System.err.println(start + " " + end + " " + (end - start));
-    }
-}
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
deleted file mode 100644
index 3faca96..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/TestSecondaryIndexJob.java
+++ /dev/null
@@ -1,228 +0,0 @@
-package edu.uci.ics.asterix.file;
-
-import java.io.DataOutput;
-import java.io.File;
-import java.util.ArrayList;
-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.exceptions.AsterixException;
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AObjectSerializerDeserializer;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.api.client.HyracksConnection;
-import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
-import edu.uci.ics.hyracks.api.constraints.PartitionConstraintHelper;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-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.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.api.job.JobId;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.DoubleParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.FloatParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IntegerParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.LongParserFactory;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
-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.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;
-
-public class TestSecondaryIndexJob {
-
-    private static final HashMap<ATypeTag, IValueParserFactory> typeToValueParserFactMap = new HashMap<ATypeTag, IValueParserFactory>();
-    static {
-        typeToValueParserFactMap.put(ATypeTag.INT32, IntegerParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.FLOAT, FloatParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.DOUBLE, DoubleParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.INT64, LongParserFactory.INSTANCE);
-        typeToValueParserFactMap.put(ATypeTag.STRING, UTF8StringParserFactory.INSTANCE);
-    }
-
-    public static int DEFAULT_INPUT_DATA_COLUMN = 0;
-    public static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
-
-    @SuppressWarnings("unchecked")
-    public JobSpecification createJobSpec() throws AsterixException, HyracksDataException {
-
-        JobSpecification spec = new JobSpecification();
-
-        // ---------- START GENERAL BTREE STUFF
-
-        IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
-        IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
-
-        // ---------- END GENERAL BTREE STUFF
-
-        List<String> nodeGroup = new ArrayList<String>();
-        nodeGroup.add("nc1");
-        nodeGroup.add("nc2");
-
-        // ---------- START KEY PROVIDER OP
-
-        // TODO: should actually be empty tuple source
-        // build tuple containing low and high search keys
-        ArrayTupleBuilder tb = new ArrayTupleBuilder(1); // just one dummy field
-        DataOutput dos = tb.getDataOutput();
-
-        tb.reset();
-        AObjectSerializerDeserializer.INSTANCE.serialize(new AString("Jodi Rotruck"), dos); // dummy
-        // field
-        tb.addFieldEndOffset();
-
-        ISerializerDeserializer[] keyRecDescSers = { AObjectSerializerDeserializer.INSTANCE };
-        RecordDescriptor keyRecDesc = new RecordDescriptor(keyRecDescSers);
-
-        ConstantTupleSourceOperatorDescriptor keyProviderOp = new ConstantTupleSourceOperatorDescriptor(spec,
-                keyRecDesc, tb.getFieldEndOffsets(), tb.getByteArray(), tb.getSize());
-        String[] keyProviderOpLocationConstraint = new String[nodeGroup.size()];
-        for (int p = 0; p < nodeGroup.size(); p++) {
-            keyProviderOpLocationConstraint[p] = nodeGroup.get(p);
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, keyProviderOp, keyProviderOpLocationConstraint);
-
-        // ---------- END KEY PROVIDER OP
-
-        // ---------- START SECONRARY INDEX SCAN
-
-        ITypeTraits[] secondaryTypeTraits = new ITypeTraits[2];
-        secondaryTypeTraits[0] = new ITypeTraits() {
-
-            @Override
-            public boolean isFixedLength() {
-                return false;
-            }
-
-            @Override
-            public int getFixedLength() {
-                return -1;
-            }
-        };
-
-        secondaryTypeTraits[1] = new ITypeTraits() {
-
-            @Override
-            public boolean isFixedLength() {
-                return true;
-            }
-
-            @Override
-            public int getFixedLength() {
-                return 5;
-            }
-        };
-
-        ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[2];
-        secondaryRecFields[0] = AObjectSerializerDeserializer.INSTANCE;
-        secondaryRecFields[1] = AObjectSerializerDeserializer.INSTANCE;
-        IBinaryComparatorFactory[] secondaryComparatorFactories = new IBinaryComparatorFactory[2];
-        secondaryComparatorFactories[0] = AObjectAscBinaryComparatorFactory.INSTANCE;
-        secondaryComparatorFactories[1] = AObjectAscBinaryComparatorFactory.INSTANCE;
-
-        int[] lowKeyFields = null; // -infinity
-        int[] highKeyFields = null; // +infinity
-        RecordDescriptor secondaryRecDesc = new RecordDescriptor(secondaryRecFields);
-        // TODO: change file splits according to mount points in cluster config
-        IFileSplitProvider secondarySplitProvider = new ConstantFileSplitProvider(new FileSplit[] {
-                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);
-        String[] secondarySearchOpLocationConstraint = new String[nodeGroup.size()];
-        for (int p = 0; p < nodeGroup.size(); p++) {
-            secondarySearchOpLocationConstraint[p] = nodeGroup.get(p);
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, secondarySearchOp,
-                secondarySearchOpLocationConstraint);
-
-        // ---------- END SECONDARY INDEX SCAN
-
-        PrinterOperatorDescriptor printer = new PrinterOperatorDescriptor(spec);
-        String[] printerLocationConstraint = new String[nodeGroup.size()];
-        for (int p = 0; p < nodeGroup.size(); p++) {
-            printerLocationConstraint[p] = nodeGroup.get(p);
-        }
-        PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, printerLocationConstraint);
-
-        // ---------- START CONNECT THE OPERATORS
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, secondarySearchOp, 0);
-
-        spec.connect(new OneToOneConnectorDescriptor(spec), secondarySearchOp, 0, printer, 0);
-
-        // ---------- END CONNECT THE OPERATORS
-
-        spec.addRoot(printer);
-
-        return spec;
-    }
-
-    public static void main(String[] args) throws Exception {
-        String host;
-        String appName;
-        String ddlFile;
-
-        switch (args.length) {
-            case 0: {
-                host = "127.0.0.1";
-                appName = "asterix";
-                ddlFile = "/home/nicnic/workspace/asterix/trunk/asterix/asterix-app/src/test/resources/demo0927/local/create-index.aql";
-                System.out.println("No arguments specified, using defauls:");
-                System.out.println("HYRACKS HOST: " + host);
-                System.out.println("APPNAME:      " + appName);
-                System.out.println("DDLFILE:      " + ddlFile);
-            }
-                break;
-
-            case 3: {
-                host = args[0];
-                appName = args[1];
-                ddlFile = args[2];
-            }
-                break;
-
-            default: {
-                System.out.println("USAGE:");
-                System.out.println("ARG 1: Hyracks Host (IP or Hostname)");
-                System.out.println("ARG 2: Application Name (e.g., asterix)");
-                System.out.println("ARG 3: DDL File");
-                host = null;
-                appName = null;
-                ddlFile = null;
-                System.exit(0);
-            }
-                break;
-        }
-
-        int port = 1098;
-        IHyracksClientConnection hcc = new HyracksConnection(host, port);
-
-        TestSecondaryIndexJob tij = new TestSecondaryIndexJob();
-        JobSpecification jobSpec = tij.createJobSpec();
-
-        long start = System.currentTimeMillis();
-        JobId jobId = hcc.startJob(jobSpec);
-        hcc.waitForCompletion(jobId);
-        long end = System.currentTimeMillis();
-        System.err.println(start + " " + end + " " + (end - start));
-    }
-}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixNodeState.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixNodeState.java
deleted file mode 100644
index dbf1625..0000000
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/AsterixNodeState.java
+++ /dev/null
@@ -1,33 +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.hyracks.bootstrap;
-
-import java.io.Serializable;
-
-public class AsterixNodeState implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-
-    private int apiNodeDataServerPort;
-
-    public int getAPINodeDataServerPort() {
-        return apiNodeDataServerPort;
-    }
-
-    public void setAPINodeDataServerPort(int port) {
-        this.apiNodeDataServerPort = port;
-    }
-
-}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index cee5b7f..8d38b56 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -2,7 +2,6 @@
 
 import java.rmi.RemoteException;
 import java.rmi.server.UnicastRemoteObject;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -12,103 +11,159 @@
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataBootstrap;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
+import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager.SystemState;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
 
 public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
-    private static final Logger LOGGER = Logger.getLogger(NCApplicationEntryPoint.class.getName());
+	private static final Logger LOGGER = Logger
+			.getLogger(NCApplicationEntryPoint.class.getName());
 
-    private INCApplicationContext ncApplicationContext = null;
-    private AsterixAppRuntimeContext runtimeContext;
-    private String nodeId;
-    private boolean isMetadataNode = false;
-    private boolean stopInitiated = false;
+	private INCApplicationContext ncApplicationContext = null;
+	private AsterixAppRuntimeContext runtimeContext;
+	private String nodeId;
+	private boolean isMetadataNode = false;
+	private boolean stopInitiated = false;
+	private SystemState systemState = SystemState.NEW_UNIVERSE;
 
-    @Override
-    public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception {
-        ncApplicationContext = ncAppCtx;
-        nodeId = ncApplicationContext.getNodeId();
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Starting Asterix node controller: " + nodeId);
-        }
+	@Override
+	public void start(INCApplicationContext ncAppCtx, String[] args)
+			throws Exception {
+		ncApplicationContext = ncAppCtx;
+		nodeId = ncApplicationContext.getNodeId();
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("Starting Asterix node controller: " + nodeId);
+		}
 
-        runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext);
-        runtimeContext.initialize();
-        ncApplicationContext.setApplicationObject(runtimeContext);
-        JVMShutdownHook sHook = new JVMShutdownHook(this);
-        Runtime.getRuntime().addShutdownHook(sHook);
+		runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext);
+		runtimeContext.initialize();
+		ncApplicationContext.setApplicationObject(runtimeContext);
+		JVMShutdownHook sHook = new JVMShutdownHook(this);
+		Runtime.getRuntime().addShutdownHook(sHook);
 
-    }
+		// #. recover if the system is corrupted by checking system state.
+		IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem()
+				.getRecoveryManager();
+		systemState = recoveryMgr.getSystemState();
+		if (systemState != SystemState.NEW_UNIVERSE) {
+			PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
+					.getLocalResourceRepository();
+			localResourceRepository.initialize(nodeId, null, false,
+					runtimeContext.getResourceIdFactory());
+		}
+		if (systemState == SystemState.CORRUPTED) {
+			recoveryMgr.startRecovery(true);
+		} else if (systemState == SystemState.NEW_UNIVERSE) {
+			recoveryMgr.checkpoint(true);
+		}
+	}
 
-    @Override
-    public void stop() throws Exception {
-        if (!stopInitiated) {
-            stopInitiated = true;
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Stopping Asterix node controller: " + nodeId);
-            }
+	@Override
+	public void stop() throws Exception {
+		if (!stopInitiated) {
+			stopInitiated = true;
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Stopping Asterix node controller: " + nodeId);
+			}
 
-            if (isMetadataNode) {
-                MetadataBootstrap.stopUniverse();
-            }
-            runtimeContext.deinitialize();
-        } else {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Duplicate attempt to stop ignored: " + nodeId);
-            }
-        }
-    }
+			IRecoveryManager recoveryMgr = runtimeContext
+					.getTransactionSubsystem().getRecoveryManager();
+			recoveryMgr.checkpoint(true);
 
-    @Override
-    public void notifyStartupComplete() throws Exception {
-        IAsterixStateProxy proxy = (IAsterixStateProxy) ncApplicationContext.getDistributedState();
-        isMetadataNode = nodeId.equals(proxy.getAsterixProperties().getMetadataNodeName());
-        if (isMetadataNode) {
-            registerRemoteMetadataNode(proxy);
+			if (isMetadataNode) {
+				MetadataBootstrap.stopUniverse();
+			}
+			runtimeContext.deinitialize();
+		} else {
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Duplicate attempt to stop ignored: " + nodeId);
+			}
+		}
+	}
 
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Bootstrapping metadata");
-            }
-            MetadataManager.INSTANCE = new MetadataManager(proxy);
-            MetadataManager.INSTANCE.init();
-            MetadataBootstrap.startUniverse(proxy.getAsterixProperties(), ncApplicationContext);
-        }
-    }
+	@Override
+	public void notifyStartupComplete() throws Exception {
+		IAsterixStateProxy proxy = (IAsterixStateProxy) ncApplicationContext
+				.getDistributedState();
 
-    public void registerRemoteMetadataNode(IAsterixStateProxy proxy) throws RemoteException {
-        IMetadataNode stub = null;
-        MetadataNode.INSTANCE.initialize(runtimeContext);
-        stub = (IMetadataNode) UnicastRemoteObject.exportObject(MetadataNode.INSTANCE, 0);
-        proxy.setMetadataNode(stub);
+		if (systemState == SystemState.NEW_UNIVERSE) {
+			PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
+					.getLocalResourceRepository();
+			System.out.println("nodeid" + nodeId);
+			System.out.println("proxy" + proxy);
+			System.out.println("stores"
+					+ proxy.getAsterixProperties().getStores());
+			System.out.println("store"
+					+ proxy.getAsterixProperties().getStores().get(nodeId)[0]);
 
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Metadata node bound");
-        }
-    }
+			localResourceRepository.initialize(nodeId, proxy
+					.getAsterixProperties().getStores().get(nodeId)[0], true,
+					null);
+		}
 
-    /**
-     * Shutdown hook that invokes {@link NCApplicationEntryPoint#stop() stop} method.
-     */
-    private static class JVMShutdownHook extends Thread {
+		isMetadataNode = nodeId.equals(proxy.getAsterixProperties()
+				.getMetadataNodeName());
+		if (isMetadataNode) {
+			registerRemoteMetadataNode(proxy);
 
-        private final NCApplicationEntryPoint ncAppEntryPoint;
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Bootstrapping metadata");
+			}
+			MetadataManager.INSTANCE = new MetadataManager(proxy);
+			MetadataManager.INSTANCE.init();
+			MetadataBootstrap.startUniverse(proxy.getAsterixProperties(),
+					ncApplicationContext,
+					systemState == SystemState.NEW_UNIVERSE);
+			MetadataBootstrap.startDDLRecovery();
+		}
 
-        public JVMShutdownHook(NCApplicationEntryPoint ncAppEntryPoint) {
-            this.ncAppEntryPoint = ncAppEntryPoint;
-        }
+		IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem()
+				.getRecoveryManager();
+		recoveryMgr.checkpoint(true);
 
-        public void run() {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Shutdown hook in progress");
-            }
-            try {
-                ncAppEntryPoint.stop();
-            } catch (Exception e) {
-                if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("Exception in executing shutdown hook" + e);
-                }
-            }
-        }
-    }
+		// TODO
+		// reclaim storage for orphaned index artifacts in NCs.
+	}
+
+	public void registerRemoteMetadataNode(IAsterixStateProxy proxy)
+			throws RemoteException {
+		IMetadataNode stub = null;
+		MetadataNode.INSTANCE.initialize(runtimeContext);
+		stub = (IMetadataNode) UnicastRemoteObject.exportObject(
+				MetadataNode.INSTANCE, 0);
+		proxy.setMetadataNode(stub);
+
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("Metadata node bound");
+		}
+	}
+
+	/**
+	 * Shutdown hook that invokes {@link NCApplicationEntryPoint#stop() stop}
+	 * method.
+	 */
+	private static class JVMShutdownHook extends Thread {
+
+		private final NCApplicationEntryPoint ncAppEntryPoint;
+
+		public JVMShutdownHook(NCApplicationEntryPoint ncAppEntryPoint) {
+			this.ncAppEntryPoint = ncAppEntryPoint;
+		}
+
+		public void run() {
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Shutdown hook in progress");
+			}
+			try {
+				ncAppEntryPoint.stop();
+			} catch (Exception e) {
+				if (LOGGER.isLoggable(Level.WARNING)) {
+					LOGGER.warning("Exception in executing shutdown hook" + e);
+				}
+			}
+		}
+	}
 
 }
\ No newline at end of file
diff --git a/asterix-app/src/main/resources/asterix-idefix.properties b/asterix-app/src/main/resources/asterix-idefix.properties
deleted file mode 100755
index 278bcb4..0000000
--- a/asterix-app/src/main/resources/asterix-idefix.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-nc1.stores=/home/nicnic/Work/Asterix/tests/tpch/nc1data
-nc2.stores=/home/nicnic/Work/Asterix/tests/tpch/nc2data
diff --git a/asterix-app/src/main/resources/asterix-peach.properties b/asterix-app/src/main/resources/asterix-peach.properties
deleted file mode 100644
index 20a6eeb..0000000
--- a/asterix-app/src/main/resources/asterix-peach.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-nc1.stores=/tmp/nc1/
-nc2.stores=/tmp/nc2/
diff --git a/asterix-app/src/main/resources/asterix-rainbow.properties b/asterix-app/src/main/resources/asterix-rainbow.properties
deleted file mode 100644
index d5febe4..0000000
--- a/asterix-app/src/main/resources/asterix-rainbow.properties
+++ /dev/null
@@ -1,5 +0,0 @@
-rainbow-01.stores=/data/onose/rainbow-01/
-rainbow-02.stores=/data/onose/rainbow-02/
-rainbow-03.stores=/data/onose/rainbow-03/
-rainbow-04.stores=/data/onose/rainbow-04/
-rainbow-05.stores=/data/onose/rainbow-05/
\ No newline at end of file
diff --git a/asterix-app/src/main/resources/asterix.properties b/asterix-app/src/main/resources/asterix.properties
deleted file mode 100755
index 78cd2b9..0000000
--- a/asterix-app/src/main/resources/asterix.properties
+++ /dev/null
@@ -1,10 +0,0 @@
-asterix-001.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-002.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-003.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-004.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-005.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-006.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-007.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-008.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-009.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
-asterix-010.stores=/mnt/data/sda/space/onose,/mnt/data/sdb/space/onose,/mnt/data/sdc/space/onose,/mnt/data/sdd/space/onose
diff --git a/asterix-app/src/main/resources/hyracks-initdb-deployment.properties b/asterix-app/src/main/resources/hyracks-initdb-deployment.properties
deleted file mode 100644
index e40db59..0000000
--- a/asterix-app/src/main/resources/hyracks-initdb-deployment.properties
+++ /dev/null
@@ -1,2 +0,0 @@
-cc.bootstrap.class=edu.uci.ics.initdb.hyracks.bootstrap.CCBootstrapImpl
-nc.bootstrap.class=edu.uci.ics.initdb.hyracks.bootstrap.NCBootstrapImpl
\ No newline at end of file
diff --git a/asterix-app/src/main/resources/idefix-4nc.properties b/asterix-app/src/main/resources/idefix-4nc.properties
deleted file mode 100755
index 747eb41..0000000
--- a/asterix-app/src/main/resources/idefix-4nc.properties
+++ /dev/null
@@ -1,4 +0,0 @@
-nc1.stores=/home/nicnic/Work/Asterix/tests/tpch/nc1data
-nc2.stores=/home/nicnic/Work/Asterix/tests/tpch/nc2data
-nc3.stores=/home/nicnic/Work/Asterix/tests/tpch/nc3data
-nc4.stores=/home/nicnic/Work/Asterix/tests/tpch/nc4data
diff --git a/asterix-app/src/main/resources/test.properties b/asterix-app/src/main/resources/test.properties
index 01a593b..4947dbf 100755
--- a/asterix-app/src/main/resources/test.properties
+++ b/asterix-app/src/main/resources/test.properties
@@ -1,5 +1,5 @@
 MetadataNode=nc1
 NewUniverse=true
-nc1.stores=/tmp/nc1data/
-nc2.stores=/tmp/nc2data/
+nc1.stores=nc1data
+nc2.stores=nc2data
 OutputDir=/tmp/asterix_output/
diff --git a/asterix-app/src/main/resources/testnc1.properties b/asterix-app/src/main/resources/testnc1.properties
deleted file mode 100755
index c0ad3de..0000000
--- a/asterix-app/src/main/resources/testnc1.properties
+++ /dev/null
@@ -1 +0,0 @@
-nc1.stores=nc1data
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index c38410a..7aa04c3 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -54,6 +54,13 @@
         try {
 
             while ((lineExpected = readerExpected.readLine()) != null) {
+                // Skipe the blank line in the expected file.
+                if (lineExpected.isEmpty()) {
+                    continue;
+                }
+                if (jArray.length() <= chunkCounter) {
+                    throw new Exception("No more results available.");
+                }
                 JSONArray resultArray = jArray.getJSONArray(chunkCounter);
 
                 if ((lineActual = resultArray.getString(recordCounter)) == null) {
@@ -70,15 +77,8 @@
                 if (recordCounter >= resultArray.length()) {
                     chunkCounter++;
                     recordCounter = 0;
-                    if (chunkCounter >= jArray.length()) {
-                        break;
-                    }
                 }
             }
-
-            while ((lineExpected = readerExpected.readLine()) != null) {
-                // TODO(khurram): Print out the remaining expected file contents
-            }
         } finally {
             readerExpected.close();
         }
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
index 369d1a8..2519544 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/metadata/MetadataTest.java
@@ -23,14 +23,6 @@
 import java.util.List;
 import java.util.logging.Logger;
 
-import org.apache.commons.io.FileUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
 import org.apache.commons.httpclient.DefaultHttpMethodRetryHandler;
 import org.apache.commons.httpclient.HttpClient;
 import org.apache.commons.httpclient.HttpStatus;
@@ -39,18 +31,20 @@
 import org.apache.commons.httpclient.params.HttpMethodParams;
 import org.apache.commons.io.FileUtils;
 import org.json.JSONObject;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 import edu.uci.ics.asterix.api.common.AsterixHyracksIntegrationUtil;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.test.aql.TestsUtils;
 import edu.uci.ics.asterix.testframework.context.TestCaseContext;
-
-import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.external.util.IdentitiyResolverFactory;
 import edu.uci.ics.asterix.testframework.context.TestFileContext;
 import edu.uci.ics.asterix.testframework.xml.TestCase.CompilationUnit;
 
-
 /**
  * Executes the Metadata tests.
  */
@@ -73,10 +67,9 @@
         outdir.mkdirs();
 
         File log = new File("asterix_logs");
-        if (log.exists())
+        if (log.exists()) {
             FileUtils.deleteDirectory(log);
-        File lsn = new File("last_checkpoint_lsn");
-        lsn.deleteOnExit();
+        }
 
         AsterixHyracksIntegrationUtil.init();
 
@@ -97,10 +90,9 @@
         }
 
         File log = new File("asterix_logs");
-        if (log.exists())
+        if (log.exists()) {
             FileUtils.deleteDirectory(log);
-        File lsn = new File("last_checkpoint_lsn");
-        lsn.deleteOnExit();
+        }
     }
 
     @Parameters
@@ -166,7 +158,7 @@
         }
     }
 
-   // To execute Update statements
+    // To execute Update statements
     // Insert and Delete statements are executed here
     public void executeUpdate(String str) throws Exception {
         final String url = "http://localhost:19101/update";
@@ -193,8 +185,8 @@
             System.err.println("Method failed: " + method.getStatusLine());
         }
     }
- 
-   // Executes Query and returns results as JSONArray
+
+    // Executes Query and returns results as JSONArray
     public JSONObject executeQuery(String str) throws Exception {
 
         final String url = "http://localhost:19101/query";
@@ -231,8 +223,8 @@
         }
         return result;
     }
- 
-   @Test
+
+    @Test
     public void test() throws Exception {
         List<TestFileContext> testFileCtxs;
         List<TestFileContext> expectedResultFileCtxs;
@@ -276,7 +268,7 @@
                             }
                             queryCount++;
                             break;
-                       default:
+                        default:
                             throw new IllegalArgumentException("No statements of type " + ctx.getType());
                     }
                 } catch (Exception e) {
@@ -289,5 +281,5 @@
             }
         }
     }
-     
+
 }
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java
index 10cba14..8530ba1 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/optimizer/OptimizerTest.java
@@ -10,6 +10,7 @@
 import java.util.Collection;
 import java.util.logging.Logger;
 
+import org.apache.commons.io.FileUtils;
 import org.junit.AfterClass;
 import org.junit.Assume;
 import org.junit.BeforeClass;
@@ -57,6 +58,12 @@
         System.setProperty(GlobalConfig.WEB_SERVER_PORT_PROPERTY, "19002");
         File outdir = new File(PATH_ACTUAL);
         outdir.mkdirs();
+        
+        File log = new File("asterix_logs");
+        if (log.exists()) {
+            FileUtils.deleteDirectory(log); 
+        }
+        
         AsterixHyracksIntegrationUtil.init();
         // Set the node resolver to be the identity resolver that expects node names 
         // to be node controller ids; a valid assumption in test environment. 
@@ -72,6 +79,11 @@
         if (files == null || files.length == 0) {
             outdir.delete();
         }
+        
+        File log = new File("asterix_logs");
+        if (log.exists()) {
+            FileUtils.deleteDirectory(log); 
+        }
     }
 
     private static void suiteBuild(File dir, Collection<Object[]> testArgs, String path) {
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
index dc7896c..9a7e1d0 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
@@ -56,10 +56,9 @@
         outdir.mkdirs();
 
         File log = new File("asterix_logs");
-        if (log.exists())
+        if (log.exists()) {
             FileUtils.deleteDirectory(log);
-        File lsn = new File("last_checkpoint_lsn");
-        lsn.deleteOnExit();
+        }
 
         AsterixHyracksIntegrationUtil.init();
 
@@ -87,10 +86,9 @@
         }
 
         File log = new File("asterix_logs");
-        if (log.exists())
+        if (log.exists()) {
             FileUtils.deleteDirectory(log);
-        File lsn = new File("last_checkpoint_lsn");
-        lsn.deleteOnExit();
+        }
         HDFSCluster.getInstance().cleanup();
     }
 
@@ -233,6 +231,8 @@
 
         List<CompilationUnit> cUnits = tcCtx.getTestCase().getCompilationUnit();
         for (CompilationUnit cUnit : cUnits) {
+            LOGGER.info("[TEST]: " + tcCtx.getTestCase().getFilePath() + "/" + cUnit.getName());
+
             testFileCtxs = tcCtx.getTestFiles(cUnit);
             expectedResultFileCtxs = tcCtx.getExpectedResultFiles(cUnit);
 
@@ -267,8 +267,6 @@
                             throw new IllegalArgumentException("No statements of type " + ctx.getType());
                     }
                 } catch (Exception e) {
-                    LOGGER.severe("Test \"" + testFile + "\" FAILED!");
-                    e.printStackTrace();
                     if (cUnit.getExpectedError().isEmpty()) {
                         throw new Exception("Test \"" + testFile + "\" FAILED!", e);
                     }
diff --git a/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt b/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt
index 5359017..c9ef6ee 100644
--- a/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt
+++ b/asterix-app/src/test/resources/metadata-transactions/init-state-queries.txt
@@ -1 +1 @@
-customers_orders.aql
\ No newline at end of file
+customers_orders.aql
diff --git a/asterix-app/src/test/resources/optimizerts/ignore.txt b/asterix-app/src/test/resources/optimizerts/ignore.txt
index 5e09322..babe741 100644
--- a/asterix-app/src/test/resources/optimizerts/ignore.txt
+++ b/asterix-app/src/test/resources/optimizerts/ignore.txt
@@ -1,2 +1 @@
-distinct_aggregate.aql
-cell-aggregation-with-filtering.aql
\ No newline at end of file
+distinct_aggregate.aql
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-27.aql b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-27.aql
new file mode 100644
index 0000000..e9ccb87
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-27.aql
@@ -0,0 +1,24 @@
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the primary BTree index is used 
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 5th Feb 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/btree-index_btree-primary-27.adm";
+
+create type TestType as open {
+    fname : string,
+    lname : string
+}
+
+create dataset testdst(TestType) primary key fname,lname;
+
+for $emp in dataset('testdst') 
+where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname < "Tomes"
+return $emp
diff --git a/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-28.aql b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-28.aql
new file mode 100644
index 0000000..5e7b4e3
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-28.aql
@@ -0,0 +1,24 @@
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the primary BTree index is used 
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 5th Feb 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/btree-index_btree-primary-28.adm";
+
+create type TestType as open {
+    fname : string,
+    lname : string
+}
+
+create dataset testdst(TestType)  primary key fname,lname;
+
+for $emp in dataset('testdst') 
+where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname < "Mary" and $emp.lname <= "Tomes"
+return $emp
diff --git a/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-29.aql b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-29.aql
new file mode 100644
index 0000000..c7d4753
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-29.aql
@@ -0,0 +1,24 @@
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the primary BTree index is used 
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 5th Feb 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/btree-index_btree-primary-29.adm";
+
+create type TestType as open {
+    fname : string,
+    lname : string
+}
+
+create dataset testdst(TestType)  primary key fname,lname;
+
+for $emp in dataset('testdst') 
+where $emp.fname > "Craig" and $emp.lname >= "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
+return $emp
diff --git a/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-30.aql b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-30.aql
new file mode 100644
index 0000000..a9a2573
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-primary-30.aql
@@ -0,0 +1,24 @@
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the primary BTree index is used 
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 5th Feb 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/btree-index_btree-primary-30.adm";
+
+create type TestType as open {
+    fname : string,
+    lname : string
+}
+
+create dataset testdst(TestType)  primary key fname,lname;
+
+for $emp in dataset('testdst') 
+where $emp.fname >= "Craig" and $emp.lname > "Kevin" and $emp.fname < "Mary" and $emp.lname < "Tomes"
+return $emp
diff --git a/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
new file mode 100644
index 0000000..2b0a300
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-59.aql
@@ -0,0 +1,28 @@
+/*
+ *  Description     : BTree Index verification test
+ *                  : This test is intended to verify that the secondary BTree index is used 
+ *                  : in the optimized query plan.
+ *  Expected Result : Success
+ *  Date            : 13th Aug 2012
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+write output to nc1:"rttest/btree-index_btree-secondary-49.adm";
+
+create type TestType as open {
+    id : int32,
+    fname : string,
+    lname : string
+}
+
+// create internal dataset
+create dataset testdst(TestType)  primary key id;
+
+create index sec_Idx on testdst(fname,lname);
+
+for $emp in dataset('testdst') 
+where $emp.fname > "Craig" and $emp.lname > "Kevin" and $emp.fname <= "Mary" and $emp.lname < "Tomes"
+return $emp
diff --git a/asterix-app/src/test/resources/optimizerts/queries/cell-aggregation-with-filtering.aql b/asterix-app/src/test/resources/optimizerts/queries/cell-aggregation-with-filtering.aql
deleted file mode 100644
index c883645..0000000
--- a/asterix-app/src/test/resources/optimizerts/queries/cell-aggregation-with-filtering.aql
+++ /dev/null
@@ -1,37 +0,0 @@
-drop dataverse twitter if exists;
-create dataverse twitter;
-use dataverse twitter;
-
-create type Tweet as closed {
-	id: int32,
-	tweetid: int64,
-	loc: point,
-	time: datetime,
-	text: string
-}
-
-create nodegroup group1 if exists on nc1, nc2;
-
-create dataset TwitterData(Tweet)
-  primary key id on group1;
-
-
-load dataset TwitterData 
-using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
-(("path"="nc1://data/twitter/extrasmalltweets.txt"),("format"="adm")) pre-sorted;
-
-create index rtree_index_point on TwitterData(loc) type rtree;
-
-write output to nc1:"rttest/spatial_cell-aggregation-with-filtering.adm";
-
-for $t in dataset('TwitterData')
-let $region := polygon("
-	33.80503407287759,-126.41235263538363 
-	44.9090773200516,-126.41235263538363 
-	44.9090773200516,-87.65258701038363 
-	33.80503407287759,-87.65258701038363")
-
-where spatial-intersect($t.loc, $region)
-group by $c := spatial-cell($t.loc, create-point(24.5,-125.5), 3.0, 3.0) with $t
-order by count($t)
-return { "cell": $c, "count": count($t) }
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
index ea43d41..be64ee5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
@@ -2,21 +2,22 @@
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- STREAM_PROJECT  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- STREAM_PROJECT  |PARTITIONED|
-                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- BTREE_SEARCH  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- ASSIGN  |PARTITIONED|
-                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
index be2e54a..fac45f2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
@@ -3,18 +3,20 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
index be2e54a..fac45f2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
@@ -3,18 +3,20 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
index be2e54a..fac45f2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
@@ -3,18 +3,20 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                -- BTREE_SEARCH  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
+                    -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-27.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-27.plan
new file mode 100644
index 0000000..40853fa
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-27.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-28.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-28.plan
new file mode 100644
index 0000000..5021595
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-28.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-29.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-29.plan
new file mode 100644
index 0000000..5021595
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-29.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-30.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-30.plan
new file mode 100644
index 0000000..5021595
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-30.plan
@@ -0,0 +1,10 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- BTREE_SEARCH  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
index 42779bf..2110116 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
@@ -1,14 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
index 396b992..7087a22 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
@@ -1,14 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
index 396b992..7087a22 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
@@ -1,14 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
index 396b992..7087a22 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
@@ -1,14 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
index 396b992..7087a22 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
@@ -1,14 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
index 396b992..7087a22 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
@@ -1,14 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
index a716ad5..a48d23a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
@@ -1,14 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
index a716ad5..a48d23a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
@@ -1,14 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
index a716ad5..a48d23a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
@@ -1,14 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
index a716ad5..a48d23a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
@@ -1,14 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$8(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
index 5c71405..f18d0ad 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
@@ -1,14 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
index 5c71405..f18d0ad 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
@@ -1,14 +1,17 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
index 6eed3ae..5b0ae2a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
@@ -1,14 +1,15 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-        -- BTREE_SEARCH  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- ASSIGN  |PARTITIONED|
-                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+    -- STREAM_SELECT  |PARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- BTREE_SEARCH  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$9(ASC)]  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
new file mode 100644
index 0000000..7087a22
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan b/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
index 096400e..089064b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
@@ -2,15 +2,18 @@
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
index 487cb63..66b693c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
@@ -2,15 +2,18 @@
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
index 487cb63..66b693c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
@@ -2,15 +2,18 @@
   -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
-        -- STREAM_PROJECT  |PARTITIONED|
-          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-            -- BTREE_SEARCH  |PARTITIONED|
-              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- STREAM_PROJECT  |PARTITIONED|
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- BTREE_SEARCH  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- BTREE_SEARCH  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
index 395fec7..e969e5d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-all.plan
@@ -1,4 +1,4 @@
--- SINK  |PARTITIONED|
+-- COMMIT  |PARTITIONED|
   -- STREAM_PROJECT  |PARTITIONED|
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INSERT_DELETE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index 79daefd..bf8978b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -1,4 +1,4 @@
--- SINK  |PARTITIONED|
+-- COMMIT  |PARTITIONED|
   -- STREAM_PROJECT  |PARTITIONED|
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INDEX_INSERT_DELETE  |PARTITIONED|
@@ -22,4 +22,4 @@
                                           -- BTREE_SEARCH  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
index 0e7cf78..0608f69 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete.plan
@@ -1,4 +1,4 @@
--- SINK  |PARTITIONED|
+-- COMMIT  |PARTITIONED|
   -- STREAM_PROJECT  |PARTITIONED|
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INSERT_DELETE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
index 650f512..e6cf237 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
@@ -1,4 +1,4 @@
--- SINK  |PARTITIONED|
+-- COMMIT  |PARTITIONED|
   -- STREAM_PROJECT  |PARTITIONED|
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INDEX_INSERT_DELETE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
index 5cfcdf3..0b756d9 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert.plan
@@ -1,4 +1,4 @@
--- SINK  |PARTITIONED|
+-- COMMIT  |PARTITIONED|
   -- STREAM_PROJECT  |PARTITIONED|
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INSERT_DELETE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/ignore.txt b/asterix-app/src/test/resources/runtimets/ignore.txt
new file mode 100644
index 0000000..6722cdb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/ignore.txt
@@ -0,0 +1,44 @@
+custord/join_q_04.aql
+scan/spatial_types_02.aql
+scan/temp_types_02.aql
+fuzzyjoin/dblp-splits-3_1.aql
+fuzzyjoin/events-users-aqlplus_1.aql
+subset-collection/04.aql
+custord/freq-clerk.aql
+custord/denorm-cust-order_01.aql
+custord/denorm-cust-order_03.aql
+custord/co.aql
+comparison/numeric-comparison_01.aql
+dapd/q3.aql
+failure/q1_pricing_summary_report_failure.aql
+dml/load-from-hdfs.aql
+open-closed/open-closed-15
+open-closed/open-closed-16
+open-closed/open-closed-17
+open-closed/open-closed-18
+open-closed/open-closed-19
+open-closed/open-closed-20
+open-closed/open-closed-21
+open-closed/open-closed-22
+open-closed/open-closed-28
+open-closed/open-closed-30
+open-closed/heterog-list02
+open-closed/heterog-list03
+open-closed/c2c
+quantifiers/somesat_03.aql
+quantifiers/somesat_04.aql
+quantifiers/somesat_05.aql
+quantifiers/everysat_02.aql
+quantifiers/everysat_03.aql
+flwor
+string/startwith03.aql
+aggregate/droptype.aql
+failure/delete-rtree.aql
+failure/delete.aql
+failure/insert-rtree.aql
+failure/insert.aql
+failure/q1_pricing_summary_report_failure.aql
+failure/verify_delete-rtree.aql
+failure/verify_delete.aql
+failure/verify_insert-rtree.aql
+failure/verify_insert.aql
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv20.aql b/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv20.aql
deleted file mode 100644
index beebca4..0000000
--- a/asterix-app/src/test/resources/runtimets/queries/cross-dataverse/cross-dv20.aql
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Description  : Test various syntax for dataset access
- *              : Using parentheses for dataset access is now optional
- *              : New syntax can use fully qualified names to access datasets.
- * Expected Res : Success
- * Date         : 6th March 2013
- */
-
-drop dataverse student if exists;
-drop dataverse teacher if exists;
-
-create dataverse student;
-create dataverse teacher;
-
-write output to nc1:"rttest/cross-dataverse_cross-dv20.adm";
-
-create type student.stdType as open {
-id : int32,
-name : string,
-age : int32,
-sex : string,
-dept : string
-}
-
-create type teacher.tchrType as open {
-id : int32,
-name : string,
-age : int32,
-sex : string,
-dept : string
-}
-
-create dataset student.ugdstd(stdType) primary key id;
-create dataset student.gdstd(stdType) primary key id;
-create dataset teacher.prof(tchrType) primary key id;
-create dataset teacher.pstdoc(tchrType) primary key id;
-
-insert into dataset student.ugdstd({"id":457,"name":"John Doe","age":22,"sex":"M","dept":"Dance"});
-
-insert into dataset student.gdstd({"id":418,"name":"John Smith","age":26,"sex":"M","dept":"Economics"});
-
-insert into dataset teacher.prof({"id":152,"name":"John Meyer","age":42,"sex":"M","dept":"History"});
-
-insert into dataset teacher.pstdoc({"id":259,"name":"Sophia Reece","age":36,"sex":"F","dept":"Anthropology"});
-
-for $s in dataset student.ugdstd
-for $p in dataset('teacher.prof')
-for $a in dataset("student.gdstd")
-for $b in dataset teacher.pstdoc
-return {"ug-student":$s,"prof":$p,"grd-student":$a,"postdoc":$b}
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.1.ddl.aql
new file mode 100644
index 0000000..0f548a1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+ * Description  : This test case is to verify the fix for issue288
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=288
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type LineIDType as closed {
+  l_orderkey: int32, 
+  l_linenumber: int32, 
+  l_suppkey: int32
+}
+
+create dataset LineID(LineIDType)
+  primary key l_orderkey, l_linenumber;
+
+create dataset LineID2(LineIDType)
+  primary key l_orderkey, l_linenumber;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.2.update.aql
new file mode 100644
index 0000000..181085f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.2.update.aql
@@ -0,0 +1,25 @@
+/*
+ * Description  : This test case is to verify the fix for issue288
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=288
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+
+use dataverse test;
+
+load dataset LineID 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/tpch0.001/lineitem_0.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+
+insert into dataset LineID (
+let $x:=1
+let $y:=2
+let $z:=3
+return {
+	"l_orderkey": $x,
+	"l_linenumber": $y,
+	"l_suppkey": $z
+}
+);
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.3.ddl.aql
new file mode 100644
index 0000000..c3a2bef
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.3.ddl.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue288
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=288
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+use dataverse test;
+
+create index idx_LineID_1 on LineID(l_linenumber);
+create index idx_LineID_2 on LineID2(l_linenumber);
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.4.update.aql
new file mode 100644
index 0000000..2ae4ba0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.4.update.aql
@@ -0,0 +1,25 @@
+/*
+ * Description  : This test case is to verify the fix for issue288
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=288
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset LineID2
+(
+	for $r in dataset('LineID')
+	return $r
+);
+
+// If we replace the insert statement with this, it will work
+/* insert into dataset LineID2
+(
+	for $r in dataset('LineID')
+	return {
+"l_orderkey": $r.l_orderkey, 
+"l_linenumber": $r.l_linenumber, 
+"l_suppkey": $r.l_suppkey}
+);
+*/ 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.5.query.aql
new file mode 100644
index 0000000..a1196bd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue288/query-issue288.5.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue288
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=288
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+ 
+use dataverse test;
+   
+for $c in dataset('LineID2')
+where $c.l_linenumber=2
+return $c 
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..d00347d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,24 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..1d5d4fd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..a79828c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..7e917b5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id>50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..c1c6247
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index that are built on nullable fields.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..e98b328
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
@@ -0,0 +1,23 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
new file mode 100644
index 0000000..381ce87
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..ede69f6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
new file mode 100644
index 0000000..9c58c7e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id>50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
new file mode 100644
index 0000000..ca070fc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..45570ac
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,24 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..d26c270
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..3747bb1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..eb15450
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id<50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..c1fb278
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..9d04c14
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
@@ -0,0 +1,24 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.2.update.aql
new file mode 100644
index 0000000..a5995f6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..97aef53
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.4.update.aql
new file mode 100644
index 0000000..001e605
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id<50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.5.query.aql
new file mode 100644
index 0000000..7910859
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary fuzzy keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..370ac1c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,24 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..a0c9b77
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..81104ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index ngram_index on DBLP(title) type ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..68fc68b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id>50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..cc3c827
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..e6f184c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.1.ddl.aql
@@ -0,0 +1,23 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.2.update.aql
new file mode 100644
index 0000000..5257ea5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..127c55d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index ngram_index on DBLP(title) type ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.4.update.aql
new file mode 100644
index 0000000..6a87cc9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id>50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.5.query.aql
new file mode 100644
index 0000000..5650a21
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary ngram inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..36b3d31
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,22 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..e66540c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..98ccfea
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+create index keyword_index on DBLP(title) type keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..0116a0a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index that are built on nullable fields.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id<50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..979fb45
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index that are built on nullable fields. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+
+use dataverse test;
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..5332316
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.1.ddl.aql
@@ -0,0 +1,23 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.2.update.aql
new file mode 100644
index 0000000..95fd5e8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..3481942
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.3.ddl.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index keyword_index on DBLP(title) type keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.4.update.aql
new file mode 100644
index 0000000..d480735
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.4.update.aql
@@ -0,0 +1,10 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+delete $o from dataset DBLP where $o.id<50;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.5.query.aql
new file mode 100644
index 0000000..39603e7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-delete-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test deletion from secondary keyword inverted index.
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..21ff64a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..9108ad6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..426654c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
+create index fuzzy_ngram_index1 on DBLP1(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..2fe30df
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.4.update.aql
@@ -0,0 +1,21 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..f61df2a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..af890c4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
new file mode 100644
index 0000000..382a4a6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..8109213
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index fuzzy_ngram_index on DBLP(title) type fuzzy ngram(3);
+create index fuzzy_ngram_index1 on DBLP1(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
new file mode 100644
index 0000000..b829b15
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.4.update.aql
@@ -0,0 +1,21 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
new file mode 100644
index 0000000..324b351
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..7dbf7ee
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..1d8126f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..798622a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
+create index fuzzy_keyword_index1 on DBLP1(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..3234c4e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.4.update.aql
@@ -0,0 +1,20 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..43ad95c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..5dd6f1a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.2.update.aql
new file mode 100644
index 0000000..25e9446
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..f43c5cf
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index fuzzy_keyword_index on DBLP(title) type fuzzy keyword;
+create index fuzzy_keyword_index1 on DBLP1(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.4.update.aql
new file mode 100644
index 0000000..b8a1af1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.4.update.aql
@@ -0,0 +1,20 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.5.query.aql
new file mode 100644
index 0000000..0fe611a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-fuzzy-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary fuzzy keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..5f82843
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..bde4abd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..23a302b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index ngram_index on DBLP(title) type ngram(3);
+create index ngram_index1 on DBLP1(title) type ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..ccef47a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.4.update.aql
@@ -0,0 +1,21 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..ec2c974
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..eee618b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.2.update.aql
new file mode 100644
index 0000000..a22b722
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..d40a27b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index ngram_index on DBLP(title) type ngram(3);
+create index ngram_index1 on DBLP1(title) type ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.4.update.aql
new file mode 100644
index 0000000..3d81edf
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.4.update.aql
@@ -0,0 +1,21 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.5.query.aql
new file mode 100644
index 0000000..bd9eb2b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.5.query.aql
@@ -0,0 +1,13 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-ngram-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary ngram inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.1.ddl.aql
new file mode 100644
index 0000000..bafad5b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string?,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.2.update.aql
new file mode 100644
index 0000000..e3f0c6f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-nulls.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..64d07c0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index keyword_index on DBLP(title) type keyword;
+create index keyword_index1 on DBLP1(title) type keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.4.update.aql
new file mode 100644
index 0000000..b2f265fb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.4.update.aql
@@ -0,0 +1,20 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.5.query.aql
new file mode 100644
index 0000000..10bff06
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index-nullable.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index that are built on nullable fields.  
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.1.ddl.aql
new file mode 100644
index 0000000..642a0c8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.1.ddl.aql
@@ -0,0 +1,25 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset DBLP1(DBLPType) primary key id;
+
+
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.2.update.aql
new file mode 100644
index 0000000..c51f848
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.2.update.aql
@@ -0,0 +1,12 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+load dataset DBLP using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..ac90624
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.3.ddl.aql
@@ -0,0 +1,11 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+create index keyword_index on DBLP(title) type keyword;
+create index keyword_index1 on DBLP1(title) type keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.4.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.4.update.aql
new file mode 100644
index 0000000..57ce169
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.4.update.aql
@@ -0,0 +1,20 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+insert into dataset DBLP1 (
+for $o in dataset('DBLP')
+order by $o.id
+return {
+		"id": $o.id,
+		"dblpid": $o.dblpid,
+		"title": $o.title,
+		"authors": $o.authors,
+		"misc": $o.misc
+	}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.5.query.aql
new file mode 100644
index 0000000..73fc519
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.5.query.aql
@@ -0,0 +1,14 @@
+/* 
+ * Test case Name  : scan-insert-inverted-index-word-secondary-index.aql
+ * Description     : This test is intended to test insertion from secondary keyword inverted index. 
+ * Expected Result : Success
+ * Date            : March 31 2013
+ */
+ 
+use dataverse test;
+
+for $o in dataset('DBLP1')
+let $jacc := similarity-jaccard-check(word-tokens($o.title), word-tokens("Transactions for Cooperative Environments"), 0.5f)
+where $jacc[0]
+return $o
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/delete-rtree.aql b/asterix-app/src/test/resources/runtimets/queries/failure/delete-rtree.aql
new file mode 100644
index 0000000..85616e4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/failure/delete-rtree.aql
@@ -0,0 +1,30 @@
+drop dataverse test if exists;
+  
+create dataverse test;
+use dataverse test;
+
+create type MyRecord as closed {
+	id: int32,
+	tweetid: int64,
+	loc: point,
+	time: datetime,
+	text: string
+}
+
+create dataset MyData(MyRecord)
+  primary key id;
+
+create index rtree_index on MyData(loc) type rtree;
+
+load dataset MyData 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/twitter/smalltweets.txt"),("format"="adm")) pre-sorted;
+
+delete $l from dataset MyData where spatial-intersect($l.loc, create-rectangle(create-point(0.0,-100.0), create-point(55.5,50.0))) die after 1000;
+
+write output to nc1:"rttest/failure_delete-rtree.adm";      
+
+for $o in dataset('MyData')
+where spatial-intersect($o.loc, create-rectangle(create-point(0.0,-100.0), create-point(55.5,50.0)))
+order by $o.id
+return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/delete.aql b/asterix-app/src/test/resources/runtimets/queries/failure/delete.aql
new file mode 100644
index 0000000..125fd99
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/failure/delete.aql
@@ -0,0 +1,38 @@
+drop dataverse test if exists;
+  
+create dataverse test;
+use dataverse test;
+
+create type LineItemType as closed {
+  l_orderkey: int32, 
+  l_partkey: int32, 
+  l_suppkey: int32, 
+  l_linenumber: int32, 
+  l_quantity: int32, 
+  l_extendedprice: double,
+  l_discount: double, 
+  l_tax: double,
+  l_returnflag: string, 
+  l_linestatus: string, 
+  l_shipdate: string,
+  l_commitdate: string, 
+  l_receiptdate: string, 
+  l_shipinstruct: string, 
+  l_shipmode: string, 
+  l_comment: string
+}
+
+create dataset LineItem(LineItemType)
+  primary key l_orderkey, l_linenumber;
+
+load dataset LineItem 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/tpch0.001/lineitem.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+delete $l from dataset LineItem where $l.l_orderkey>=10 die after 1000;
+
+write output to nc1:"rttest/failure_delete.adm";      
+for $c in dataset('LineItem')
+where $c.l_orderkey>=10
+order by $c.l_orderkey, $c.l_linenumber
+return $c 
diff --git a/asterix-app/src/test/resources/runtimets/queries/failure/verify_delete-rtree.aql b/asterix-app/src/test/resources/runtimets/queries/failure/verify_delete-rtree.aql
new file mode 100644
index 0000000..a8d7d37
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/failure/verify_delete-rtree.aql
@@ -0,0 +1,8 @@
+use dataverse test;
+
+write output to nc1:"rttest/failure_verify_delete-rtree.adm";
+
+for $o in dataset('MyData')
+where spatial-intersect($o.loc, create-rectangle(create-point(0.0,-100.0), create-point(55.5,50.0)))
+order by $o.id
+return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.1.ddl.aql
new file mode 100644
index 0000000..754a23c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.1.ddl.aql
@@ -0,0 +1,25 @@
+/*
+ * Description     : Test that BTree index is used in query plan
+ *                 : define the BTree index on a composite key (fname,lanme)
+ *                 : predicate => where $l.fname > "Julio" and $l.lname > "Mattocks" and
+ *					 $l.fname <= "Micco" and $l.lname < "Vangieson" 
+ * Expected Result : Success
+ * Issue           : Issue 174
+ * Date            : 5th Feb, 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as closed {
+id:int32,
+fname:string,
+lname:string,
+age:int32,
+dept:string
+}
+
+create dataset employee(Emp) primary key id;
+
+create index idx_employee_f_l_name on employee(fname,lname);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.2.update.aql
new file mode 100644
index 0000000..ba6c29e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.2.update.aql
@@ -0,0 +1,15 @@
+/*
+ * Description     : Test that BTree index is used in query plan
+ *                 : define the BTree index on a composite key (fname,lanme)
+ *                 : predicate => where $l.fname > "Julio" and $l.lname > "Mattocks" and
+ *					 $l.fname <= "Micco" and $l.lname < "Vangieson" 
+ * Expected Result : Success
+ * Issue           : Issue 174
+ * Date            : 5th Feb, 2013
+ */
+
+use dataverse test;
+
+load dataset employee
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.3.query.aql
new file mode 100644
index 0000000..dddd46f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.3.query.aql
@@ -0,0 +1,16 @@
+/*
+ * Description     : Test that BTree index is used in query plan
+ *                 : define the BTree index on a composite key (fname,lanme)
+ *                 : predicate => where $l.fname > "Julio" and $l.lname > "Mattocks" and
+ *					 $l.fname <= "Micco" and $l.lname < "Vangieson" 
+ * Expected Result : Success
+ * Issue           : Issue 174
+ * Date            : 5th Feb, 2013
+ */
+
+use dataverse test;
+
+for $l in dataset('employee')
+where $l.fname > "Julio" and $l.lname > "Mattocks" and $l.fname <= "Micco" and $l.lname < "Vangieson" 
+order by $l.id
+return $l
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.1.ddl.aql
index 2e951e2..5103c09 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.1.ddl.aql
@@ -21,5 +21,3 @@
 
 create dataset employee(Emp) primary key id;
 
-create index idx_employee_f_l_name on employee(fname,lname);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.3.ddl.aql
new file mode 100644
index 0000000..8a7bafe
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.3.ddl.aql
@@ -0,0 +1,6 @@
+
+use dataverse test;
+
+// create secondary index
+
+create index idx_employee_f_l_name on employee(fname,lname);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-composite-key/btree-index-composite-key.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.1.ddl.aql
index aeb8652c..4ed0993 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.1.ddl.aql
@@ -24,5 +24,3 @@
 
 create dataset Orders(OrderType) primary key o_orderkey;
 
-create index idx_Orders_Custkey on Orders(o_custkey);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.3.ddl.aql
new file mode 100644
index 0000000..2489b15
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.3.ddl.aql
@@ -0,0 +1,6 @@
+
+use dataverse tpch;
+
+// create secondary index on Orders(o_custkey)
+
+create index idx_Orders_Custkey on Orders(o_custkey);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/btree-index-rewrite-multiple/btree-index-rewrite-multiple.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.1.ddl.aql
index 82c835d..80e9b17 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.1.ddl.aql
@@ -19,5 +19,3 @@
 
 create dataset Customers(CustomerType) primary key cid;
 
-create index age_index on Customers(age);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.3.ddl.aql
new file mode 100644
index 0000000..7548774
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.3.ddl.aql
@@ -0,0 +1,6 @@
+
+use dataverse test;
+
+// create secondary index on Customers(age)
+
+create index age_index on Customers(age);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/cust-index-age-nullable/cust-index-age-nullable.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.ddl.aql
new file mode 100644
index 0000000..d9ca825
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.ddl.aql
@@ -0,0 +1,18 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  primary key id on group1;
+
+create index ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.2.update.aql
new file mode 100644
index 0000000..830e08a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.3.query.aql
new file mode 100644
index 0000000..866b045
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.3.query.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+where contains($o.title, "Multimedia")
+order by $o.id
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.ddl.aql
new file mode 100644
index 0000000..7afd29a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.ddl.aql
@@ -0,0 +1,18 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  primary key id on group1;
+
+create index ngram_index on DBLP(authors) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.2.update.aql
new file mode 100644
index 0000000..830e08a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.3.query.aql
new file mode 100644
index 0000000..45ac926
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.3.query.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 5)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.ddl.aql
new file mode 100644
index 0000000..7afd29a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.ddl.aql
@@ -0,0 +1,18 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  primary key id on group1;
+
+create index ngram_index on DBLP(authors) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.2.update.aql
new file mode 100644
index 0000000..830e08a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.3.query.aql
new file mode 100644
index 0000000..0f4c003
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.3.query.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+let $ed := edit-distance-check($o.authors, "Amihay Motro", 1)
+where $ed[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.ddl.aql
new file mode 100644
index 0000000..d9ca825
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.ddl.aql
@@ -0,0 +1,18 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  primary key id on group1;
+
+create index ngram_index on DBLP(title) type fuzzy ngram(3);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.2.update.aql
new file mode 100644
index 0000000..830e08a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql
new file mode 100644
index 0000000..79b65bb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+for $o in dataset('DBLP')
+let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
+where $jacc[0]
+return $o
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.ddl.aql
new file mode 100644
index 0000000..8a57319
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.ddl.aql
@@ -0,0 +1,25 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  primary key cid on group1;
+
+create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.2.update.aql
new file mode 100644
index 0000000..063b172
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.ddl.aql
new file mode 100644
index 0000000..8a57319
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.ddl.aql
@@ -0,0 +1,25 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  primary key cid on group1;
+
+create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.2.update.aql
new file mode 100644
index 0000000..063b172
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.ddl.aql
new file mode 100644
index 0000000..8a57319
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.ddl.aql
@@ -0,0 +1,25 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: [string],
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  primary key cid on group1;
+
+create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.2.update.aql
new file mode 100644
index 0000000..063b172
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k_olist/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.ddl.aql
new file mode 100644
index 0000000..3e2562a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.ddl.aql
@@ -0,0 +1,25 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as closed {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int32? } ]
+}
+
+create nodegroup group1 if not exists on nc1;
+
+create dataset Customers(CustomerType) 
+  primary key cid on group1;
+
+create index interests_index on Customers(interests) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.2.update.aql
new file mode 100644
index 0000000..0deeabc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset Customers 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/semistructured/co1k/customer.adm"),("format"="adm"));
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.ddl.aql
new file mode 100644
index 0000000..326b264
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.ddl.aql
@@ -0,0 +1,18 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  primary key id on group1;
+
+create index keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.2.update.aql
new file mode 100644
index 0000000..830e08a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.ddl.aql
new file mode 100644
index 0000000..326b264
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.ddl.aql
@@ -0,0 +1,18 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create nodegroup group1 if not exists on nc1, nc2;
+
+create dataset DBLP(DBLPType) 
+  primary key id on group1;
+
+create index keyword_index on DBLP(title) type fuzzy keyword;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.2.update.aql
new file mode 100644
index 0000000..830e08a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.2.update.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.3.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.3.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.1.ddl.aql
index 7e7ae8c..49c6b3b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.1.ddl.aql
@@ -15,5 +15,3 @@
 create dataset DBLP(DBLPType) 
   primary key id on group1;
 
-create index ngram_index on DBLP(title) type ngram(3);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.3.ddl.aql
new file mode 100644
index 0000000..4fbadcd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.3.ddl.aql
@@ -0,0 +1,6 @@
+
+use dataverse test;
+
+// create secondary index of type ngram on DBLP(title)
+
+create index ngram_index on DBLP(title) type ngram(3);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-contains/inverted-index-ngram-contains.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.1.ddl.aql
index d2ef8c8..49c6b3b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.1.ddl.aql
@@ -15,5 +15,3 @@
 create dataset DBLP(DBLPType) 
   primary key id on group1;
 
-create index ngram_index on DBLP(authors) type ngram(3);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.3.ddl.aql
new file mode 100644
index 0000000..ad6365d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.3.ddl.aql
@@ -0,0 +1,4 @@
+
+use dataverse test;
+
+create index ngram_index on DBLP(authors) type ngram(3);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance-panic/inverted-index-ngram-edit-distance-panic.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.1.ddl.aql
index d2ef8c8..49c6b3b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.1.ddl.aql
@@ -15,5 +15,3 @@
 create dataset DBLP(DBLPType) 
   primary key id on group1;
 
-create index ngram_index on DBLP(authors) type ngram(3);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.3.ddl.aql
new file mode 100644
index 0000000..2a43a57
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index ngram_index on DBLP(authors) type ngram(3);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-edit-distance/inverted-index-ngram-edit-distance.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.1.ddl.aql
index 7e7ae8c..49c6b3b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.1.ddl.aql
@@ -15,5 +15,3 @@
 create dataset DBLP(DBLPType) 
   primary key id on group1;
 
-create index ngram_index on DBLP(title) type ngram(3);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.3.ddl.aql
new file mode 100644
index 0000000..4209874
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index ngram_index on DBLP(title) type ngram(3);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.1.ddl.aql
index 207c8ec..b2dc701 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.1.ddl.aql
@@ -20,5 +20,3 @@
 create dataset Customers(CustomerType) 
   primary key cid on group1;
 
-create index interests_index on Customers(interests) type keyword;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.3.ddl.aql
new file mode 100644
index 0000000..5356b79
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.3.ddl.aql
@@ -0,0 +1,4 @@
+
+use dataverse test;
+
+create index interests_index on Customers(interests) type keyword;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.4.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.3.query.aql
copy to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance-panic/inverted-index-olist-edit-distance-panic.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.1.ddl.aql
index e5774db..dfc0f4e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.1.ddl.aql
@@ -22,5 +22,3 @@
 create dataset Customers(CustomerType) 
   primary key cid on group1;
 
-create index interests_index on Customers(interests) type keyword;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.3.ddl.aql
new file mode 100644
index 0000000..6d7c0cb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index interests_index on Customers(interests) type keyword;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.4.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.3.query.aql
copy to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-edit-distance/inverted-index-olist-edit-distance.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.1.ddl.aql
index e5774db..dfc0f4e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.1.ddl.aql
@@ -22,5 +22,3 @@
 create dataset Customers(CustomerType) 
   primary key cid on group1;
 
-create index interests_index on Customers(interests) type keyword;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.3.ddl.aql
new file mode 100644
index 0000000..5356b79
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.3.ddl.aql
@@ -0,0 +1,4 @@
+
+use dataverse test;
+
+create index interests_index on Customers(interests) type keyword;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.4.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.3.query.aql
copy to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-olist-jaccard/inverted-index-olist-jaccard.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.1.ddl.aql
index a7177ba..a350a46 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.1.ddl.aql
@@ -22,5 +22,3 @@
 create dataset Customers(CustomerType) 
   primary key cid on group1;
 
-create index interests_index on Customers(interests) type keyword;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.3.ddl.aql
new file mode 100644
index 0000000..5356b79
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.3.ddl.aql
@@ -0,0 +1,4 @@
+
+use dataverse test;
+
+create index interests_index on Customers(interests) type keyword;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.4.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.3.query.aql
copy to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ulist-jaccard/inverted-index-ulist-jaccard.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.1.ddl.aql
index bad04b6..49c6b3b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.1.ddl.aql
@@ -15,5 +15,3 @@
 create dataset DBLP(DBLPType) 
   primary key id on group1;
 
-create index keyword_index on DBLP(title) type keyword;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.3.ddl.aql
new file mode 100644
index 0000000..c8db25b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.3.ddl.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+create index keyword_index on DBLP(title) type keyword;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.4.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.3.query.aql
copy to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-contains/inverted-index-word-contains.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.1.ddl.aql
index bad04b6..49c6b3b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.1.ddl.aql
@@ -15,5 +15,3 @@
 create dataset DBLP(DBLPType) 
   primary key id on group1;
 
-create index keyword_index on DBLP(title) type keyword;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.3.ddl.aql
new file mode 100644
index 0000000..93b64c1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index keyword_index on DBLP(title) type keyword;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.4.query.aql
similarity index 100%
copy from asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.3.query.aql
copy to asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-word-jaccard/inverted-index-word-jaccard.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.1.ddl.aql
index 51fdf24..2cfb39a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.1.ddl.aql
@@ -17,5 +17,3 @@
 create dataset Orders(OrderType)
   primary key o_orderkey;
 
-create index idx_Orders_Custkey on Orders(o_custkey) ;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.3.ddl.aql
new file mode 100644
index 0000000..a92308f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse tpch;
+
+create index idx_Orders_Custkey on Orders(o_custkey) ;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive-open/orders-index-custkey-conjunctive-open.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.1.ddl.aql
index 51fdf24..2cfb39a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.1.ddl.aql
@@ -17,5 +17,3 @@
 create dataset Orders(OrderType)
   primary key o_orderkey;
 
-create index idx_Orders_Custkey on Orders(o_custkey) ;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.3.ddl.aql
new file mode 100644
index 0000000..ed3fab0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse tpch;
+
+create index idx_Orders_Custkey on Orders(o_custkey);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-conjunctive/orders-index-custkey-conjunctive.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.1.ddl.aql
index 4d38a74..b541c58 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.1.ddl.aql
@@ -20,5 +20,3 @@
 create dataset Orders(OrderType)
   primary key o_orderkey on group1;
 
-create index idx_Orders_Custkey on Orders(o_custkey);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.3.ddl.aql
new file mode 100644
index 0000000..ed3fab0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse tpch;
+
+create index idx_Orders_Custkey on Orders(o_custkey);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey-open/orders-index-custkey-open.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.1.ddl.aql
index 8b12616..2cfb39a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.1.ddl.aql
@@ -17,5 +17,3 @@
 create dataset Orders(OrderType)
   primary key o_orderkey;
 
-create index idx_Orders_Custkey on Orders(o_custkey);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.3.ddl.aql
new file mode 100644
index 0000000..ed3fab0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse tpch;
+
+create index idx_Orders_Custkey on Orders(o_custkey);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/orders-index-custkey/orders-index-custkey.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.1.ddl.aql
index 265e110..530d7a9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.1.ddl.aql
@@ -25,6 +25,3 @@
 create dataset LineItem(LineItemType)
   primary key l_orderkey, l_linenumber;
 
-create index idx_LineItem_partkey on LineItem(l_linenumber);
-create index idx_LineItem_suppkey on LineItem(l_suppkey);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.3.ddl.aql
new file mode 100644
index 0000000..a3572c4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.3.ddl.aql
@@ -0,0 +1,5 @@
+use dataverse test;
+
+create index idx_LineItem_partkey on LineItem(l_linenumber);
+create index idx_LineItem_suppkey on LineItem(l_suppkey);
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/range-search-open/range-search-open.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.1.ddl.aql
index 89eeec3..5281e04 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.1.ddl.aql
@@ -25,7 +25,3 @@
 create dataset LineItem(LineItemType)
   primary key l_orderkey, l_linenumber;
 
-
-create index idx_LineItem_partkey on LineItem(l_linenumber);
-create index idx_LineItem_suppkey on LineItem(l_suppkey);
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.3.ddl.aql
new file mode 100644
index 0000000..75edda0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.3.ddl.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+create index idx_LineItem_partkey on LineItem(l_linenumber);
+create index idx_LineItem_suppkey on LineItem(l_suppkey);
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/range-search/range-search.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.1.ddl.aql
index 088317c..77b9ef2 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.1.ddl.aql
@@ -16,5 +16,3 @@
 create dataset MyData(MyRecord)
   primary key id;
 
-create index rtree_index_point on MyData(point) type rtree;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.3.ddl.aql
new file mode 100644
index 0000000..b6a123e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index rtree_index_point on MyData(point) type rtree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-nullable/rtree-secondary-index-nullable.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.1.ddl.aql
index be8978c..9de9b5e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.1.ddl.aql
@@ -16,5 +16,3 @@
 create dataset MyData(MyRecord)
   primary key id;
 
-create index rtree_index_point on MyData(point) type rtree;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.3.ddl.aql
new file mode 100644
index 0000000..b6a123e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index rtree_index_point on MyData(point) type rtree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index-open/rtree-secondary-index-open.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.1.ddl.aql
index 41973e1..52fa76f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.1.ddl.aql
@@ -17,5 +17,3 @@
 create dataset MyData(MyRecord)
   primary key id;
 
-create index rtree_index_point on MyData(point) type rtree;
-
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.3.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.3.ddl.aql
new file mode 100644
index 0000000..b6a123e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.3.ddl.aql
@@ -0,0 +1,3 @@
+use dataverse test;
+
+create index rtree_index_point on MyData(point) type rtree;
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.4.query.aql
similarity index 100%
rename from asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.3.query.aql
rename to asterix-app/src/test/resources/runtimets/queries/index-selection/rtree-secondary-index/rtree-secondary-index.4.query.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.1.ddl.aql
new file mode 100644
index 0000000..230315e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.1.ddl.aql
@@ -0,0 +1,31 @@
+/*
+ * Description    : Left-outer joins two datasets, DBLP and CSX, based on their title.
+ *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.2.update.aql
new file mode 100644
index 0000000..923f6e4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.2.update.aql
@@ -0,0 +1,17 @@
+/*
+ * Description    : Left-outer joins two datasets, DBLP and CSX, based on their title.
+ *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.3.query.aql
new file mode 100644
index 0000000..edcb9b2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285-2/query_issue285-2.3.query.aql
@@ -0,0 +1,25 @@
+/*
+ * Description    : Left-outer joins two datasets, DBLP and CSX, based on their title.
+ *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *
+ *					TODO(@Sattam): given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *					
+ *					regression test 2 for issue 285--having an order by and limit for the outer loop relation
+ *
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+for $a in dataset('DBLP')
+order by $a.id
+limit 10
+return {
+"aid": $a.id,
+"bids": for $b in dataset('CSX')
+where $a.authors = $b.authors
+order by $b.id
+return $b.id
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.1.ddl.aql
new file mode 100644
index 0000000..a04adc8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.1.ddl.aql
@@ -0,0 +1,35 @@
+/*
+ * Description    : Left-outer joins two datasets, DBLP and CSX, based on their title.
+ *                  DBLP has a secondary btree index on title.
+ *					
+ *					TODO(@Sattam): given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *					
+ *					regression test for issue 285
+ * Success        : Yes
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use dataverse test;
+
+create type DBLPType as closed {
+  id: int32, 
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create type CSXType as closed {
+  id: int32, 
+  csxid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset DBLP(DBLPType) primary key id;
+create dataset CSX(CSXType) primary key id;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.2.update.aql
new file mode 100644
index 0000000..923f6e4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.2.update.aql
@@ -0,0 +1,17 @@
+/*
+ * Description    : Left-outer joins two datasets, DBLP and CSX, based on their title.
+ *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+load dataset DBLP 
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset CSX
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/pub-small/csx-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.3.query.aql
new file mode 100644
index 0000000..f5ff114
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/leftouterjoin/query_issue285/query_issue285.3.query.aql
@@ -0,0 +1,23 @@
+/*
+ * Description    : Left-outer joins two datasets, DBLP and CSX, based on their title.
+ *                  DBLP has a secondary btree index on title, and given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *
+ *					TODO(@Sattam): given the 'indexnl' hint 
+ *                  we expect the join to be transformed into an indexed nested-loop join.
+ *					
+ *					regression test for issue 285--having an order by for the outer loop relation
+ * Success        : Yes
+ */
+
+use dataverse test;
+
+for $a in dataset('DBLP')
+order by $a.id
+return {
+"aid": $a.id,
+"bids": for $b in dataset('CSX')
+where $a.authors = $b.authors
+order by $b.id
+return $b.id
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.1.ddl.aql
new file mode 100644
index 0000000..b9683dd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.1.ddl.aql
@@ -0,0 +1,29 @@
+/*
+ * Description  : Load dataset with float numbers containing "E-4f"
+ * Expected Res : Success
+ * Date         : 01 Apr 2013
+ */
+ 
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32, 
+  street: string,
+  city: string
+}
+
+create type CustomerType as closed {
+  cid: int32, 
+  name: string,
+  age: int32?,
+  address: AddressType?,
+  lastorder: {
+    oid: int32,
+    total: float
+  }
+}
+
+create dataset Customers(CustomerType) 
+primary key cid;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
new file mode 100644
index 0000000..f0671ca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.2.update.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : Load dataset with float numbers containing "E-4f"
+ * Expected Res : Success
+ * Date         : 01 Apr 2013
+ */
+ 
+use dataverse test;
+
+load dataset Customers
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/custord-tiny/customer-tiny-neg.adm"),("format"="adm")) pre-sorted;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.3.query.aql
new file mode 100644
index 0000000..1d2c9b6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/load/issue289_query/issue289_query.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : Load dataset with float numbers containing "E-4f"
+ * Expected Res : Success
+ * Date         : 01 Apr 2013
+ */
+ 
+use dataverse test;
+
+count(
+for $l in dataset('Customers')
+return $l
+)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even20/round-half-to-even20.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even20/round-half-to-even20.3.query.aql
index 462de75..d14fb40 100644
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even20/round-half-to-even20.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even20/round-half-to-even20.3.query.aql
@@ -8,5 +8,5 @@
 let $c5 := int16("0")
 let $c6 := int32("0")
 let $c7 := int64("0")
-return {"f0": numeric-round-half-to-even2($c0,2), "f1": numeric-round-half-to-even2($c1,2),"f2": numeric-round-half-to-even2($c2,2), "f3": numeric-round-half-to-even2($c3,2),
-	"f4": numeric-round-half-to-even2($c4,2),"f5": numeric-round-half-to-even2($c5,2) ,"f6": numeric-round-half-to-even2($c6,2), "f7": numeric-round-half-to-even2($c7,2)}
+return {"f0": numeric-round-half-to-even($c0,2), "f1": numeric-round-half-to-even($c1,2),"f2": numeric-round-half-to-even($c2,2), "f3": numeric-round-half-to-even($c3,2),
+	"f4": numeric-round-half-to-even($c4,2),"f5": numeric-round-half-to-even($c5,2) ,"f6": numeric-round-half-to-even($c6,2), "f7": numeric-round-half-to-even($c7,2)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even21/round-half-to-even21.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even21/round-half-to-even21.3.query.aql
index 6847ed7..958a63d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even21/round-half-to-even21.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even21/round-half-to-even21.3.query.aql
@@ -8,5 +8,5 @@
 let $c5 := int16("22")
 let $c6 := int32("23")
 let $c7 := int64("27")
-return {"f0": numeric-round-half-to-even2($c0,2), "f1": numeric-round-half-to-even2($c1,2),"f2": numeric-round-half-to-even2($c2,2), "f3": numeric-round-half-to-even2($c3,2),
-	"f4": numeric-round-half-to-even2($c4,2),"f5": numeric-round-half-to-even2($c5,2) ,"f6": numeric-round-half-to-even2($c6,2), "f7": numeric-round-half-to-even2($c7,2)}
+return {"f0": numeric-round-half-to-even($c0,2), "f1": numeric-round-half-to-even($c1,2),"f2": numeric-round-half-to-even($c2,2), "f3": numeric-round-half-to-even($c3,2),
+	"f4": numeric-round-half-to-even($c4,2),"f5": numeric-round-half-to-even($c5,2) ,"f6": numeric-round-half-to-even($c6,2), "f7": numeric-round-half-to-even($c7,2)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even22/round-half-to-even22.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even22/round-half-to-even22.3.query.aql
index 9338f10..e9aa46c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even22/round-half-to-even22.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even22/round-half-to-even22.3.query.aql
@@ -7,5 +7,5 @@
 let $c4 := float("-INF")
 let $c5 := float("-0.0")
 let $c6 := float("0.0")
-return {"d0": numeric-round-half-to-even2($c0,2), "d1": numeric-round-half-to-even2($c1,2),"d2": numeric-round-half-to-even2($c2,3),
-        "d3": numeric-round-half-to-even2($c3,4),"d4": numeric-round-half-to-even2($c4,5),"d5": numeric-round-half-to-even2($c5,6), "d6": numeric-round-half-to-even2($c6,0)}
+return {"d0": numeric-round-half-to-even($c0,2), "d1": numeric-round-half-to-even($c1,2),"d2": numeric-round-half-to-even($c2,3),
+        "d3": numeric-round-half-to-even($c3,4),"d4": numeric-round-half-to-even($c4,5),"d5": numeric-round-half-to-even($c5,6), "d6": numeric-round-half-to-even($c6,0)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even23/round-half-to-even23.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even23/round-half-to-even23.3.query.aql
index c7260ab..8fe4615 100644
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even23/round-half-to-even23.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even23/round-half-to-even23.3.query.aql
@@ -7,5 +7,5 @@
 let $c4 := double("-INF")
 let $c5 := double("-0.0")
 let $c6 := double("0.0")
-return {"d0": numeric-round-half-to-even2($c0,2), "d1": numeric-round-half-to-even2($c1,2),"d2": numeric-round-half-to-even2($c2,3),
-        "d3": numeric-round-half-to-even2($c3,4),"d4": numeric-round-half-to-even2($c4,5),"d5": numeric-round-half-to-even2($c5,6), "d6": numeric-round-half-to-even2($c6,0)}
+return {"d0": numeric-round-half-to-even($c0,2), "d1": numeric-round-half-to-even($c1,2),"d2": numeric-round-half-to-even($c2,3),
+        "d3": numeric-round-half-to-even($c3,4),"d4": numeric-round-half-to-even($c4,5),"d5": numeric-round-half-to-even($c5,6), "d6": numeric-round-half-to-even($c6,0)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even24/round-half-to-even24.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even24/round-half-to-even24.3.query.aql
index 8a2d135..9b3095f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even24/round-half-to-even24.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/numeric/round-half-to-even24/round-half-to-even24.3.query.aql
@@ -5,5 +5,5 @@
 let $c2 := double("3.567812E+3")
 let $c3 := double("4.7564E-3")
 let $c4 := double("35612.25")
-return {"d0": numeric-round-half-to-even2($c0,2), "d1": numeric-round-half-to-even2($c1,2),"d2": numeric-round-half-to-even2($c2,2),
-        "d3": numeric-round-half-to-even2($c3,2),"d4": numeric-round-half-to-even2($c4,-2)}
+return {"d0": numeric-round-half-to-even($c0,2), "d1": numeric-round-half-to-even($c1,2),"d2": numeric-round-half-to-even($c2,2),
+        "d3": numeric-round-half-to-even($c3,2),"d4": numeric-round-half-to-even($c4,-2)}
diff --git a/asterix-app/src/test/resources/runtimets/queries/spatial/circle-intersect-circle/circle-intersect-circle.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/spatial/circle-intersect-circle/circle-intersect-circle.3.query.aql
index e318dbc..e762ddf8 100644
--- a/asterix-app/src/test/resources/runtimets/queries/spatial/circle-intersect-circle/circle-intersect-circle.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/spatial/circle-intersect-circle/circle-intersect-circle.3.query.aql
@@ -1,6 +1,6 @@
 use dataverse test;
 
 for $o in dataset('MyData')
-where spatial-intersect(create-circle(create-point(0.0,0.0), 5.0), create-circle(create-point(9.9,0.0), 5.0))
+where spatial-intersect(create-circle(create-point(0.0,0.0), 5.0), $o.circle)
 order by $o.id
 return {"id":$o.id}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches21/matches21.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches21/matches21.3.query.aql
index dff183b..e874fcb 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/matches21/matches21.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches21/matches21.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse test;
 
-let $c1 := matches2("abracadabra","Bra","")
+let $c1 := matches("abracadabra","Bra","")
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches22/matches22.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches22/matches22.3.query.aql
index ac16ae8..7b5c9c0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/matches22/matches22.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches22/matches22.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse test;
 
-let $c1 := matches2("abracadabra","Bra","i")
+let $c1 := matches("abracadabra","Bra","i")
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matches23/matches23.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/matches23/matches23.3.query.aql
index 8289897..9d81aae 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/matches23/matches23.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matches23/matches23.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse test;
 
-let $c1 := matches2("helloworld","hello world","x")
+let $c1 := matches("helloworld","hello world","x")
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/matchesnull/matchesnull.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/matchesnull/matchesnull.3.query.aql
index 5f01b89..6ce6a77 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/matchesnull/matchesnull.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/matchesnull/matchesnull.3.query.aql
@@ -3,7 +3,7 @@
 let $c1 := matches("helloworld",null)
 let $c2 := matches("",null)
 let $c3 := matches(null,null)
-let $c4 := matches2("helloworld",null, "")
-let $c5 := matches2("",null, "i")
-let $c6 := matches2(null,null, null)
+let $c4 := matches("helloworld",null, "")
+let $c5 := matches("",null, "i")
+let $c6 := matches(null,null, null)
 return {"result1": $c1, "result2": $c2, "result3": $c3, "result4": $c4, "result5": $c5, "result6": $c6}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/replace21/replace21.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/replace21/replace21.3.query.aql
index a814373..3552126 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/replace21/replace21.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/replace21/replace21.3.query.aql
@@ -1,6 +1,6 @@
 use dataverse test;
 
-let $c1 := replace2("abracadabra","Bra", "kkk" , "")
-let $c2 := replace2("abracadabra","Bra", "kkk" ,"i")
-let $c3 := replace2("helloworld","hello world", "kkk" , "x")
+let $c1 := replace("abracadabra","Bra", "kkk" , "")
+let $c2 := replace("abracadabra","Bra", "kkk" ,"i")
+let $c3 := replace("helloworld","hello world", "kkk" , "x")
 return {"result1": $c1,"result2": $c2,"result3": $c3}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/replace22/replace22.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/replace22/replace22.3.query.aql
index 1dcbe08..df67481 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/replace22/replace22.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/replace22/replace22.3.query.aql
@@ -1,11 +1,11 @@
 use dataverse test;
 
-let $c1 := replace2("abracadabra","", null , null)
-let $c2 := replace2("abracadabra","bra", "XXX" ,"")
-let $c3 := replace2(null,"hello world", "XxXx" , "x")
-let $c4 := replace2("abracadabra","bra", "XXX" ,null)
-let $c5 := replace2("abracadabra",null, "XXX" ,null)
-let $c6 := replace2("abracadabra","Bra", null ,"i")
-let $c7 := replace2("abracadabra","Bra", "" ,"i")
-let $c8 := replace2("abracadabra","", "XXX" ,"")
+let $c1 := replace("abracadabra","", null , null)
+let $c2 := replace("abracadabra","bra", "XXX" ,"")
+let $c3 := replace(null,"hello world", "XxXx" , "x")
+let $c4 := replace("abracadabra","bra", "XXX" ,null)
+let $c5 := replace("abracadabra",null, "XXX" ,null)
+let $c6 := replace("abracadabra","Bra", null ,"i")
+let $c7 := replace("abracadabra","Bra", "" ,"i")
+let $c8 := replace("abracadabra","", "XXX" ,"")
 return {"result1": $c1,"result2": $c2,"result3": $c3,"result4": $c4,"result5": $c5,"result6": $c6,"result7": $c7,"result8": $c8}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
index 422f87c..e7b638c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr01/substr01.3.query.aql
@@ -6,23 +6,23 @@
  */
 
 let $str1:="Hello World"
-let $str2:=substring2($str1,10)
+let $str2:=substring($str1,10)
 
 let $str3:="This is a test string"
-let $str4:=substring2($str3,21)
+let $str4:=substring($str3,21)
 
 let $str5:="This is a test string"
-let $str6:=substring2($str5,22)
+let $str6:=substring($str5,22)
 
 let $str7:="This is a test string"
-let $str8:=substring2($str7,0)
+let $str8:=substring($str7,0)
 
 let $str9:="This is a test string"
-let $str10:=substring2($str9,-1)
+let $str10:=substring($str9,-1)
 
 let $str11:="This is a test string"
 let $str12:="This is a another test string"
-let $str13:=substring2(string-concat([$str11,$str12]),21)
+let $str13:=substring(string-concat([$str11,$str12]),21)
 
-let $str14:=substring2("UC Irvine",string-length("UC Irvine")/2)
+let $str14:=substring("UC Irvine",string-length("UC Irvine")/2)
 return { "str2":$str2,"str4":$str4,"str6":$str6,"str8":$str8,"str10":$str10,"str13":$str13,"str14":$str14}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
index b00bea5..0f5b1ff 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr04/substr04.3.query.aql
@@ -6,14 +6,14 @@
  */
 
 for $a in [ substring2("hello world",7,11),
-substring2("hello world",1,11),
-substring2("hello world",3,7),
-substring2("ABCD",3,6),
-substring2("ABCD",0,4),
-substring2("UC Irvine",4,string-length("UC Irvine")),
-substring2("UC Irvine",0,string-length("UC Irvine")),
-substring2("UC Irvine",1,string-length("UC Irvine")),
-substring2(substring2("UC Irvine",4),0,string-length("Irvine")),
-substring2(substring2("UC Irvine",4),0,(string-length("Irvine")/2))
+substring("hello world",1,11),
+substring("hello world",3,7),
+substring("ABCD",3,6),
+substring("ABCD",0,4),
+substring("UC Irvine",4,string-length("UC Irvine")),
+substring("UC Irvine",0,string-length("UC Irvine")),
+substring("UC Irvine",1,string-length("UC Irvine")),
+substring(substring("UC Irvine",4),0,string-length("Irvine")),
+substring(substring("UC Irvine",4),0,(string-length("Irvine")/2))
 ]
 return $a
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
index df111b3..1b2cfe5 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr05/substr05.3.query.aql
@@ -11,4 +11,4 @@
 
 for $a in dataset('testdst')
 order by $a.name
-return substring2($a.name,4,string-length($a.name));
+return substring($a.name,4,string-length($a.name));
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
index a2cb9d3..b6be396 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substr06/substr06.3.query.aql
@@ -10,4 +10,4 @@
 
 for $a in dataset('testdst')
 order by $a.name
-return substring2($a.name,4);
+return substring($a.name,4);
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
index 2ef7ba8..6e32225 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-1/substring2-1.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse test;
 
-let $c1 := substring2("HEllow",2)
+let $c1 := substring("HEllow",2)
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-2/substring2-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-2/substring2-2.3.query.aql
index db40e40..854eac6 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substring2-2/substring2-2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-2/substring2-2.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse test;
 
-let $c1 := substring2("HEllow",0)
+let $c1 := substring("HEllow",0)
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
index 77b847c..f8a53f4 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-3/substring2-3.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse test;
 
-let $c1 := substring2("HEllow",10)
+let $c1 := substring("HEllow",10)
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
index 27c5ce2..192698c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/substring2-4/substring2-4.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse test;
 
-let $c1 := substring2("HEllow",-1)
+let $c1 := substring("HEllow",-1)
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/toLowerCase02/toLowerCase02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/toLowerCase02/toLowerCase02.3.query.aql
index 31cb9e3..24ccdcc 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/toLowerCase02/toLowerCase02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/toLowerCase02/toLowerCase02.3.query.aql
@@ -17,6 +17,6 @@
     lowercase("AbCdEfGhIjKlMnOpQrStUvWxYz"),
     lowercase("abcdefghijkABCDEFGHIJK"),
     lowercase("HIJKLMNOPQRhijklmnopqr"),
-    lowercase(substring2("ABCDEFghIJKLMnopQRSTuvwxYZ01234",0)),
+    lowercase(substring("ABCDEFghIJKLMnopQRSTuvwxYZ01234",0)),
     lowercase("A33B2CD1EF78GHijk123LMNopqrstUVW3x2y01035Z")]
 return $a
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv20.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv20.adm
deleted file mode 100644
index 83de609..0000000
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv20.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "ug-student": { "id": 457, "name": "John Doe", "age": 22, "sex": "M", "dept": "Dance" }, "prof": { "id": 152, "name": "John Meyer", "age": 42, "sex": "M", "dept": "History" }, "grd-student": { "id": 418, "name": "John Smith", "age": 26, "sex": "M", "dept": "Economics" }, "postdoc": { "id": 259, "name": "Sophia Reece", "age": 36, "sex": "F", "dept": "Anthropology" } }
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/delete-syntax-change.adm b/asterix-app/src/test/resources/runtimets/results/dml/delete-syntax-change.adm
deleted file mode 100644
index 7039cfe..0000000
--- a/asterix-app/src/test/resources/runtimets/results/dml/delete-syntax-change.adm
+++ /dev/null
@@ -1,25 +0,0 @@
-{ "l_orderkey": 1, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17, "l_extendedprice": 17954.55d, "l_discount": 0.04d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "egular courts above the" }
-{ "l_orderkey": 1, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36, "l_extendedprice": 34850.16d, "l_discount": 0.09d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly final dependencies: slyly bold " }
-{ "l_orderkey": 1, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8, "l_extendedprice": 7712.48d, "l_discount": 0.1d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "riously. regular, express dep" }
-{ "l_orderkey": 1, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28, "l_extendedprice": 25284.0d, "l_discount": 0.09d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lites. fluffily even de" }
-{ "l_orderkey": 1, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24, "l_extendedprice": 22200.48d, "l_discount": 0.1d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending foxes. slyly re" }
-{ "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32, "l_extendedprice": 29312.32d, "l_discount": 0.07d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
-{ "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 38269.8d, "l_discount": 0.0d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
-{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40725.0d, "l_discount": 0.06d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
-{ "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49, "l_extendedprice": 45080.98d, "l_discount": 0.1d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
-{ "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27, "l_extendedprice": 27786.24d, "l_discount": 0.06d, "l_tax": 0.07d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
-{ "l_orderkey": 3, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2, "l_extendedprice": 1860.06d, "l_discount": 0.01d, "l_tax": 0.06d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. fluffily pending d" }
-{ "l_orderkey": 3, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28, "l_extendedprice": 30357.04d, "l_discount": 0.04d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ages nag slyly pending" }
-{ "l_orderkey": 3, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 26, "l_extendedprice": 25039.56d, "l_discount": 0.1d, "l_tax": 0.02d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges sleep after the caref" }
-{ "l_orderkey": 4, "l_partkey": 89, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 30, "l_extendedprice": 29672.4d, "l_discount": 0.03d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-10", "l_commitdate": "1995-12-14", "l_receiptdate": "1996-01-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "- quickly regular packages sleep. idly" }
-{ "l_orderkey": 5, "l_partkey": 109, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 15, "l_extendedprice": 15136.5d, "l_discount": 0.02d, "l_tax": 0.04d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-31", "l_commitdate": "1994-08-31", "l_receiptdate": "1994-11-20", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ts wake furiously " }
-{ "l_orderkey": 5, "l_partkey": 124, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26, "l_extendedprice": 26627.12d, "l_discount": 0.07d, "l_tax": 0.08d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-16", "l_commitdate": "1994-09-25", "l_receiptdate": "1994-10-19", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sts use slyly quickly special instruc" }
-{ "l_orderkey": 5, "l_partkey": 38, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50, "l_extendedprice": 46901.5d, "l_discount": 0.08d, "l_tax": 0.03d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-10-13", "l_receiptdate": "1994-08-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "eodolites. fluffily unusual" }
-{ "l_orderkey": 6, "l_partkey": 140, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 37, "l_extendedprice": 38485.18d, "l_discount": 0.08d, "l_tax": 0.03d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-27", "l_commitdate": "1992-05-15", "l_receiptdate": "1992-05-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "p furiously special foxes" }
-{ "l_orderkey": 7, "l_partkey": 183, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12, "l_extendedprice": 12998.16d, "l_discount": 0.07d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-07", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-06-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ss pinto beans wake against th" }
-{ "l_orderkey": 7, "l_partkey": 146, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9, "l_extendedprice": 9415.26d, "l_discount": 0.08d, "l_tax": 0.08d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-01", "l_commitdate": "1996-03-02", "l_receiptdate": "1996-02-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "es. instructions" }
-{ "l_orderkey": 7, "l_partkey": 95, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 46, "l_extendedprice": 45774.14d, "l_discount": 0.1d, "l_tax": 0.07d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-27", "l_receiptdate": "1996-02-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " unusual reques" }
-{ "l_orderkey": 7, "l_partkey": 164, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 28, "l_extendedprice": 29796.48d, "l_discount": 0.03d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-21", "l_commitdate": "1996-04-08", "l_receiptdate": "1996-04-20", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": ". slyly special requests haggl" }
-{ "l_orderkey": 7, "l_partkey": 152, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 38, "l_extendedprice": 39981.7d, "l_discount": 0.08d, "l_tax": 0.01d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-02-24", "l_receiptdate": "1996-02-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ns haggle carefully ironic deposits. bl" }
-{ "l_orderkey": 7, "l_partkey": 80, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 35, "l_extendedprice": 34302.8d, "l_discount": 0.06d, "l_tax": 0.03d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-16", "l_commitdate": "1996-02-23", "l_receiptdate": "1996-01-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "jole. excuses wake carefully alongside of " }
-{ "l_orderkey": 7, "l_partkey": 158, "l_suppkey": 3, "l_linenumber": 7, "l_quantity": 5, "l_extendedprice": 5290.75d, "l_discount": 0.04d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-10", "l_commitdate": "1996-03-26", "l_receiptdate": "1996-02-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ithely regula" }
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/insert-syntax.adm b/asterix-app/src/test/resources/runtimets/results/dml/insert-syntax.adm
deleted file mode 100644
index 9407868..0000000
--- a/asterix-app/src/test/resources/runtimets/results/dml/insert-syntax.adm
+++ /dev/null
@@ -1,4 +0,0 @@
-{ "id": 1, "name": "Person One", "hobbies": {{ "Rock", "Metal" }} }
-{ "id": 2, "name": "Person Two", "hobbies": {{ "Rock", "Jazz" }} }
-{ "id": 3, "name": "Person Three", "hobbies": {{ "Blues" }} }
-{ "id": 4, "name": "Person Four", "hobbies": {{ "Metal", "Jazz" }} }
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/query-issue288/query-issue288.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/query-issue288/query-issue288.1.adm
new file mode 100644
index 0000000..e6716e8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/query-issue288/query-issue288.1.adm
@@ -0,0 +1 @@
+{ "l_orderkey": 1, "l_linenumber": 2, "l_suppkey": 3 }
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..7810e91
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable.1.adm
@@ -0,0 +1 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.adm
new file mode 100644
index 0000000..7810e91
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-ngram-secondary-index/scan-delete-inverted-index-fuzzy-ngram-secondary-index.1.adm
@@ -0,0 +1 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable/scan-delete-inverted-index-fuzzy-word-secondary-index-nullable.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-fuzzy-word-secondary-index/scan-delete-inverted-index-fuzzy-word-secondary-index.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..7810e91
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-ngram-secondary-index-nullable/scan-delete-inverted-index-ngram-secondary-index-nullable.1.adm
@@ -0,0 +1 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.1.adm
new file mode 100644
index 0000000..7810e91
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-ngram-secondary-index/scan-delete-inverted-index-ngram-secondary-index.1.adm
@@ -0,0 +1 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-word-secondary-index-nullable/scan-delete-inverted-index-word-secondary-index-nullable.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-delete-inverted-index-word-secondary-index/scan-delete-inverted-index-word-secondary-index.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable/scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable.1.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-ngram-secondary-index/scan-insert-inverted-index-fuzzy-ngram-secondary-index.1.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable/scan-insert-inverted-index-fuzzy-word-secondary-index-nullable.1.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-fuzzy-word-secondary-index/scan-insert-inverted-index-fuzzy-word-secondary-index.1.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-ngram-secondary-index-nullable/scan-insert-inverted-index-ngram-secondary-index-nullable.1.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.1.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-ngram-secondary-index/scan-insert-inverted-index-ngram-secondary-index.1.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.1.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-word-secondary-index-nullable/scan-insert-inverted-index-word-secondary-index-nullable.1.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.1.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/scan-insert-inverted-index-word-secondary-index/scan-insert-inverted-index-word-secondary-index.1.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.1.adm
new file mode 100644
index 0000000..590b9ff
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/btree-index-composite-key-mixed-intervals/btree-index-composite-key-mixed-intervals.1.adm
@@ -0,0 +1,11 @@
+{ "id": 215, "fname": "Karina", "lname": "Michelsen", "age": 46, "dept": "IT" }
+{ "id": 219, "fname": "Kurt", "lname": "Petermann", "age": 27, "dept": "Payroll" }
+{ "id": 463, "fname": "Marcie", "lname": "States", "age": 28, "dept": "IT" }
+{ "id": 589, "fname": "Lorrie", "lname": "Sharon", "age": 27, "dept": "IT" }
+{ "id": 681, "fname": "Max", "lname": "Teachout", "age": 34, "dept": "IT" }
+{ "id": 781, "fname": "Karina", "lname": "Tuthill", "age": 46, "dept": "Payroll" }
+{ "id": 955, "fname": "Max", "lname": "Mell", "age": 33, "dept": "HR" }
+{ "id": 965, "fname": "Micco", "lname": "Mercy", "age": 31, "dept": "Payroll" }
+{ "id": 1426, "fname": "Marcie", "lname": "Rasnake", "age": 42, "dept": "Sales" }
+{ "id": 5438, "fname": "Lakisha", "lname": "Quashie", "age": 29, "dept": "HR" }
+{ "id": 9941, "fname": "Khurram Faraaz", "lname": "Mohammed", "age": 30, "dept": "HR" }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-contains/fuzzy-inverted-index-ngram-contains.1.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.adm
new file mode 100644
index 0000000..a218d95
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance-panic/fuzzy-inverted-index-ngram-edit-distance-panic.1.adm
@@ -0,0 +1 @@
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.adm
new file mode 100644
index 0000000..a218d95
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-edit-distance/fuzzy-inverted-index-ngram-edit-distance.1.adm
@@ -0,0 +1 @@
+{ "id": 22, "dblpid": "books/acm/kim95/Motro95", "title": "Management of Uncerainty in database Systems.", "authors": "Amihai Motro", "misc": "2002-01-03 457-476 1995 Modern Database Systems db/books/collections/kim95.html#Motro95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.1.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.adm
new file mode 100644
index 0000000..9e33b16
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance-panic/fuzzy-inverted-index-olist-edit-distance-panic.1.adm
@@ -0,0 +1,854 @@
+{ "cid": 1, "name": "Trudie Minick", "age": 75, "address": { "number": 6740, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Fishing", "Squash" ], "children": [ { "name": "Arie Minick", "age": 56 }, { "name": "Alline Minick", "age": 57 }, { "name": "Petronila Minick", "age": 56 } ] }
+{ "cid": 2, "name": "Elin Debell", "age": 82, "address": { "number": 5649, "street": "Hill St.", "city": "Portland" }, "interests": [ "Bass", "Wine" ], "children": [ { "name": "Elvina Debell", "age": null }, { "name": "Renaldo Debell", "age": 51 }, { "name": "Divina Debell", "age": 57 } ] }
+{ "cid": 3, "name": "Phung Wheetley", "age": 12, "address": { "number": 5549, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Raelene Wheetley", "age": null }, { "name": "Dudley Wheetley", "age": null } ] }
+{ "cid": 4, "name": "Bernita Gungor", "age": 87, "address": { "number": 1208, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [ { "name": "Valencia Gungor", "age": 72 }, { "name": "Evangeline Gungor", "age": 76 }, { "name": "Odell Gungor", "age": null }, { "name": "Denny Gungor", "age": null } ] }
+{ "cid": 5, "name": "Heide Naifeh", "age": null, "address": null, "interests": [ "Music", "Databases" ], "children": [ { "name": "Deirdre Naifeh", "age": null }, { "name": "Jacquelyne Naifeh", "age": 39 } ] }
+{ "cid": 6, "name": "Cris Kager", "age": 70, "address": { "number": 8402, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Walking" ], "children": [ { "name": "Carmelo Kager", "age": 34 }, { "name": "Faustina Kager", "age": null } ] }
+{ "cid": 7, "name": "Karie Kaehler", "age": 59, "address": { "number": 9875, "street": "View St.", "city": "San Jose" }, "interests": [ "Computers", "Skiing", "Basketball", "Movies" ], "children": [ { "name": "Spring Kaehler", "age": 17 } ] }
+{ "cid": 8, "name": "Audria Haylett", "age": 44, "address": { "number": 4872, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cooking", "Fishing", "Video Games" ], "children": [ { "name": "Lacie Haylett", "age": 19 } ] }
+{ "cid": 9, "name": "Dreama Nuccio", "age": 55, "address": { "number": 95, "street": "Main St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Ricardo Nuccio", "age": 28 }, { "name": "See Nuccio", "age": 34 } ] }
+{ "cid": 10, "name": "Trent Liedy", "age": 51, "address": { "number": 1758, "street": "Oak St.", "city": "San Jose" }, "interests": [  ], "children": [  ] }
+{ "cid": 11, "name": "Meta Simek", "age": 13, "address": { "number": 4384, "street": "7th St.", "city": "San Jose" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Oretha Simek", "age": null }, { "name": "Terence Simek", "age": null } ] }
+{ "cid": 12, "name": "Laurinda Raimann", "age": null, "address": null, "interests": [ "Basketball", "Coffee" ], "children": [ { "name": "Lulu Raimann", "age": null }, { "name": "Refugia Raimann", "age": 19 }, { "name": "Jimmie Raimann", "age": 10 }, { "name": "Cindy Raimann", "age": null } ] }
+{ "cid": 13, "name": "Nicol Kolmer", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Erika Kolmer", "age": 40 }, { "name": "Justin Kolmer", "age": null }, { "name": "Dorathy Kolmer", "age": null }, { "name": "Anastacia Kolmer", "age": 27 } ] }
+{ "cid": 14, "name": "Chance Nicoson", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Willette Nicoson", "age": 39 }, { "name": "Glennis Nicoson", "age": null }, { "name": "Philip Nicoson", "age": null }, { "name": "Cody Nicoson", "age": 26 } ] }
+{ "cid": 15, "name": "Berry Faubel", "age": 55, "address": { "number": 2806, "street": "Oak St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Tiffiny Faubel", "age": 12 }, { "name": "Hilaria Faubel", "age": 19 }, { "name": "Wesley Faubel", "age": 37 }, { "name": "Wei Faubel", "age": 28 } ] }
+{ "cid": 16, "name": "Felisa Auletta", "age": 55, "address": { "number": 7737, "street": "View St.", "city": "San Jose" }, "interests": [ "Skiing", "Coffee", "Wine" ], "children": [ { "name": "Rosalia Auletta", "age": 36 } ] }
+{ "cid": 17, "name": "Ingeborg Monkhouse", "age": null, "address": null, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }
+{ "cid": 18, "name": "Dewayne Ardan", "age": 32, "address": { "number": 8229, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Wine", "Walking", "Bass" ], "children": [ { "name": "Wen Ardan", "age": null }, { "name": "Sachiko Ardan", "age": 11 }, { "name": "Francis Ardan", "age": 20 } ] }
+{ "cid": 20, "name": "Annice Fulwider", "age": 59, "address": { "number": 4257, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Arica Fulwider", "age": 47 }, { "name": "Charlotte Fulwider", "age": 16 }, { "name": "Robbi Fulwider", "age": 29 } ] }
+{ "cid": 21, "name": "Gidget Galamay", "age": 34, "address": { "number": 2854, "street": "Washington St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Brunilda Galamay", "age": null }, { "name": "Bethel Galamay", "age": null }, { "name": "Devon Galamay", "age": 17 } ] }
+{ "cid": 22, "name": "Sarita Burrer", "age": null, "address": null, "interests": [ "Cigars", "Computers" ], "children": [  ] }
+{ "cid": 23, "name": "Micheal Konen", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Myong Konen", "age": 26 }, { "name": "Celinda Konen", "age": 33 }, { "name": "Tammy Konen", "age": 53 }, { "name": "Chester Konen", "age": null } ] }
+{ "cid": 24, "name": "Hosea Wilburn", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 25, "name": "Goldie Vanhandel", "age": 37, "address": { "number": 6568, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Fishing", "Cigars" ], "children": [  ] }
+{ "cid": 26, "name": "Jone Okuna", "age": 78, "address": { "number": 6006, "street": "7th St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Franchesca Okuna", "age": null }, { "name": "Fred Okuna", "age": 17 }, { "name": "Marcellus Okuna", "age": null } ] }
+{ "cid": 27, "name": "Hollie Hyun", "age": null, "address": null, "interests": [ "Skiing", "Walking" ], "children": [ { "name": "Morton Hyun", "age": null }, { "name": "Farrah Hyun", "age": 40 }, { "name": "Ali Hyun", "age": null } ] }
+{ "cid": 28, "name": "Ariana Gillert", "age": 54, "address": { "number": 7331, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [ { "name": "Inge Gillert", "age": null }, { "name": "Jeraldine Gillert", "age": 13 } ] }
+{ "cid": 29, "name": "Ruthanne Tavana", "age": null, "address": null, "interests": [ "Movies" ], "children": [  ] }
+{ "cid": 30, "name": "Deedee Centner", "age": null, "address": null, "interests": [ "Skiing", "Wine", "Databases", "Movies" ], "children": [ { "name": "Lorilee Centner", "age": 30 }, { "name": "Thad Centner", "age": null } ] }
+{ "cid": 31, "name": "Venus Toboz", "age": 44, "address": { "number": 9465, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [ { "name": "Ashlie Toboz", "age": null } ] }
+{ "cid": 32, "name": "Tia Berkley", "age": 30, "address": { "number": 4507, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Music" ], "children": [ { "name": "Carmon Berkley", "age": null }, { "name": "Kristina Berkley", "age": null }, { "name": "Cristi Berkley", "age": 19 } ] }
+{ "cid": 33, "name": "Rayford Velmontes", "age": null, "address": null, "interests": [ "Fishing", "Video Games" ], "children": [  ] }
+{ "cid": 34, "name": "Sam Tannahill", "age": null, "address": null, "interests": [ "Books" ], "children": [  ] }
+{ "cid": 36, "name": "Neoma Preist", "age": 69, "address": { "number": 4830, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Databases", "Computers", "Coffee" ], "children": [ { "name": "Shery Preist", "age": null }, { "name": "Kelvin Preist", "age": 43 } ] }
+{ "cid": 37, "name": "Eliana Vient", "age": 89, "address": { "number": 4882, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Dario Vient", "age": 43 } ] }
+{ "cid": 38, "name": "Lawanna Abadi", "age": 35, "address": { "number": 6942, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Arthur Abadi", "age": 10 } ] }
+{ "cid": 39, "name": "Brock Froncillo", "age": 72, "address": { "number": 4645, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Skiing" ], "children": [ { "name": "Cole Froncillo", "age": null }, { "name": "Ivana Froncillo", "age": null }, { "name": "Hugh Froncillo", "age": 23 } ] }
+{ "cid": 40, "name": "Fidelia Connie", "age": 81, "address": { "number": 2298, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Basketball", "Base Jumping", "Walking", "Skiing" ], "children": [ { "name": "Elfreda Connie", "age": 43 }, { "name": "Josephine Connie", "age": 30 }, { "name": "Lucas Connie", "age": null } ] }
+{ "cid": 41, "name": "Kevin Giottonini", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Victor Giottonini", "age": 37 }, { "name": "Alverta Giottonini", "age": null } ] }
+{ "cid": 42, "name": "Asley Simco", "age": 38, "address": { "number": 3322, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Fishing", "Running", "Cigars" ], "children": [ { "name": "Micheal Simco", "age": null }, { "name": "Lawerence Simco", "age": null } ] }
+{ "cid": 44, "name": "Agustin Clubs", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Maxwell Clubs", "age": 31 }, { "name": "Rayna Clubs", "age": null }, { "name": "Darwin Clubs", "age": null } ] }
+{ "cid": 46, "name": "Columbus Huntington", "age": 22, "address": { "number": 3809, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Movies" ], "children": [ { "name": "Dana Huntington", "age": 10 }, { "name": "Rosa Huntington", "age": null } ] }
+{ "cid": 48, "name": "Delia Salveson", "age": 44, "address": { "number": 5596, "street": "7th St.", "city": "Portland" }, "interests": [ "Cigars", "Running", "Walking", "Running" ], "children": [ { "name": "Logan Salveson", "age": 21 }, { "name": "Temple Salveson", "age": 17 }, { "name": "Kimi Salveson", "age": null }, { "name": "Jacob Salveson", "age": 20 } ] }
+{ "cid": 49, "name": "Asa Schwing", "age": 70, "address": { "number": 2261, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Tennis" ], "children": [ { "name": "Joy Schwing", "age": 15 } ] }
+{ "cid": 50, "name": "Lise Gorelli", "age": null, "address": null, "interests": [ "Books", "Wine", "Skiing", "Computers" ], "children": [ { "name": "Darleen Gorelli", "age": null }, { "name": "Latia Gorelli", "age": null }, { "name": "Page Gorelli", "age": null }, { "name": "Columbus Gorelli", "age": null } ] }
+{ "cid": 51, "name": "Simonne Cape", "age": null, "address": null, "interests": [ "Bass", "Bass", "Books" ], "children": [ { "name": "Leland Cape", "age": null }, { "name": "Gearldine Cape", "age": null } ] }
+{ "cid": 52, "name": "Janna Tish", "age": 12, "address": { "number": 2598, "street": "Washington St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Mackenzie Tish", "age": null }, { "name": "Ettie Tish", "age": null }, { "name": "Hortencia Tish", "age": null }, { "name": "Paul Tish", "age": null } ] }
+{ "cid": 53, "name": "Ricardo Greiwe", "age": 24, "address": { "number": 8983, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 54, "name": "Haywood Vasiloff", "age": 63, "address": { "number": 8780, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Celsa Vasiloff", "age": 40 }, { "name": "Shawana Vasiloff", "age": 43 }, { "name": "Joel Vasiloff", "age": 42 }, { "name": "Timmy Vasiloff", "age": 33 } ] }
+{ "cid": 55, "name": "Terrence Bryant", "age": 12, "address": { "number": 3188, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine", "Cooking" ], "children": [ { "name": "Dayna Bryant", "age": null } ] }
+{ "cid": 56, "name": "Andria Killelea", "age": null, "address": null, "interests": [ "Cigars", "Skiing" ], "children": [  ] }
+{ "cid": 57, "name": "Celestine Mac", "age": null, "address": null, "interests": [ "Wine", "Computers", "Books" ], "children": [ { "name": "Kathyrn Mac", "age": 44 } ] }
+{ "cid": 58, "name": "Rosemarie Mattei", "age": 80, "address": { "number": 1390, "street": "Park St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Sonya Mattei", "age": 52 }, { "name": "Elenor Mattei", "age": null } ] }
+{ "cid": 59, "name": "Rea Villicana", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 61, "name": "Linsey Mose", "age": 17, "address": { "number": 9198, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles" ], "children": [ { "name": "Tilda Mose", "age": null }, { "name": "Lillie Mose", "age": null }, { "name": "Robyn Mose", "age": null } ] }
+{ "cid": 62, "name": "Kiley Machnik", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 64, "name": "Victor Susor", "age": 32, "address": { "number": 1690, "street": "Main St.", "city": "Portland" }, "interests": [ "Running", "Computers" ], "children": [  ] }
+{ "cid": 66, "name": "Lenny Latson", "age": null, "address": null, "interests": [ "Music", "Video Games" ], "children": [  ] }
+{ "cid": 67, "name": "Tobie Mattan", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 68, "name": "Chery Basini", "age": null, "address": null, "interests": [ "Video Games" ], "children": [  ] }
+{ "cid": 69, "name": "Many Yeargain", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brande Yeargain", "age": null }, { "name": "Tawna Yeargain", "age": null }, { "name": "Doris Yeargain", "age": null }, { "name": "Valeria Yeargain", "age": 51 } ] }
+{ "cid": 70, "name": "Mellisa Lek", "age": 62, "address": { "number": 4281, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Databases" ], "children": [  ] }
+{ "cid": 71, "name": "Alva Sieger", "age": null, "address": null, "interests": [ "Movies", "Walking" ], "children": [ { "name": "Renetta Sieger", "age": null }, { "name": "Shiloh Sieger", "age": 57 }, { "name": "Lavina Sieger", "age": null }, { "name": "Larraine Sieger", "age": null } ] }
+{ "cid": 73, "name": "Kelsey Flever", "age": 20, "address": { "number": 3555, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Puzzles", "Video Games" ], "children": [ { "name": "Isis Flever", "age": null }, { "name": "Gonzalo Flever", "age": null } ] }
+{ "cid": 74, "name": "Lonnie Ercolani", "age": 79, "address": { "number": 2655, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Music", "Coffee" ], "children": [ { "name": "Cassi Ercolani", "age": null } ] }
+{ "cid": 76, "name": "Opal Blewett", "age": null, "address": null, "interests": [ "Running", "Coffee", "Fishing" ], "children": [ { "name": "Violette Blewett", "age": null } ] }
+{ "cid": 77, "name": "Chantal Parriera", "age": 78, "address": { "number": 5967, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Squash", "Movies", "Coffee" ], "children": [  ] }
+{ "cid": 78, "name": "Wesley Huggler", "age": 80, "address": { "number": 3078, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Base Jumping", "Movies", "Skiing" ], "children": [ { "name": "Chassidy Huggler", "age": null }, { "name": "Emogene Huggler", "age": null }, { "name": "Cheryle Huggler", "age": null } ] }
+{ "cid": 79, "name": "Alyce Schoenle", "age": 57, "address": { "number": 1345, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Stewart Schoenle", "age": 16 }, { "name": "Bruce Schoenle", "age": 44 } ] }
+{ "cid": 81, "name": "Lavonda Manford", "age": 87, "address": { "number": 2423, "street": "Main St.", "city": "San Jose" }, "interests": [  ], "children": [  ] }
+{ "cid": 82, "name": "Gloria Junkins", "age": null, "address": null, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 83, "name": "Filiberto Couillard", "age": null, "address": null, "interests": [ "Cooking", "Books" ], "children": [ { "name": "Diane Couillard", "age": 19 }, { "name": "Asa Couillard", "age": 23 }, { "name": "Zaida Couillard", "age": 57 }, { "name": "Shavonne Couillard", "age": null } ] }
+{ "cid": 84, "name": "Huong Kachel", "age": null, "address": null, "interests": [ "Music", "Tennis", "Base Jumping" ], "children": [ { "name": "Katlyn Kachel", "age": 40 }, { "name": "Sherman Kachel", "age": null }, { "name": "Susana Kachel", "age": 32 } ] }
+{ "cid": 85, "name": "Fatimah Steltenpohl", "age": 25, "address": { "number": 6175, "street": "Park St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Genoveva Steltenpohl", "age": 14 } ] }
+{ "cid": 86, "name": "Sofia Mongiovi", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Rosamaria Mongiovi", "age": 25 } ] }
+{ "cid": 87, "name": "Torie Horuath", "age": 21, "address": { "number": 2713, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Puzzles", "Cigars", "Walking" ], "children": [ { "name": "Joshua Horuath", "age": 10 } ] }
+{ "cid": 88, "name": "Courtney Muckleroy", "age": null, "address": null, "interests": [ "Wine", "Movies", "Skiing" ], "children": [ { "name": "Alona Muckleroy", "age": 30 }, { "name": "Flora Muckleroy", "age": 41 }, { "name": "Angel Muckleroy", "age": null }, { "name": "Daniella Muckleroy", "age": null } ] }
+{ "cid": 89, "name": "Calandra Hedden", "age": 33, "address": { "number": 1231, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Wine" ], "children": [ { "name": "Damien Hedden", "age": 19 } ] }
+{ "cid": 90, "name": "Dorethea Korns", "age": null, "address": null, "interests": [ "Cooking", "Computers" ], "children": [ { "name": "Catheryn Korns", "age": 22 } ] }
+{ "cid": 91, "name": "Luna Machen", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Randal Machen", "age": 59 }, { "name": "Emely Machen", "age": null } ] }
+{ "cid": 92, "name": "Kenny Laychock", "age": 15, "address": { "number": 4790, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Basketball" ], "children": [  ] }
+{ "cid": 93, "name": "Garth Raigosa", "age": null, "address": null, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 94, "name": "Edgardo Dunnegan", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lyndia Dunnegan", "age": null } ] }
+{ "cid": 95, "name": "Gavin Locey", "age": 86, "address": { "number": 8162, "street": "Lake St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Terrell Locey", "age": null }, { "name": "Kazuko Locey", "age": 36 }, { "name": "Risa Locey", "age": null }, { "name": "Dorethea Locey", "age": 13 } ] }
+{ "cid": 96, "name": "Mara Aument", "age": 72, "address": { "number": 7709, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Cigars", "Cooking", "Movies" ], "children": [ { "name": "Leonardo Aument", "age": 22 } ] }
+{ "cid": 97, "name": "Mui Slosek", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Susanne Slosek", "age": 29 }, { "name": "Colleen Slosek", "age": null } ] }
+{ "cid": 98, "name": "Casimira Hilbrand", "age": 72, "address": { "number": 9693, "street": "Main St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Gudrun Hilbrand", "age": 18 }, { "name": "Dacia Hilbrand", "age": 26 }, { "name": "Kortney Hilbrand", "age": null }, { "name": "Luci Hilbrand", "age": null } ] }
+{ "cid": 99, "name": "Bernardina Thacher", "age": 35, "address": { "number": 1582, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Movies", "Fishing", "Fishing" ], "children": [ { "name": "Randee Thacher", "age": null }, { "name": "China Thacher", "age": null } ] }
+{ "cid": 101, "name": "Meaghan Vandel", "age": null, "address": null, "interests": [ "Music", "Base Jumping", "Books" ], "children": [ { "name": "Larissa Vandel", "age": null } ] }
+{ "cid": 102, "name": "Melany Rotan", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Christiana Rotan", "age": 21 }, { "name": "Lavina Rotan", "age": null }, { "name": "Billy Rotan", "age": null } ] }
+{ "cid": 103, "name": "Rosamond Milera", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 104, "name": "Neda Dilts", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Nona Dilts", "age": 28 }, { "name": "Wm Dilts", "age": null }, { "name": "Svetlana Dilts", "age": 46 }, { "name": "Iva Dilts", "age": 59 } ] }
+{ "cid": 105, "name": "Camilla Lohman", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Melania Lohman", "age": 50 }, { "name": "Mike Lohman", "age": 53 }, { "name": "Cassaundra Lohman", "age": 32 }, { "name": "Jay Lohman", "age": null } ] }
+{ "cid": 106, "name": "Charles Verna", "age": null, "address": null, "interests": [ "Bass", "Books" ], "children": [ { "name": "Betsy Verna", "age": 37 }, { "name": "Chae Verna", "age": 35 }, { "name": "Naoma Verna", "age": 42 } ] }
+{ "cid": 110, "name": "Karmen Milanesi", "age": 67, "address": { "number": 6223, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash" ], "children": [ { "name": "Emely Milanesi", "age": null }, { "name": "Adam Milanesi", "age": null }, { "name": "Gregg Milanesi", "age": null }, { "name": "Sean Milanesi", "age": 37 } ] }
+{ "cid": 111, "name": "Eddy Ortea", "age": 16, "address": { "number": 6874, "street": "Main St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Shera Ortea", "age": null } ] }
+{ "cid": 112, "name": "Dorie Lave", "age": 10, "address": { "number": 2286, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Coffee" ], "children": [ { "name": "Grady Lave", "age": null }, { "name": "Daysi Lave", "age": null } ] }
+{ "cid": 113, "name": "Alayna Daleske", "age": 87, "address": { "number": 4739, "street": "Main St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Hester Daleske", "age": null }, { "name": "Magnolia Daleske", "age": null }, { "name": "Bettye Daleske", "age": 32 } ] }
+{ "cid": 114, "name": "Stephine Capinpin", "age": 78, "address": { "number": 5618, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Puzzles", "Basketball" ], "children": [ { "name": "Krystal Capinpin", "age": 31 }, { "name": "Angelic Capinpin", "age": 45 } ] }
+{ "cid": 115, "name": "Jason Oakden", "age": 89, "address": { "number": 8182, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Music", "Basketball", "Movies" ], "children": [ { "name": "Johnson Oakden", "age": null }, { "name": "Neva Oakden", "age": null }, { "name": "Juliann Oakden", "age": null }, { "name": "Elmer Oakden", "age": null } ] }
+{ "cid": 116, "name": "Conrad Zozaya", "age": 81, "address": { "number": 1667, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Jenette Zozaya", "age": 17 } ] }
+{ "cid": 118, "name": "Ellis Skillom", "age": 78, "address": { "number": 9337, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running", "Cigars" ], "children": [ { "name": "Emory Skillom", "age": null } ] }
+{ "cid": 119, "name": "Chan Morreau", "age": 22, "address": { "number": 1774, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Squash" ], "children": [ { "name": "Arlette Morreau", "age": null } ] }
+{ "cid": 120, "name": "Jan Gianandrea", "age": null, "address": null, "interests": [ "Databases", "Movies", "Cigars" ], "children": [ { "name": "Keesha Gianandrea", "age": null }, { "name": "Vashti Gianandrea", "age": 35 }, { "name": "Larry Gianandrea", "age": 29 } ] }
+{ "cid": 121, "name": "Shiela Gaustad", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Phebe Gaustad", "age": null }, { "name": "Mavis Gaustad", "age": null }, { "name": "Zula Gaustad", "age": 37 } ] }
+{ "cid": 122, "name": "Wei Perpall", "age": 43, "address": { "number": 916, "street": "Washington St.", "city": "Los Angeles" }, "interests": [ "Bass" ], "children": [ { "name": "Mitchel Perpall", "age": 11 }, { "name": "Aliza Perpall", "age": null }, { "name": "King Perpall", "age": null }, { "name": "Santana Perpall", "age": 22 } ] }
+{ "cid": 123, "name": "Marian Courrege", "age": 30, "address": { "number": 7321, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Coffee" ], "children": [  ] }
+{ "cid": 124, "name": "Kelley Dressman", "age": null, "address": null, "interests": [ "Squash", "Databases", "Fishing" ], "children": [ { "name": "Evie Dressman", "age": null }, { "name": "Fredericka Dressman", "age": null }, { "name": "Leigh Dressman", "age": null }, { "name": "Luna Dressman", "age": 29 } ] }
+{ "cid": 125, "name": "Leigh Pusey", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Elbert Pusey", "age": 44 }, { "name": "Golden Pusey", "age": null }, { "name": "Maria Pusey", "age": null } ] }
+{ "cid": 126, "name": "Grayce Keir", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Antonia Keir", "age": 25 } ] }
+{ "cid": 127, "name": "Christian Anthes", "age": 32, "address": { "number": 6258, "street": "7th St.", "city": "Portland" }, "interests": [ "Running", "Bass" ], "children": [ { "name": "Sophia Anthes", "age": null } ] }
+{ "cid": 128, "name": "Edwin Harwick", "age": null, "address": null, "interests": [ "Fishing", "Squash", "Basketball" ], "children": [ { "name": "Tomeka Harwick", "age": 34 }, { "name": "Caroline Harwick", "age": 57 }, { "name": "Peter Harwick", "age": null }, { "name": "Adele Harwick", "age": null } ] }
+{ "cid": 129, "name": "Marisha Canzoneri", "age": 84, "address": { "number": 5507, "street": "View St.", "city": "Mountain View" }, "interests": [ "Music", "Databases", "Walking", "Walking" ], "children": [  ] }
+{ "cid": 130, "name": "Kandis Hissem", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Arianna Hissem", "age": null }, { "name": "Necole Hissem", "age": 53 }, { "name": "Manie Hissem", "age": null }, { "name": "Deshawn Hissem", "age": 27 } ] }
+{ "cid": 131, "name": "Kourtney Whitesel", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 132, "name": "Cindi Turntine", "age": 64, "address": { "number": 9432, "street": "Park St.", "city": "Portland" }, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Howard Turntine", "age": null } ] }
+{ "cid": 134, "name": "Alica Frontiero", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 135, "name": "Josette Dries", "age": null, "address": null, "interests": [ "Base Jumping", "Movies" ], "children": [ { "name": "Ben Dries", "age": 36 }, { "name": "Wm Dries", "age": 29 } ] }
+{ "cid": 136, "name": "Aubrey Kasuboski", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 137, "name": "Camellia Pressman", "age": 81, "address": { "number": 3947, "street": "Park St.", "city": "Seattle" }, "interests": [ "Movies", "Books", "Bass" ], "children": [ { "name": "Dwana Pressman", "age": null }, { "name": "Johnathan Pressman", "age": null }, { "name": "Kasey Pressman", "age": null }, { "name": "Mitch Pressman", "age": null } ] }
+{ "cid": 138, "name": "Ora Villafane", "age": null, "address": null, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Deeann Villafane", "age": 22 }, { "name": "Cody Villafane", "age": 47 } ] }
+{ "cid": 139, "name": "Micheline Argenal", "age": null, "address": null, "interests": [ "Bass", "Walking", "Movies" ], "children": [ { "name": "Joye Argenal", "age": 51 }, { "name": "Richard Argenal", "age": 46 }, { "name": "Sarah Argenal", "age": 21 }, { "name": "Jacinda Argenal", "age": 21 } ] }
+{ "cid": 140, "name": "Maryland Neas", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Brunilda Neas", "age": 28 } ] }
+{ "cid": 141, "name": "Adena Klockars", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Bass", "Cigars" ], "children": [  ] }
+{ "cid": 142, "name": "Ervin Softleigh", "age": null, "address": null, "interests": [ "Computers", "Skiing", "Cooking", "Coffee" ], "children": [ { "name": "Russell Softleigh", "age": 50 }, { "name": "Kristy Softleigh", "age": 54 }, { "name": "Refugio Softleigh", "age": null } ] }
+{ "cid": 143, "name": "Katelynn Kanzler", "age": 80, "address": { "number": 9453, "street": "Washington St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Carl Kanzler", "age": null } ] }
+{ "cid": 144, "name": "Celesta Sosebee", "age": 19, "address": { "number": 2683, "street": "7th St.", "city": "Portland" }, "interests": [ "Databases", "Databases" ], "children": [ { "name": "Jesse Sosebee", "age": null }, { "name": "Oralee Sosebee", "age": null }, { "name": "Sunday Sosebee", "age": null } ] }
+{ "cid": 145, "name": "Carey Bousman", "age": 61, "address": { "number": 16, "street": "Oak St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Lynda Bousman", "age": 32 }, { "name": "Evalyn Bousman", "age": 17 } ] }
+{ "cid": 146, "name": "Glennis Vanruiten", "age": 14, "address": { "number": 8272, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Squash", "Databases" ], "children": [ { "name": "Joanie Vanruiten", "age": null }, { "name": "Long Vanruiten", "age": null }, { "name": "Abdul Vanruiten", "age": null } ] }
+{ "cid": 147, "name": "Marla Pollan", "age": 24, "address": { "number": 9271, "street": "Oak St.", "city": "Portland" }, "interests": [ "Music" ], "children": [ { "name": "Song Pollan", "age": 11 }, { "name": "Lili Pollan", "age": 13 }, { "name": "Shaunte Pollan", "age": 12 }, { "name": "Sandie Pollan", "age": null } ] }
+{ "cid": 148, "name": "Coy Dulay", "age": 66, "address": { "number": 9793, "street": "Hill St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Emile Dulay", "age": null }, { "name": "Letitia Dulay", "age": 38 } ] }
+{ "cid": 149, "name": "Marcella Diamond", "age": 62, "address": { "number": 720, "street": "7th St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Ezra Diamond", "age": null } ] }
+{ "cid": 150, "name": "Jesus Vanleeuwen", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Sueann Vanleeuwen", "age": 47 }, { "name": "Refugia Vanleeuwen", "age": null }, { "name": "Taisha Vanleeuwen", "age": null }, { "name": "Nathaniel Vanleeuwen", "age": null } ] }
+{ "cid": 151, "name": "Charlyn Soyars", "age": 21, "address": { "number": 2796, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [  ] }
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [  ] }
+{ "cid": 156, "name": "Bobbye Kauppi", "age": 79, "address": { "number": 2051, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Cigars", "Movies" ], "children": [  ] }
+{ "cid": 157, "name": "Mckenzie Tahir", "age": 78, "address": { "number": 6752, "street": "Hill St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Margarita Tahir", "age": 18 }, { "name": "Mia Tahir", "age": 47 }, { "name": "Gaylord Tahir", "age": null } ] }
+{ "cid": 158, "name": "Rosalva Harvath", "age": 84, "address": { "number": 5569, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Wine", "Skiing", "Coffee" ], "children": [ { "name": "Taneka Harvath", "age": null }, { "name": "Ina Harvath", "age": 54 }, { "name": "Joanne Harvath", "age": 51 } ] }
+{ "cid": 159, "name": "Jeanmarie Franchini", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Nikita Franchini", "age": null }, { "name": "Willetta Franchini", "age": null }, { "name": "Ester Franchini", "age": 12 } ] }
+{ "cid": 160, "name": "Yevette Chanez", "age": null, "address": null, "interests": [ "Bass", "Wine", "Coffee" ], "children": [ { "name": "Walter Chanez", "age": 11 }, { "name": "Pa Chanez", "age": 27 } ] }
+{ "cid": 161, "name": "Lucia Tata", "age": 85, "address": { "number": 8058, "street": "Park St.", "city": "Seattle" }, "interests": [ "Basketball", "Bass" ], "children": [ { "name": "Jenifer Tata", "age": 70 }, { "name": "Erna Tata", "age": null } ] }
+{ "cid": 162, "name": "Chang Reek", "age": 85, "address": { "number": 5943, "street": "Washington St.", "city": "Portland" }, "interests": [ "Tennis", "Movies" ], "children": [ { "name": "Camelia Reek", "age": null }, { "name": "Eleonora Reek", "age": 36 }, { "name": "Shalonda Reek", "age": 39 }, { "name": "Stefan Reek", "age": 64 } ] }
+{ "cid": 163, "name": "Marcelene Sparano", "age": 36, "address": { "number": 5722, "street": "View St.", "city": "San Jose" }, "interests": [ "Basketball", "Databases" ], "children": [ { "name": "Luz Sparano", "age": null }, { "name": "Cassandra Sparano", "age": 21 }, { "name": "Martina Sparano", "age": 21 }, { "name": "Elisabeth Sparano", "age": null } ] }
+{ "cid": 164, "name": "Lucrecia Dahlhauser", "age": null, "address": null, "interests": [ "Wine" ], "children": [  ] }
+{ "cid": 165, "name": "Melodie Starrick", "age": null, "address": null, "interests": [ "Walking" ], "children": [ { "name": "Adria Starrick", "age": null }, { "name": "Tasha Starrick", "age": 25 } ] }
+{ "cid": 166, "name": "Gregorio Plummer", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Santiago Plummer", "age": null }, { "name": "Malisa Plummer", "age": 59 }, { "name": "Tracie Plummer", "age": 40 }, { "name": "Florentina Plummer", "age": 23 } ] }
+{ "cid": 169, "name": "Casandra Fierge", "age": 55, "address": { "number": 175, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 170, "name": "Dana Lese", "age": 38, "address": { "number": 575, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Walking", "Coffee" ], "children": [ { "name": "Yasmine Lese", "age": 24 }, { "name": "Ezekiel Lese", "age": 20 }, { "name": "Ammie Lese", "age": 27 }, { "name": "Robert Lese", "age": 15 } ] }
+{ "cid": 171, "name": "Eddie Shebchuk", "age": 86, "address": { "number": 3304, "street": "Lake St.", "city": "Portland" }, "interests": [ "Books" ], "children": [ { "name": "Harmony Shebchuk", "age": null } ] }
+{ "cid": 172, "name": "Weldon Alquesta", "age": null, "address": null, "interests": [ "Music", "Fishing", "Music" ], "children": [ { "name": "Kip Alquesta", "age": null } ] }
+{ "cid": 173, "name": "Annamae Lucien", "age": 46, "address": { "number": 1253, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Squash" ], "children": [ { "name": "Sanjuana Lucien", "age": 21 }, { "name": "Nathanael Lucien", "age": 27 }, { "name": "Jae Lucien", "age": null }, { "name": "Judith Lucien", "age": null } ] }
+{ "cid": 174, "name": "Taneka Baldassare", "age": 50, "address": { "number": 5787, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Junko Baldassare", "age": null }, { "name": "Denisha Baldassare", "age": null }, { "name": "Hermina Baldassare", "age": 17 }, { "name": "Lexie Baldassare", "age": null } ] }
+{ "cid": 175, "name": "Loise Obhof", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Susann Obhof", "age": null }, { "name": "Signe Obhof", "age": 38 } ] }
+{ "cid": 176, "name": "Kellie Andruszkiewic", "age": null, "address": null, "interests": [ "Fishing", "Puzzles", "Wine", "Skiing" ], "children": [ { "name": "Xiao Andruszkiewic", "age": null }, { "name": "Al Andruszkiewic", "age": 43 } ] }
+{ "cid": 177, "name": "Wilda Hanisch", "age": null, "address": null, "interests": [ "Wine", "Computers" ], "children": [ { "name": "Shannan Hanisch", "age": null }, { "name": "Marissa Hanisch", "age": 30 }, { "name": "Keely Hanisch", "age": 54 }, { "name": "Humberto Hanisch", "age": 17 } ] }
+{ "cid": 178, "name": "Athena Kaluna", "age": null, "address": null, "interests": [ "Running", "Computers", "Basketball" ], "children": [ { "name": "Rosalba Kaluna", "age": 48 }, { "name": "Max Kaluna", "age": 10 } ] }
+{ "cid": 179, "name": "Antonette Bernice", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Solange Bernice", "age": null } ] }
+{ "cid": 180, "name": "Theda Hilz", "age": 35, "address": { "number": 9918, "street": "Oak St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Ethan Hilz", "age": null }, { "name": "Bill Hilz", "age": 12 } ] }
+{ "cid": 181, "name": "Toni Sanghani", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Hollie Sanghani", "age": 29 } ] }
+{ "cid": 182, "name": "Christiana Westlie", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Ilda Westlie", "age": 18 } ] }
+{ "cid": 183, "name": "Ladawn Vyas", "age": 64, "address": { "number": 2663, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 184, "name": "Mirtha Ricciardi", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Elsa Ricciardi", "age": 30 }, { "name": "Vicente Ricciardi", "age": null }, { "name": "Sau Ricciardi", "age": 28 } ] }
+{ "cid": 185, "name": "Abigail Zugg", "age": 22, "address": { "number": 6676, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Computers", "Basketball", "Video Games", "Basketball" ], "children": [ { "name": "Peter Zugg", "age": 10 }, { "name": "Ariane Zugg", "age": null } ] }
+{ "cid": 187, "name": "Seema Hartsch", "age": 80, "address": { "number": 6629, "street": "Lake St.", "city": "Portland" }, "interests": [ "Coffee", "Coffee", "Cigars" ], "children": [ { "name": "Suellen Hartsch", "age": null }, { "name": "Pennie Hartsch", "age": 20 }, { "name": "Aubrey Hartsch", "age": null }, { "name": "Randy Hartsch", "age": 32 } ] }
+{ "cid": 188, "name": "Brynn Bendorf", "age": 23, "address": { "number": 1168, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Skiing" ], "children": [ { "name": "Leesa Bendorf", "age": 11 }, { "name": "Daine Bendorf", "age": null } ] }
+{ "cid": 189, "name": "Shyla Saathoff", "age": 85, "address": { "number": 9679, "street": "Main St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Johanne Saathoff", "age": 61 }, { "name": "Janett Saathoff", "age": null } ] }
+{ "cid": 190, "name": "Kristel Axelson", "age": null, "address": null, "interests": [ "Movies", "Books" ], "children": [ { "name": "Deja Axelson", "age": null } ] }
+{ "cid": 191, "name": "Lula Pangburn", "age": 42, "address": { "number": 1309, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Skiing", "Cooking", "Walking", "Video Games" ], "children": [ { "name": "Love Pangburn", "age": 11 }, { "name": "Bryant Pangburn", "age": 13 }, { "name": "Kenda Pangburn", "age": 14 } ] }
+{ "cid": 193, "name": "Melisa Maccarter", "age": 50, "address": { "number": 1494, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Basketball" ], "children": [ { "name": "Yetta Maccarter", "age": null }, { "name": "Geralyn Maccarter", "age": null } ] }
+{ "cid": 194, "name": "Leslee Apking", "age": 41, "address": { "number": 8107, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Irena Apking", "age": null }, { "name": "Arla Apking", "age": null } ] }
+{ "cid": 195, "name": "Annetta Demille", "age": 17, "address": { "number": 5722, "street": "Park St.", "city": "Portland" }, "interests": [ "Bass" ], "children": [ { "name": "Natacha Demille", "age": null }, { "name": "Giuseppe Demille", "age": null }, { "name": "Kami Demille", "age": null }, { "name": "Jewell Demille", "age": null } ] }
+{ "cid": 196, "name": "Darwin Seekell", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Kathryne Seekell", "age": null }, { "name": "Marlon Seekell", "age": null }, { "name": "Shiloh Seekell", "age": 51 } ] }
+{ "cid": 197, "name": "Garth Giannitti", "age": null, "address": null, "interests": [ "Coffee", "Cigars" ], "children": [ { "name": "Patsy Giannitti", "age": null }, { "name": "Ray Giannitti", "age": 35 }, { "name": "Kamala Giannitti", "age": 35 }, { "name": "Lauran Giannitti", "age": 25 } ] }
+{ "cid": 198, "name": "Thelma Youkers", "age": null, "address": null, "interests": [ "Basketball", "Movies", "Cooking" ], "children": [ { "name": "Shamika Youkers", "age": 28 } ] }
+{ "cid": 199, "name": "Rogelio Hannan", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Blanche Hannan", "age": null }, { "name": "Elvira Hannan", "age": null }, { "name": "Cinderella Hannan", "age": null } ] }
+{ "cid": 200, "name": "Stacey Bertran", "age": 78, "address": { "number": 9050, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Eugenia Bertran", "age": 59 }, { "name": "Lorri Bertran", "age": 29 }, { "name": "Corrie Bertran", "age": 52 } ] }
+{ "cid": 201, "name": "Tiny Hoysradt", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Simon Hoysradt", "age": 24 } ] }
+{ "cid": 202, "name": "Evangelina Poloskey", "age": 46, "address": { "number": 8285, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Wine", "Squash" ], "children": [ { "name": "Anthony Poloskey", "age": 27 }, { "name": "Olga Poloskey", "age": 10 }, { "name": "Carmon Poloskey", "age": 13 }, { "name": "Tanja Poloskey", "age": 20 } ] }
+{ "cid": 203, "name": "Elke Mazurowski", "age": 52, "address": { "number": 9276, "street": "View St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Esta Mazurowski", "age": null }, { "name": "Clarence Mazurowski", "age": 14 } ] }
+{ "cid": 204, "name": "Londa Herdt", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marnie Herdt", "age": 47 } ] }
+{ "cid": 205, "name": "Moises Plake", "age": null, "address": null, "interests": [ "Puzzles", "Computers" ], "children": [  ] }
+{ "cid": 206, "name": "Armand Hauersperger", "age": 67, "address": { "number": 7266, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine" ], "children": [ { "name": "Charlott Hauersperger", "age": 47 }, { "name": "Kayla Hauersperger", "age": null }, { "name": "Maris Hauersperger", "age": 52 } ] }
+{ "cid": 207, "name": "Phyliss Honda", "age": 22, "address": { "number": 8387, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Cooking", "Music", "Books" ], "children": [ { "name": "Bee Honda", "age": null }, { "name": "Cyril Honda", "age": null }, { "name": "Vertie Honda", "age": null } ] }
+{ "cid": 210, "name": "Jillian Roadruck", "age": null, "address": null, "interests": [ "Coffee", "Tennis" ], "children": [ { "name": "Marguerite Roadruck", "age": null }, { "name": "Ilana Roadruck", "age": null }, { "name": "Chantelle Roadruck", "age": 19 }, { "name": "Nikia Roadruck", "age": 43 } ] }
+{ "cid": 211, "name": "Kristian Knepshield", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 212, "name": "Christi Vichi", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 213, "name": "Micheal Evoy", "age": 68, "address": { "number": 1219, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Skiing", "Computers", "Books", "Puzzles" ], "children": [ { "name": "Socorro Evoy", "age": null }, { "name": "Gertude Evoy", "age": 36 }, { "name": "Araceli Evoy", "age": null }, { "name": "Yasmin Evoy", "age": null } ] }
+{ "cid": 214, "name": "Louvenia Zaffalon", "age": null, "address": null, "interests": [ "Skiing", "Books" ], "children": [  ] }
+{ "cid": 215, "name": "Ashton Schadegg", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Ciara Schadegg", "age": null }, { "name": "Karisa Schadegg", "age": 11 }, { "name": "Hayden Schadegg", "age": 44 } ] }
+{ "cid": 216, "name": "Odilia Lampson", "age": null, "address": null, "interests": [ "Wine", "Databases", "Basketball" ], "children": [ { "name": "Callie Lampson", "age": null } ] }
+{ "cid": 217, "name": "Scott Fulks", "age": null, "address": null, "interests": [ "Computers" ], "children": [  ] }
+{ "cid": 218, "name": "Clarinda Stagliano", "age": 76, "address": { "number": 3258, "street": "Park St.", "city": "San Jose" }, "interests": [ "Video Games", "Cigars" ], "children": [  ] }
+{ "cid": 219, "name": "Joelle Valazquez", "age": 73, "address": { "number": 9775, "street": "Park St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Gene Valazquez", "age": null }, { "name": "Ilona Valazquez", "age": null } ] }
+{ "cid": 220, "name": "Soila Hannemann", "age": null, "address": null, "interests": [ "Wine", "Puzzles", "Basketball" ], "children": [ { "name": "Piper Hannemann", "age": 44 } ] }
+{ "cid": 221, "name": "Delois Fiqueroa", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Cherri Fiqueroa", "age": null } ] }
+{ "cid": 222, "name": "Malcom Bloomgren", "age": 39, "address": { "number": 4674, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Databases", "Skiing" ], "children": [ { "name": "Rosia Bloomgren", "age": null }, { "name": "Bryant Bloomgren", "age": 15 }, { "name": "Donnie Bloomgren", "age": null } ] }
+{ "cid": 223, "name": "Margurite Embelton", "age": 19, "address": { "number": 554, "street": "Oak St.", "city": "Portland" }, "interests": [ "Running", "Fishing" ], "children": [ { "name": "Sherie Embelton", "age": null }, { "name": "Monica Embelton", "age": null }, { "name": "Jeanne Embelton", "age": null }, { "name": "Santiago Embelton", "age": null } ] }
+{ "cid": 224, "name": "Rene Rowey", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Walking", "Computers" ], "children": [ { "name": "Necole Rowey", "age": 26 }, { "name": "Sharyl Rowey", "age": 20 }, { "name": "Yvone Rowey", "age": 36 } ] }
+{ "cid": 225, "name": "Shantel Drapeaux", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Felicidad Drapeaux", "age": null }, { "name": "Wanetta Drapeaux", "age": 52 }, { "name": "Louise Drapeaux", "age": 28 }, { "name": "Pat Drapeaux", "age": null } ] }
+{ "cid": 226, "name": "Debrah Deppert", "age": 62, "address": { "number": 7699, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Coffee" ], "children": [ { "name": "Tonie Deppert", "age": 25 }, { "name": "Neil Deppert", "age": null } ] }
+{ "cid": 227, "name": "Carlos Skyes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Cortney Skyes", "age": 32 } ] }
+{ "cid": 228, "name": "Donnette Brumbley", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Madlyn Brumbley", "age": null }, { "name": "Apolonia Brumbley", "age": 13 }, { "name": "Stephine Brumbley", "age": null }, { "name": "Zelma Brumbley", "age": 51 } ] }
+{ "cid": 229, "name": "Raymundo Meurin", "age": null, "address": null, "interests": [ "Bass", "Basketball", "Databases" ], "children": [ { "name": "Mariela Meurin", "age": null } ] }
+{ "cid": 230, "name": "Tobias Vicars", "age": 66, "address": { "number": 638, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Wine", "Walking", "Books", "Walking" ], "children": [  ] }
+{ "cid": 231, "name": "Arianne Wedlow", "age": 68, "address": { "number": 9663, "street": "7th St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Birdie Wedlow", "age": 32 }, { "name": "Pearle Wedlow", "age": 13 }, { "name": "Jordon Wedlow", "age": 43 }, { "name": "Katherin Wedlow", "age": 18 } ] }
+{ "cid": 232, "name": "Joey Potes", "age": null, "address": null, "interests": [ "Bass", "Bass", "Base Jumping" ], "children": [ { "name": "Bobby Potes", "age": null } ] }
+{ "cid": 233, "name": "Sammy Coalter", "age": null, "address": null, "interests": [ "Fishing", "Base Jumping" ], "children": [ { "name": "Twana Coalter", "age": null }, { "name": "Nenita Coalter", "age": 30 } ] }
+{ "cid": 234, "name": "Ilana Brothern", "age": 36, "address": { "number": 4850, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles", "Walking", "Fishing" ], "children": [ { "name": "Shayne Brothern", "age": null }, { "name": "Phillis Brothern", "age": null } ] }
+{ "cid": 235, "name": "Orpha Craycraft", "age": null, "address": null, "interests": [ "Skiing", "Squash" ], "children": [  ] }
+{ "cid": 236, "name": "Muriel Laib", "age": 25, "address": { "number": 4481, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Fishing", "Tennis" ], "children": [ { "name": "Jann Laib", "age": null }, { "name": "Lila Laib", "age": 10 }, { "name": "Elyse Laib", "age": 11 } ] }
+{ "cid": 237, "name": "Sona Hehn", "age": 47, "address": { "number": 3720, "street": "Oak St.", "city": "Portland" }, "interests": [ "Computers", "Squash", "Coffee" ], "children": [ { "name": "Marquerite Hehn", "age": null }, { "name": "Suellen Hehn", "age": 29 }, { "name": "Herb Hehn", "age": 29 } ] }
+{ "cid": 238, "name": "Marcelina Redic", "age": null, "address": null, "interests": [ "Cigars", "Cigars", "Coffee" ], "children": [ { "name": "Renate Redic", "age": null }, { "name": "Kyoko Redic", "age": null }, { "name": "Dorthey Redic", "age": null } ] }
+{ "cid": 239, "name": "Celsa Fondow", "age": null, "address": null, "interests": [ "Base Jumping", "Computers", "Cooking", "Wine" ], "children": [  ] }
+{ "cid": 241, "name": "Lesha Ambrosia", "age": 49, "address": { "number": 6133, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running" ], "children": [ { "name": "Venice Ambrosia", "age": null } ] }
+{ "cid": 242, "name": "Jerold Shabot", "age": null, "address": null, "interests": [ "Fishing", "Walking", "Walking", "Puzzles" ], "children": [ { "name": "Marie Shabot", "age": 26 } ] }
+{ "cid": 243, "name": "Love Hoftiezer", "age": 88, "address": { "number": 2491, "street": "Main St.", "city": "Portland" }, "interests": [ "Cigars", "Coffee", "Books" ], "children": [ { "name": "Kellee Hoftiezer", "age": 77 } ] }
+{ "cid": 244, "name": "Rene Shenk", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Skiing" ], "children": [ { "name": "Victor Shenk", "age": 28 }, { "name": "Doris Shenk", "age": null }, { "name": "Max Shenk", "age": 51 } ] }
+{ "cid": 245, "name": "Lupe Abshear", "age": 55, "address": { "number": 7269, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Song Abshear", "age": null }, { "name": "Honey Abshear", "age": 31 } ] }
+{ "cid": 246, "name": "Kenda Heikkinen", "age": 63, "address": { "number": 8924, "street": "View St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [  ] }
+{ "cid": 247, "name": "Minda Heron", "age": 25, "address": { "number": 1629, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Tennis" ], "children": [  ] }
+{ "cid": 249, "name": "Kiana Satiago", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Stacy Satiago", "age": null } ] }
+{ "cid": 250, "name": "Angeles Saltonstall", "age": null, "address": null, "interests": [ "Tennis", "Fishing", "Movies" ], "children": [ { "name": "Suzanna Saltonstall", "age": null } ] }
+{ "cid": 251, "name": "Janeen Galston", "age": null, "address": null, "interests": [ "Basketball", "Base Jumping" ], "children": [  ] }
+{ "cid": 252, "name": "Almeda Charity", "age": 19, "address": { "number": 5553, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Rosia Charity", "age": null } ] }
+{ "cid": 254, "name": "Jeanice Longanecker", "age": 74, "address": { "number": 2613, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Books", "Base Jumping" ], "children": [  ] }
+{ "cid": 255, "name": "Cherri Piegaro", "age": 64, "address": { "number": 3802, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Elwood Piegaro", "age": null } ] }
+{ "cid": 256, "name": "Chester Rosenberg", "age": 46, "address": { "number": 8673, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Basketball" ], "children": [ { "name": "Gemma Rosenberg", "age": null }, { "name": "Marty Rosenberg", "age": null } ] }
+{ "cid": 257, "name": "Altha Jastrzebski", "age": 21, "address": { "number": 4405, "street": "Lake St.", "city": "Portland" }, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 258, "name": "Florentina Hense", "age": 20, "address": { "number": 8495, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Noelle Hense", "age": null }, { "name": "Roxann Hense", "age": null } ] }
+{ "cid": 259, "name": "Aurelio Darrigo", "age": 45, "address": { "number": 1114, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cooking", "Running" ], "children": [ { "name": "Leonard Darrigo", "age": 22 }, { "name": "Aron Darrigo", "age": null }, { "name": "Pamelia Darrigo", "age": 14 } ] }
+{ "cid": 260, "name": "Hedwig Caminero", "age": 81, "address": { "number": 4305, "street": "7th St.", "city": "Portland" }, "interests": [ "Video Games", "Databases" ], "children": [ { "name": "Hal Caminero", "age": null }, { "name": "Cierra Caminero", "age": 32 } ] }
+{ "cid": 263, "name": "Mellisa Machalek", "age": null, "address": null, "interests": [ "Bass", "Coffee", "Skiing" ], "children": [  ] }
+{ "cid": 264, "name": "Leon Yoshizawa", "age": 81, "address": { "number": 608, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Running", "Books", "Running" ], "children": [ { "name": "Carmela Yoshizawa", "age": 34 } ] }
+{ "cid": 265, "name": "Donte Stempien", "age": 25, "address": { "number": 3882, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Wine", "Books" ], "children": [  ] }
+{ "cid": 266, "name": "Carlee Friddle", "age": 74, "address": { "number": 6538, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases" ], "children": [ { "name": "Candie Friddle", "age": null }, { "name": "Zoila Friddle", "age": 59 } ] }
+{ "cid": 267, "name": "Renay Huddelston", "age": 68, "address": { "number": 1939, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Wine", "Base Jumping" ], "children": [ { "name": "Colene Huddelston", "age": null } ] }
+{ "cid": 268, "name": "Fernando Pingel", "age": null, "address": null, "interests": [ "Computers", "Tennis", "Books" ], "children": [ { "name": "Latrice Pingel", "age": null }, { "name": "Wade Pingel", "age": 13 }, { "name": "Christal Pingel", "age": null }, { "name": "Melania Pingel", "age": null } ] }
+{ "cid": 269, "name": "Dante Sharko", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Ahmad Sharko", "age": 34 }, { "name": "Mona Sharko", "age": null }, { "name": "Stephaine Sharko", "age": 42 }, { "name": "Adrianna Sharko", "age": null } ] }
+{ "cid": 270, "name": "Lavon Ascenzo", "age": null, "address": null, "interests": [ "Books", "Skiing" ], "children": [  ] }
+{ "cid": 271, "name": "Carey Ronin", "age": 44, "address": { "number": 8141, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Cigars", "Video Games" ], "children": [ { "name": "Lonny Ronin", "age": null }, { "name": "Armanda Ronin", "age": null } ] }
+{ "cid": 272, "name": "Frederick Valla", "age": 15, "address": { "number": 6805, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Video Games" ], "children": [ { "name": "Carroll Valla", "age": null } ] }
+{ "cid": 273, "name": "Corrinne Seaquist", "age": 24, "address": { "number": 6712, "street": "7th St.", "city": "Portland" }, "interests": [ "Puzzles", "Coffee", "Wine" ], "children": [ { "name": "Mignon Seaquist", "age": null }, { "name": "Leo Seaquist", "age": null } ] }
+{ "cid": 274, "name": "Claude Harral", "age": null, "address": null, "interests": [ "Squash", "Bass", "Cooking" ], "children": [ { "name": "Archie Harral", "age": null }, { "name": "Royal Harral", "age": null } ] }
+{ "cid": 275, "name": "Natalie Ifeanyi", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 276, "name": "Denyse Groth", "age": 81, "address": { "number": 6825, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Fishing", "Movies" ], "children": [ { "name": "Marilee Groth", "age": 12 }, { "name": "Lyla Groth", "age": 46 }, { "name": "Sarah Groth", "age": null } ] }
+{ "cid": 277, "name": "Malena Smock", "age": null, "address": null, "interests": [ "Running", "Base Jumping" ], "children": [ { "name": "Inocencia Smock", "age": 50 }, { "name": "Cleveland Smock", "age": null } ] }
+{ "cid": 278, "name": "Deb Nicole", "age": 59, "address": { "number": 9003, "street": "Park St.", "city": "Seattle" }, "interests": [ "Books", "Computers", "Walking", "Cooking" ], "children": [ { "name": "Len Nicole", "age": null } ] }
+{ "cid": 279, "name": "Saundra Croan", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Jena Croan", "age": 37 }, { "name": "Sarai Croan", "age": null }, { "name": "Junita Croan", "age": null }, { "name": "Ferdinand Croan", "age": 43 } ] }
+{ "cid": 280, "name": "Marlo Maung", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Harold Maung", "age": null } ] }
+{ "cid": 282, "name": "Emelda Dawood", "age": 32, "address": { "number": 5261, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Venus Dawood", "age": 12 }, { "name": "Gertrude Dawood", "age": null }, { "name": "Yen Dawood", "age": null }, { "name": "Theresa Dawood", "age": 16 } ] }
+{ "cid": 283, "name": "Pilar Fritts", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Jeneva Fritts", "age": null }, { "name": "Gail Fritts", "age": 25 } ] }
+{ "cid": 285, "name": "Edgar Farlin", "age": 75, "address": { "number": 3833, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Databases" ], "children": [ { "name": "Stefanie Farlin", "age": 60 }, { "name": "Catina Farlin", "age": null }, { "name": "Lizzie Farlin", "age": null }, { "name": "Beau Farlin", "age": null } ] }
+{ "cid": 286, "name": "Tara Sioma", "age": 18, "address": { "number": 9425, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Fishing" ], "children": [ { "name": "Dawna Sioma", "age": null }, { "name": "Jeanne Sioma", "age": null } ] }
+{ "cid": 288, "name": "Sharice Bachicha", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 289, "name": "Clarence Milette", "age": 16, "address": { "number": 3778, "street": "Oak St.", "city": "Seattle" }, "interests": [ "Books", "Base Jumping", "Music" ], "children": [  ] }
+{ "cid": 290, "name": "Kimberly Gullatte", "age": 51, "address": { "number": 4130, "street": "Park St.", "city": "San Jose" }, "interests": [ "Running", "Squash", "Databases" ], "children": [ { "name": "Micheal Gullatte", "age": null }, { "name": "Estrella Gullatte", "age": 40 }, { "name": "Corrine Gullatte", "age": null }, { "name": "Ward Gullatte", "age": null } ] }
+{ "cid": 291, "name": "Svetlana Moone", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Running", "Walking" ], "children": [ { "name": "Emelina Moone", "age": null }, { "name": "Candi Moone", "age": null } ] }
+{ "cid": 292, "name": "Mariana Cosselman", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Madge Cosselman", "age": 43 } ] }
+{ "cid": 293, "name": "Terresa Hofstetter", "age": 15, "address": { "number": 3338, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Computers", "Running", "Cigars", "Fishing" ], "children": [ { "name": "Hubert Hofstetter", "age": null }, { "name": "Jolie Hofstetter", "age": null } ] }
+{ "cid": 294, "name": "Foster Salimi", "age": 79, "address": { "number": 8439, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Pei Salimi", "age": null } ] }
+{ "cid": 295, "name": "Guillermina Florek", "age": 61, "address": { "number": 3704, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Movies", "Books" ], "children": [ { "name": "Donnie Florek", "age": null }, { "name": "Jeannetta Florek", "age": 38 }, { "name": "Leigha Florek", "age": null }, { "name": "Zenobia Florek", "age": 10 } ] }
+{ "cid": 296, "name": "Doreen Kea", "age": 89, "address": { "number": 7034, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Movies" ], "children": [ { "name": "Lyndsay Kea", "age": 68 }, { "name": "Trena Kea", "age": 18 } ] }
+{ "cid": 297, "name": "Adeline Frierson", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Fishing" ], "children": [ { "name": "Marci Frierson", "age": null }, { "name": "Rolanda Frierson", "age": null }, { "name": "Del Frierson", "age": null } ] }
+{ "cid": 298, "name": "Brittny Christin", "age": null, "address": null, "interests": [ "Databases", "Video Games" ], "children": [ { "name": "Hilario Christin", "age": null }, { "name": "Clarine Christin", "age": null } ] }
+{ "cid": 299, "name": "Jacob Wainman", "age": 76, "address": { "number": 4551, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Wine", "Coffee" ], "children": [ { "name": "Abram Wainman", "age": 28 }, { "name": "Ramonita Wainman", "age": 18 }, { "name": "Sheryll Wainman", "age": null } ] }
+{ "cid": 300, "name": "Garret Colgrove", "age": 85, "address": { "number": 9937, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Puzzles", "Fishing" ], "children": [ { "name": "Janna Colgrove", "age": null }, { "name": "Jerilyn Colgrove", "age": 35 } ] }
+{ "cid": 301, "name": "Cherry Steenwyk", "age": 88, "address": { "number": 4138, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Movies" ], "children": [ { "name": "Toccara Steenwyk", "age": 66 }, { "name": "Tari Steenwyk", "age": null }, { "name": "Lawanna Steenwyk", "age": null }, { "name": "Ossie Steenwyk", "age": 26 } ] }
+{ "cid": 302, "name": "Rosalie Laderer", "age": null, "address": null, "interests": [ "Tennis", "Movies", "Movies" ], "children": [ { "name": "Moriah Laderer", "age": null }, { "name": "Liana Laderer", "age": 21 }, { "name": "Genia Laderer", "age": 45 } ] }
+{ "cid": 303, "name": "Michel Bayird", "age": 37, "address": { "number": 7939, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Shan Bayird", "age": 12 } ] }
+{ "cid": 304, "name": "Francine Reddin", "age": 39, "address": { "number": 9392, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Base Jumping" ], "children": [ { "name": "Millicent Reddin", "age": null } ] }
+{ "cid": 305, "name": "Tuyet Leinbach", "age": null, "address": null, "interests": [ "Puzzles", "Walking" ], "children": [  ] }
+{ "cid": 306, "name": "Laurie Tuff", "age": null, "address": null, "interests": [ "Computers", "Base Jumping", "Bass", "Basketball" ], "children": [ { "name": "Sharie Tuff", "age": null }, { "name": "Ollie Tuff", "age": 53 }, { "name": "Gonzalo Tuff", "age": null }, { "name": "Thomas Tuff", "age": null } ] }
+{ "cid": 307, "name": "Abraham Lanphear", "age": 20, "address": { "number": 7552, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Video Games" ], "children": [ { "name": "Toccara Lanphear", "age": null }, { "name": "Milly Lanphear", "age": null } ] }
+{ "cid": 308, "name": "Solomon Schwenke", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "Gertrude Schwenke", "age": null }, { "name": "Marcell Schwenke", "age": 41 }, { "name": "Shalon Schwenke", "age": null } ] }
+{ "cid": 309, "name": "Lise Baiz", "age": 46, "address": { "number": 352, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Bass", "Squash" ], "children": [ { "name": "Alisa Baiz", "age": 18 }, { "name": "Elidia Baiz", "age": 28 }, { "name": "Ray Baiz", "age": 19 } ] }
+{ "cid": 311, "name": "Ria Haflett", "age": 14, "address": { "number": 9513, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Walking" ], "children": [ { "name": "Jimmie Haflett", "age": null }, { "name": "Dario Haflett", "age": null }, { "name": "Robbyn Haflett", "age": null } ] }
+{ "cid": 312, "name": "Epifania Chorney", "age": 62, "address": { "number": 9749, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Wine", "Puzzles", "Tennis" ], "children": [ { "name": "Lizeth Chorney", "age": 22 } ] }
+{ "cid": 313, "name": "Lasandra Raigosa", "age": null, "address": null, "interests": [ "Walking", "Walking" ], "children": [ { "name": "Lanelle Raigosa", "age": null } ] }
+{ "cid": 314, "name": "Gwendolyn Abeb", "age": 85, "address": { "number": 3977, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Basketball", "Music", "Squash", "Walking" ], "children": [ { "name": "Aurelia Abeb", "age": 14 }, { "name": "Young Abeb", "age": null }, { "name": "Shay Abeb", "age": null }, { "name": "Lavina Abeb", "age": 15 } ] }
+{ "cid": 315, "name": "Kallie Eiselein", "age": null, "address": null, "interests": [ "Computers", "Tennis" ], "children": [  ] }
+{ "cid": 316, "name": "Patrina Whitting", "age": 74, "address": { "number": 4772, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Video Games", "Bass" ], "children": [ { "name": "Rubye Whitting", "age": null } ] }
+{ "cid": 317, "name": "Zona Caffarel", "age": 52, "address": { "number": 9419, "street": "Cedar St.", "city": "Seattle" }, "interests": [ "Tennis", "Coffee" ], "children": [ { "name": "Cortez Caffarel", "age": null } ] }
+{ "cid": 318, "name": "Shaunna Royal", "age": 86, "address": { "number": 8681, "street": "7th St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Shantell Royal", "age": 37 }, { "name": "Shalon Royal", "age": 50 }, { "name": "Chung Royal", "age": 26 } ] }
+{ "cid": 319, "name": "Ashlie Rott", "age": 42, "address": { "number": 366, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Computers", "Cooking", "Databases" ], "children": [  ] }
+{ "cid": 320, "name": "Charley Hermenegildo", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Melda Hermenegildo", "age": 51 }, { "name": "Lashon Hermenegildo", "age": null } ] }
+{ "cid": 322, "name": "Jaclyn Ettl", "age": 83, "address": { "number": 4500, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Skiing" ], "children": [ { "name": "Noah Ettl", "age": 30 }, { "name": "Kesha Ettl", "age": null } ] }
+{ "cid": 323, "name": "Rebeca Grisostomo", "age": 26, "address": { "number": 399, "street": "View St.", "city": "Portland" }, "interests": [ "Music" ], "children": [ { "name": "Iva Grisostomo", "age": 12 }, { "name": "Ha Grisostomo", "age": null }, { "name": "Lorna Grisostomo", "age": null } ] }
+{ "cid": 324, "name": "Wendolyn Centorino", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 325, "name": "Ai Tarleton", "age": null, "address": null, "interests": [ "Coffee", "Music" ], "children": [ { "name": "Risa Tarleton", "age": 24 }, { "name": "Leonila Tarleton", "age": null }, { "name": "Thomasina Tarleton", "age": null } ] }
+{ "cid": 326, "name": "Tad Tellers", "age": null, "address": null, "interests": [ "Books", "Tennis", "Base Jumping" ], "children": [ { "name": "Fannie Tellers", "age": null } ] }
+{ "cid": 327, "name": "Minnie Scali", "age": null, "address": null, "interests": [ "Cooking", "Squash", "Skiing" ], "children": [ { "name": "Jalisa Scali", "age": null }, { "name": "Preston Scali", "age": null }, { "name": "Stephani Scali", "age": 47 }, { "name": "Candra Scali", "age": null } ] }
+{ "cid": 328, "name": "Mallory Sheffey", "age": 27, "address": { "number": 8532, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Cooking" ], "children": [ { "name": "Regan Sheffey", "age": 14 } ] }
+{ "cid": 330, "name": "Noma Tollefsen", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Melody Tollefsen", "age": 45 }, { "name": "Caridad Tollefsen", "age": 15 } ] }
+{ "cid": 331, "name": "Willena Provenza", "age": 43, "address": { "number": 6742, "street": "Main St.", "city": "Portland" }, "interests": [ "Basketball" ], "children": [ { "name": "Alesha Provenza", "age": 32 }, { "name": "Marty Provenza", "age": null }, { "name": "Lindy Provenza", "age": 21 }, { "name": "Junita Provenza", "age": null } ] }
+{ "cid": 332, "name": "Malcom Cafasso", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marie Cafasso", "age": null }, { "name": "Asley Cafasso", "age": 38 } ] }
+{ "cid": 333, "name": "Conchita Olivera", "age": 37, "address": { "number": 8519, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Base Jumping" ], "children": [ { "name": "Trenton Olivera", "age": null }, { "name": "Shin Olivera", "age": 26 }, { "name": "Everett Olivera", "age": 15 }, { "name": "Shera Olivera", "age": 20 } ] }
+{ "cid": 335, "name": "Odessa Dammeyer", "age": 18, "address": { "number": 6828, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Basketball", "Bass", "Cigars" ], "children": [ { "name": "Lindsey Dammeyer", "age": null } ] }
+{ "cid": 336, "name": "Jalisa Talamantez", "age": 78, "address": { "number": 9902, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Video Games", "Squash" ], "children": [  ] }
+{ "cid": 337, "name": "Kay Durney", "age": 52, "address": { "number": 4203, "street": "View St.", "city": "Seattle" }, "interests": [ "Walking" ], "children": [ { "name": "Velia Durney", "age": 38 }, { "name": "Erin Durney", "age": null } ] }
+{ "cid": 338, "name": "Dorthey Roncskevitz", "age": 38, "address": { "number": 4366, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Computers" ], "children": [ { "name": "Mindy Roncskevitz", "age": null } ] }
+{ "cid": 339, "name": "Sharonda Catalino", "age": 15, "address": { "number": 7616, "street": "Washington St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Lorine Catalino", "age": null } ] }
+{ "cid": 340, "name": "Erick Faiola", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Marquita Faiola", "age": null }, { "name": "Tasia Faiola", "age": null }, { "name": "Micheal Faiola", "age": 24 }, { "name": "Salvatore Faiola", "age": null } ] }
+{ "cid": 343, "name": "Kaylee Ozaine", "age": 78, "address": { "number": 3367, "street": "Washington St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Darwin Ozaine", "age": 35 }, { "name": "Anne Ozaine", "age": 13 }, { "name": "Kenneth Ozaine", "age": null }, { "name": "Pat Ozaine", "age": 53 } ] }
+{ "cid": 346, "name": "Elden Choma", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Valorie Choma", "age": null }, { "name": "Leslee Choma", "age": null } ] }
+{ "cid": 347, "name": "Patrick Feighan", "age": 34, "address": { "number": 7613, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Puzzles", "Books" ], "children": [ { "name": "Madaline Feighan", "age": null } ] }
+{ "cid": 348, "name": "Matthew Pantaleo", "age": 80, "address": { "number": 9782, "street": "Washington St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Faviola Pantaleo", "age": null }, { "name": "Yang Pantaleo", "age": null }, { "name": "Christopher Pantaleo", "age": null }, { "name": "Jacqui Pantaleo", "age": 58 } ] }
+{ "cid": 349, "name": "Cristine Hila", "age": null, "address": null, "interests": [ "Books" ], "children": [ { "name": "Nyla Hila", "age": 51 } ] }
+{ "cid": 352, "name": "Bonny Sischo", "age": null, "address": null, "interests": [ "Bass", "Movies", "Computers" ], "children": [ { "name": "Judith Sischo", "age": 43 }, { "name": "Adeline Sischo", "age": null }, { "name": "Dayna Sischo", "age": null } ] }
+{ "cid": 353, "name": "Melody Bernas", "age": 76, "address": { "number": 6783, "street": "Main St.", "city": "San Jose" }, "interests": [ "Base Jumping" ], "children": [ { "name": "Kristel Bernas", "age": 45 }, { "name": "Clorinda Bernas", "age": 10 }, { "name": "Natosha Bernas", "age": null } ] }
+{ "cid": 354, "name": "Marian Munzell", "age": 73, "address": { "number": 4504, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Fishing", "Puzzles" ], "children": [  ] }
+{ "cid": 355, "name": "Elois Leckband", "age": null, "address": null, "interests": [ "Skiing", "Wine" ], "children": [  ] }
+{ "cid": 356, "name": "Pearlene Sakumoto", "age": 22, "address": { "number": 5895, "street": "7th St.", "city": "San Jose" }, "interests": [ "Computers", "Bass", "Base Jumping", "Coffee" ], "children": [  ] }
+{ "cid": 357, "name": "Dario Lobach", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Kendall Lobach", "age": 37 } ] }
+{ "cid": 358, "name": "Fredricka Krum", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Darrick Krum", "age": null }, { "name": "Julieann Krum", "age": null }, { "name": "Sun Krum", "age": null }, { "name": "Rosamaria Krum", "age": 16 } ] }
+{ "cid": 360, "name": "Billye Grumet", "age": 82, "address": { "number": 7052, "street": "Main St.", "city": "Portland" }, "interests": [ "Coffee" ], "children": [ { "name": "Linnea Grumet", "age": null }, { "name": "Charline Grumet", "age": 67 } ] }
+{ "cid": 361, "name": "Angela Lacki", "age": 35, "address": { "number": 9710, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Skiing" ], "children": [  ] }
+{ "cid": 362, "name": "Alta Bantug", "age": null, "address": null, "interests": [ "Computers" ], "children": [  ] }
+{ "cid": 363, "name": "Merlene Hoying", "age": 25, "address": { "number": 2105, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Squash", "Squash", "Music" ], "children": [ { "name": "Andrew Hoying", "age": 10 } ] }
+{ "cid": 364, "name": "Joni Dazey", "age": 14, "address": { "number": 1237, "street": "Oak St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Kraig Dazey", "age": null } ] }
+{ "cid": 366, "name": "Rosia Wenzinger", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 367, "name": "Cassondra Fabiani", "age": null, "address": null, "interests": [ "Squash", "Tennis" ], "children": [ { "name": "Evia Fabiani", "age": null }, { "name": "Chaya Fabiani", "age": null }, { "name": "Sherman Fabiani", "age": null }, { "name": "Kathi Fabiani", "age": 54 } ] }
+{ "cid": 368, "name": "Tequila Scandalios", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nilsa Scandalios", "age": null }, { "name": "Kaye Scandalios", "age": 23 }, { "name": "Angelo Scandalios", "age": 24 } ] }
+{ "cid": 369, "name": "Nickole Dory", "age": 10, "address": { "number": 4761, "street": "View St.", "city": "Portland" }, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Annmarie Dory", "age": null }, { "name": "Michele Dory", "age": null }, { "name": "Annamae Dory", "age": null }, { "name": "Flora Dory", "age": null } ] }
+{ "cid": 370, "name": "Shonta Furby", "age": 18, "address": { "number": 5792, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Databases" ], "children": [ { "name": "Raleigh Furby", "age": null }, { "name": "Britta Furby", "age": null }, { "name": "Gay Furby", "age": null }, { "name": "Elenor Furby", "age": null } ] }
+{ "cid": 371, "name": "Agatha Tensley", "age": 13, "address": { "number": 1810, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Bass", "Running", "Movies" ], "children": [ { "name": "Launa Tensley", "age": null } ] }
+{ "cid": 372, "name": "Zena Keglovic", "age": 22, "address": { "number": 7675, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Basketball", "Wine" ], "children": [  ] }
+{ "cid": 373, "name": "Heather Seward", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Glinda Seward", "age": 59 }, { "name": "Maribeth Seward", "age": null }, { "name": "Teofila Seward", "age": null }, { "name": "Clemencia Seward", "age": 38 } ] }
+{ "cid": 374, "name": "Clair Quinn", "age": null, "address": null, "interests": [ "Walking", "Books" ], "children": [ { "name": "Wesley Quinn", "age": 17 }, { "name": "Maren Quinn", "age": 50 }, { "name": "Ila Quinn", "age": 43 }, { "name": "Casie Quinn", "age": null } ] }
+{ "cid": 375, "name": "Chia Sagaser", "age": 15, "address": { "number": 6025, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Skiing" ], "children": [ { "name": "Garnet Sagaser", "age": null }, { "name": "Mario Sagaser", "age": null }, { "name": "Sun Sagaser", "age": null } ] }
+{ "cid": 376, "name": "Jeffrey Hegarty", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "April Hegarty", "age": null }, { "name": "Wilbur Hegarty", "age": null }, { "name": "Hanh Hegarty", "age": null } ] }
+{ "cid": 377, "name": "Zona Klint", "age": 22, "address": { "number": 6320, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Evie Klint", "age": null }, { "name": "Sharyl Klint", "age": 11 }, { "name": "Joaquina Klint", "age": 11 }, { "name": "Doloris Klint", "age": 11 } ] }
+{ "cid": 378, "name": "Melany Matias", "age": 10, "address": { "number": 8838, "street": "Main St.", "city": "Seattle" }, "interests": [ "Coffee", "Tennis", "Bass" ], "children": [ { "name": "Earnestine Matias", "age": null }, { "name": "Lore Matias", "age": null } ] }
+{ "cid": 379, "name": "Penney Huslander", "age": 58, "address": { "number": 6919, "street": "7th St.", "city": "Portland" }, "interests": [ "Cooking", "Running" ], "children": [ { "name": "Magaret Huslander", "age": null }, { "name": "Dodie Huslander", "age": 14 } ] }
+{ "cid": 380, "name": "Silva Purdue", "age": 33, "address": { "number": 1759, "street": "7th St.", "city": "Portland" }, "interests": [ "Music", "Squash" ], "children": [ { "name": "Marshall Purdue", "age": null }, { "name": "Yuki Purdue", "age": null }, { "name": "Val Purdue", "age": 12 }, { "name": "Dominica Purdue", "age": null } ] }
+{ "cid": 381, "name": "Kassandra Ereth", "age": null, "address": null, "interests": [ "Base Jumping", "Base Jumping", "Databases", "Walking" ], "children": [ { "name": "Angelina Ereth", "age": 46 }, { "name": "Tristan Ereth", "age": null }, { "name": "Johnny Ereth", "age": null } ] }
+{ "cid": 383, "name": "Marty Castine", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nakisha Castine", "age": 40 }, { "name": "Mina Castine", "age": null }, { "name": "Katrice Castine", "age": 56 }, { "name": "Reuben Castine", "age": null } ] }
+{ "cid": 385, "name": "Jody Favaron", "age": 73, "address": { "number": 4724, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Fishing" ], "children": [ { "name": "Elane Favaron", "age": 47 }, { "name": "Katherine Favaron", "age": 38 } ] }
+{ "cid": 386, "name": "Mao Gradowski", "age": 36, "address": { "number": 5116, "street": "Washington St.", "city": "Mountain View" }, "interests": [ "Computers", "Fishing" ], "children": [ { "name": "Jeneva Gradowski", "age": null }, { "name": "Thu Gradowski", "age": 22 }, { "name": "Daphine Gradowski", "age": null }, { "name": "Providencia Gradowski", "age": null } ] }
+{ "cid": 387, "name": "Leonard Mabie", "age": 33, "address": { "number": 6703, "street": "View St.", "city": "Mountain View" }, "interests": [ "Bass", "Running", "Walking" ], "children": [ { "name": "Jone Mabie", "age": 16 }, { "name": "Claire Mabie", "age": null }, { "name": "Larraine Mabie", "age": null }, { "name": "Corrina Mabie", "age": null } ] }
+{ "cid": 389, "name": "Loraine Morfee", "age": 72, "address": { "number": 2945, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Berry Morfee", "age": 30 } ] }
+{ "cid": 390, "name": "Shera Cung", "age": 69, "address": { "number": 5850, "street": "Hill St.", "city": "San Jose" }, "interests": [ "Fishing", "Computers", "Cigars", "Base Jumping" ], "children": [ { "name": "Lenore Cung", "age": 20 } ] }
+{ "cid": 391, "name": "Lynn Gregory", "age": 51, "address": { "number": 1249, "street": "Hill St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Jeannine Gregory", "age": null }, { "name": "Jaymie Gregory", "age": null }, { "name": "Lorrine Gregory", "age": 37 } ] }
+{ "cid": 392, "name": "Isiah Nussbaumer", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 393, "name": "Rossana Monton", "age": 34, "address": { "number": 4490, "street": "Main St.", "city": "Portland" }, "interests": [ "Skiing", "Base Jumping" ], "children": [ { "name": "Glayds Monton", "age": null }, { "name": "Lily Monton", "age": null }, { "name": "Raina Monton", "age": null }, { "name": "Hilma Monton", "age": null } ] }
+{ "cid": 394, "name": "Lizette Roux", "age": 57, "address": { "number": 458, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books" ], "children": [ { "name": "Doloris Roux", "age": null } ] }
+{ "cid": 395, "name": "Bob Layman", "age": 61, "address": { "number": 3646, "street": "Washington St.", "city": "Los Angeles" }, "interests": [  ], "children": [  ] }
+{ "cid": 396, "name": "Delfina Calcara", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Sybil Calcara", "age": null } ] }
+{ "cid": 397, "name": "Blake Kealy", "age": 34, "address": { "number": 2156, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Databases", "Wine", "Cigars" ], "children": [ { "name": "Lorenza Kealy", "age": null }, { "name": "Beula Kealy", "age": 15 }, { "name": "Kristofer Kealy", "age": null }, { "name": "Shayne Kealy", "age": null } ] }
+{ "cid": 398, "name": "Piedad Paranada", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Claribel Paranada", "age": 22 }, { "name": "Lincoln Paranada", "age": null }, { "name": "Cecilia Paranada", "age": null } ] }
+{ "cid": 399, "name": "Myra Millwee", "age": null, "address": null, "interests": [ "Tennis", "Running", "Tennis" ], "children": [ { "name": "Gaye Millwee", "age": null } ] }
+{ "cid": 400, "name": "Jeffery Maresco", "age": null, "address": null, "interests": [ "Coffee", "Bass" ], "children": [  ] }
+{ "cid": 401, "name": "Moises Jago", "age": 27, "address": { "number": 3773, "street": "Main St.", "city": "San Jose" }, "interests": [ "Music" ], "children": [ { "name": "Shoshana Jago", "age": null }, { "name": "Juliet Jago", "age": null }, { "name": "Berneice Jago", "age": 13 } ] }
+{ "cid": 402, "name": "Terrilyn Shinall", "age": null, "address": null, "interests": [ "Computers", "Skiing", "Music" ], "children": [ { "name": "Minh Shinall", "age": null }, { "name": "Diedre Shinall", "age": 22 } ] }
+{ "cid": 403, "name": "Kayleigh Houey", "age": null, "address": null, "interests": [ "Fishing", "Music" ], "children": [ { "name": "Ta Houey", "age": null }, { "name": "Ayana Houey", "age": null }, { "name": "Dominique Houey", "age": null }, { "name": "Denise Houey", "age": 48 } ] }
+{ "cid": 404, "name": "Harriette Abo", "age": null, "address": null, "interests": [ "Walking", "Running" ], "children": [  ] }
+{ "cid": 405, "name": "Shawnda Landborg", "age": 73, "address": { "number": 2396, "street": "Hill St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Cherrie Landborg", "age": 10 } ] }
+{ "cid": 406, "name": "Addie Mandez", "age": null, "address": null, "interests": [ "Tennis", "Cigars", "Books" ], "children": [ { "name": "Rosendo Mandez", "age": 34 } ] }
+{ "cid": 407, "name": "Bebe Cotney", "age": null, "address": null, "interests": [ "Books", "Tennis" ], "children": [ { "name": "Daren Cotney", "age": null }, { "name": "Lady Cotney", "age": 48 } ] }
+{ "cid": 408, "name": "Ava Zornes", "age": null, "address": null, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 410, "name": "Jennie Longhenry", "age": 82, "address": { "number": 7427, "street": "Main St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Charles Longhenry", "age": 61 }, { "name": "Faviola Longhenry", "age": 25 }, { "name": "Darline Longhenry", "age": null }, { "name": "Lorean Longhenry", "age": null } ] }
+{ "cid": 411, "name": "Cindi Pepin", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Fallon Pepin", "age": 39 }, { "name": "Armanda Pepin", "age": null }, { "name": "Loriann Pepin", "age": null }, { "name": "Bambi Pepin", "age": 43 } ] }
+{ "cid": 412, "name": "Devon Szalai", "age": 26, "address": { "number": 2384, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Bass", "Books", "Books" ], "children": [ { "name": "Yolonda Szalai", "age": null }, { "name": "Denita Szalai", "age": null }, { "name": "Priscila Szalai", "age": 10 }, { "name": "Cassondra Szalai", "age": 12 } ] }
+{ "cid": 413, "name": "Maurice Landrie", "age": null, "address": null, "interests": [ "Computers", "Coffee" ], "children": [ { "name": "Gail Landrie", "age": 37 }, { "name": "Carylon Landrie", "age": null }, { "name": "Allen Landrie", "age": 16 }, { "name": "Andreas Landrie", "age": null } ] }
+{ "cid": 414, "name": "Sixta Smithheart", "age": null, "address": null, "interests": [ "Skiing", "Books", "Computers" ], "children": [ { "name": "Nicholas Smithheart", "age": null } ] }
+{ "cid": 415, "name": "Valentin Mclarney", "age": null, "address": null, "interests": [ "Squash", "Squash", "Video Games" ], "children": [ { "name": "Vanda Mclarney", "age": 17 } ] }
+{ "cid": 417, "name": "Irene Funderberg", "age": 45, "address": { "number": 8503, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Music", "Skiing", "Running" ], "children": [ { "name": "Lyndia Funderberg", "age": 14 }, { "name": "Herta Funderberg", "age": null } ] }
+{ "cid": 418, "name": "Gavin Delpino", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Gianna Delpino", "age": null }, { "name": "Carmella Delpino", "age": 55 } ] }
+{ "cid": 419, "name": "Hector Brisbone", "age": null, "address": null, "interests": [ "Databases", "Books", "Walking", "Databases" ], "children": [ { "name": "Frederick Brisbone", "age": 17 } ] }
+{ "cid": 420, "name": "Coralie Regueira", "age": null, "address": null, "interests": [ "Books", "Tennis" ], "children": [ { "name": "Latoyia Regueira", "age": 31 }, { "name": "Obdulia Regueira", "age": 12 }, { "name": "Herlinda Regueira", "age": null } ] }
+{ "cid": 421, "name": "Rubye Dillabough", "age": 55, "address": { "number": 6980, "street": "View St.", "city": "Sunnyvale" }, "interests": [ "Squash" ], "children": [ { "name": "Hyacinth Dillabough", "age": 19 }, { "name": "Arie Dillabough", "age": null } ] }
+{ "cid": 422, "name": "Annmarie Whitcher", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Honey Whitcher", "age": null }, { "name": "Dan Whitcher", "age": 22 } ] }
+{ "cid": 424, "name": "Camila Rightmire", "age": 25, "address": { "number": 7542, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Bass", "Running", "Puzzles" ], "children": [ { "name": "Donny Rightmire", "age": 14 }, { "name": "Karlene Rightmire", "age": 10 }, { "name": "Nicholas Rightmire", "age": null }, { "name": "Margareta Rightmire", "age": null } ] }
+{ "cid": 426, "name": "Agripina Philley", "age": 79, "address": { "number": 1533, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Georgianne Philley", "age": null }, { "name": "Neville Philley", "age": null }, { "name": "Brande Philley", "age": 42 }, { "name": "Tanisha Philley", "age": null } ] }
+{ "cid": 427, "name": "Janay Presutti", "age": null, "address": null, "interests": [ "Walking" ], "children": [ { "name": "Julietta Presutti", "age": null } ] }
+{ "cid": 428, "name": "Tiffany Waye", "age": null, "address": null, "interests": [ "Basketball", "Cigars" ], "children": [ { "name": "Berna Waye", "age": null }, { "name": "Kiersten Waye", "age": null }, { "name": "Romeo Waye", "age": null }, { "name": "Marvel Waye", "age": 56 } ] }
+{ "cid": 429, "name": "Eladia Scannell", "age": 20, "address": { "number": 5036, "street": "Main St.", "city": "Portland" }, "interests": [ "Skiing", "Music", "Movies" ], "children": [  ] }
+{ "cid": 430, "name": "Cari Woll", "age": 45, "address": { "number": 8226, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cooking", "Walking", "Cooking" ], "children": [ { "name": "Tomasa Woll", "age": 32 }, { "name": "Annika Woll", "age": 21 } ] }
+{ "cid": 431, "name": "Estela Tolbent", "age": 27, "address": { "number": 7186, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Databases" ], "children": [ { "name": "Joie Tolbent", "age": null }, { "name": "Angila Tolbent", "age": null }, { "name": "Anastasia Tolbent", "age": 14 } ] }
+{ "cid": 432, "name": "Judi Vinet", "age": 85, "address": { "number": 7304, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Wine" ], "children": [ { "name": "Golden Vinet", "age": 20 }, { "name": "Maragret Vinet", "age": null }, { "name": "Keshia Vinet", "age": 10 }, { "name": "Gary Vinet", "age": 73 } ] }
+{ "cid": 433, "name": "Caleb Merrbach", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Amado Merrbach", "age": 45 } ] }
+{ "cid": 434, "name": "Tamesha Soho", "age": 33, "address": { "number": 4534, "street": "Park St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Cody Soho", "age": null }, { "name": "Glennie Soho", "age": 22 } ] }
+{ "cid": 435, "name": "Britni Kazemi", "age": 69, "address": { "number": 7868, "street": "Main St.", "city": "San Jose" }, "interests": [ "Databases", "Music", "Wine" ], "children": [  ] }
+{ "cid": 436, "name": "Xenia Pool", "age": null, "address": null, "interests": [ "Books" ], "children": [  ] }
+{ "cid": 437, "name": "Marlene Macintyre", "age": 86, "address": { "number": 3708, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Wine", "Walking", "Music", "Coffee" ], "children": [ { "name": "Todd Macintyre", "age": null }, { "name": "Mechelle Macintyre", "age": 50 } ] }
+{ "cid": 438, "name": "Allegra Pefanis", "age": null, "address": null, "interests": [ "Computers", "Music", "Cigars" ], "children": [  ] }
+{ "cid": 439, "name": "Lillia Villnave", "age": 34, "address": { "number": 9212, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Otis Villnave", "age": null } ] }
+{ "cid": 440, "name": "Rosie Shappen", "age": null, "address": null, "interests": [ "Cooking", "Music", "Cigars" ], "children": [ { "name": "Jung Shappen", "age": 11 } ] }
+{ "cid": 441, "name": "Jamison Reeser", "age": 84, "address": { "number": 9376, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Tennis" ], "children": [ { "name": "Elena Reeser", "age": 28 } ] }
+{ "cid": 442, "name": "Val Disorda", "age": null, "address": null, "interests": [ "Bass" ], "children": [ { "name": "Simone Disorda", "age": 53 }, { "name": "Jacalyn Disorda", "age": 41 }, { "name": "Ron Disorda", "age": null }, { "name": "Clifton Disorda", "age": null } ] }
+{ "cid": 445, "name": "Walton Komo", "age": 16, "address": { "number": 8769, "street": "Main St.", "city": "Seattle" }, "interests": [ "Running", "Basketball", "Tennis" ], "children": [  ] }
+{ "cid": 446, "name": "Lilly Grannell", "age": 21, "address": { "number": 5894, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Computers", "Tennis", "Puzzles", "Books" ], "children": [ { "name": "Victor Grannell", "age": null } ] }
+{ "cid": 447, "name": "Iris Schoneman", "age": 34, "address": { "number": 7648, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Bass", "Wine", "Puzzles", "Cigars" ], "children": [ { "name": "Shemika Schoneman", "age": 11 }, { "name": "Maritza Schoneman", "age": 21 }, { "name": "Martha Schoneman", "age": 20 } ] }
+{ "cid": 448, "name": "Gracie Pekas", "age": 59, "address": { "number": 4732, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Wine", "Cigars" ], "children": [ { "name": "Jeanett Pekas", "age": 35 }, { "name": "Jennifer Pekas", "age": null }, { "name": "Carrol Pekas", "age": null } ] }
+{ "cid": 449, "name": "Jacinda Markle", "age": null, "address": null, "interests": [ "Basketball", "Basketball", "Computers" ], "children": [ { "name": "Tam Markle", "age": 45 } ] }
+{ "cid": 450, "name": "Althea Mohammed", "age": null, "address": null, "interests": [ "Fishing", "Databases" ], "children": [ { "name": "Jasper Mohammed", "age": null } ] }
+{ "cid": 451, "name": "Lelia Sondelski", "age": 60, "address": { "number": 4044, "street": "Park St.", "city": "Portland" }, "interests": [ "Books", "Squash", "Walking" ], "children": [  ] }
+{ "cid": 452, "name": "Casie Marasigan", "age": null, "address": null, "interests": [ "Walking", "Computers" ], "children": [ { "name": "Connie Marasigan", "age": null }, { "name": "Kimberlie Marasigan", "age": null } ] }
+{ "cid": 453, "name": "Sherlyn Deadmond", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Base Jumping" ], "children": [ { "name": "Torrie Deadmond", "age": 46 }, { "name": "Cleotilde Deadmond", "age": 55 }, { "name": "Garry Deadmond", "age": 34 }, { "name": "Valrie Deadmond", "age": null } ] }
+{ "cid": 454, "name": "Irving Lhuillier", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Emile Lhuillier", "age": null }, { "name": "Albert Lhuillier", "age": null }, { "name": "Ingeborg Lhuillier", "age": 23 }, { "name": "Shila Lhuillier", "age": 55 } ] }
+{ "cid": 455, "name": "Manual Altizer", "age": 70, "address": { "number": 6293, "street": "7th St.", "city": "Portland" }, "interests": [ "Running", "Fishing", "Coffee" ], "children": [ { "name": "Katherine Altizer", "age": null } ] }
+{ "cid": 456, "name": "Kim Cervera", "age": 89, "address": { "number": 3967, "street": "Lake St.", "city": "Portland" }, "interests": [ "Fishing" ], "children": [ { "name": "Winona Cervera", "age": 37 }, { "name": "Shanice Cervera", "age": null }, { "name": "Michaele Cervera", "age": null } ] }
+{ "cid": 457, "name": "Jenice Boger", "age": null, "address": null, "interests": [ "Skiing", "Databases", "Running" ], "children": [  ] }
+{ "cid": 458, "name": "Ivan Sien", "age": 17, "address": { "number": 9981, "street": "Lake St.", "city": "Portland" }, "interests": [ "Cooking", "Coffee" ], "children": [ { "name": "Laurence Sien", "age": null }, { "name": "Nelle Sien", "age": null }, { "name": "Thalia Sien", "age": null } ] }
+{ "cid": 459, "name": "Mable Ellwein", "age": 60, "address": { "number": 1138, "street": "Lake St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Stan Ellwein", "age": 19 }, { "name": "Ashlea Ellwein", "age": 13 }, { "name": "Tiesha Ellwein", "age": 28 } ] }
+{ "cid": 460, "name": "Jeraldine Choules", "age": null, "address": null, "interests": [ "Fishing" ], "children": [ { "name": "Berneice Choules", "age": 16 }, { "name": "Jaime Choules", "age": 21 }, { "name": "Li Choules", "age": 20 }, { "name": "Leah Choules", "age": null } ] }
+{ "cid": 461, "name": "Dessie Schnibbe", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 462, "name": "Margaret Galvis", "age": null, "address": null, "interests": [ "Base Jumping", "Movies", "Movies" ], "children": [ { "name": "Isaac Galvis", "age": 48 }, { "name": "Mei Galvis", "age": null }, { "name": "Asha Galvis", "age": null }, { "name": "Zachery Galvis", "age": null } ] }
+{ "cid": 463, "name": "Mika Rininger", "age": null, "address": null, "interests": [ "Databases", "Cooking" ], "children": [ { "name": "Inez Rininger", "age": 58 }, { "name": "Betty Rininger", "age": null }, { "name": "Laurie Rininger", "age": 48 }, { "name": "Billie Rininger", "age": null } ] }
+{ "cid": 464, "name": "Petra Kinsel", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Janise Kinsel", "age": null }, { "name": "Donnie Kinsel", "age": 26 }, { "name": "Joana Kinsel", "age": 12 } ] }
+{ "cid": 465, "name": "Rey Arango", "age": 68, "address": { "number": 1788, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Tennis" ], "children": [  ] }
+{ "cid": 466, "name": "Paulene Bagen", "age": 87, "address": { "number": 4093, "street": "View St.", "city": "Mountain View" }, "interests": [ "Music" ], "children": [ { "name": "Antione Bagen", "age": null }, { "name": "Samatha Bagen", "age": null } ] }
+{ "cid": 467, "name": "Magali Ingerson", "age": null, "address": null, "interests": [ "Books", "Base Jumping" ], "children": [ { "name": "Monty Ingerson", "age": 11 }, { "name": "Noelia Ingerson", "age": 47 }, { "name": "Tennie Ingerson", "age": null }, { "name": "Merrill Ingerson", "age": null } ] }
+{ "cid": 468, "name": "Raeann Conry", "age": 68, "address": { "number": 4312, "street": "Cedar St.", "city": "Seattle" }, "interests": [ "Squash" ], "children": [ { "name": "Ellena Conry", "age": 36 }, { "name": "Lynwood Conry", "age": 13 }, { "name": "Coreen Conry", "age": 23 } ] }
+{ "cid": 470, "name": "Yesenia Doyon", "age": 78, "address": { "number": 3641, "street": "7th St.", "city": "Seattle" }, "interests": [ "Databases", "Puzzles" ], "children": [ { "name": "Halley Doyon", "age": null }, { "name": "Teisha Doyon", "age": 33 }, { "name": "Warren Doyon", "age": null } ] }
+{ "cid": 471, "name": "Nicol Majersky", "age": null, "address": null, "interests": [ "Video Games", "Books" ], "children": [ { "name": "Alise Majersky", "age": null }, { "name": "Kathline Majersky", "age": 53 }, { "name": "Charlie Majersky", "age": 45 }, { "name": "Helaine Majersky", "age": null } ] }
+{ "cid": 472, "name": "Kelley Mischler", "age": 38, "address": { "number": 7988, "street": "Lake St.", "city": "Los Angeles" }, "interests": [ "Movies", "Cooking", "Skiing" ], "children": [ { "name": "Keila Mischler", "age": 19 }, { "name": "Evie Mischler", "age": 15 } ] }
+{ "cid": 475, "name": "Brinda Gouker", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Gayle Gouker", "age": 52 } ] }
+{ "cid": 478, "name": "Sophia Whitt", "age": 26, "address": { "number": 2787, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Fishing", "Databases" ], "children": [ { "name": "Irving Whitt", "age": 13 }, { "name": "Jeannette Whitt", "age": null } ] }
+{ "cid": 479, "name": "Danilo Varney", "age": 17, "address": { "number": 9330, "street": "Hill St.", "city": "Portland" }, "interests": [ "Wine" ], "children": [ { "name": "Shelby Varney", "age": null }, { "name": "Fidela Varney", "age": null }, { "name": "Maynard Varney", "age": null }, { "name": "Lindsay Varney", "age": null } ] }
+{ "cid": 480, "name": "Nigel Pitmon", "age": null, "address": null, "interests": [ "Puzzles", "Books" ], "children": [ { "name": "Janene Pitmon", "age": null }, { "name": "Louie Pitmon", "age": 19 }, { "name": "Genny Pitmon", "age": 24 }, { "name": "Robby Pitmon", "age": 55 } ] }
+{ "cid": 481, "name": "Leana Revera", "age": null, "address": null, "interests": [ "Running", "Skiing" ], "children": [ { "name": "Marquita Revera", "age": null } ] }
+{ "cid": 482, "name": "Samantha Stonis", "age": null, "address": null, "interests": [ "Databases" ], "children": [  ] }
+{ "cid": 483, "name": "Elsa Vigen", "age": null, "address": null, "interests": [ "Wine", "Databases" ], "children": [ { "name": "Larae Vigen", "age": null }, { "name": "Elwood Vigen", "age": null } ] }
+{ "cid": 484, "name": "Bennie Dragaj", "age": null, "address": null, "interests": [ "Fishing", "Databases", "Wine" ], "children": [ { "name": "Viva Dragaj", "age": 13 } ] }
+{ "cid": 485, "name": "Gene Rogoff", "age": null, "address": null, "interests": [ "Fishing" ], "children": [ { "name": "Ebonie Rogoff", "age": null } ] }
+{ "cid": 486, "name": "Willa Patman", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Ross Patman", "age": 42 }, { "name": "Erin Patman", "age": null }, { "name": "Vannessa Patman", "age": 11 }, { "name": "Hilaria Patman", "age": 28 } ] }
+{ "cid": 487, "name": "Zenia Virgilio", "age": 46, "address": { "number": 584, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Walking", "Squash", "Wine" ], "children": [ { "name": "Quintin Virgilio", "age": null }, { "name": "Edith Virgilio", "age": null }, { "name": "Nicolle Virgilio", "age": 33 } ] }
+{ "cid": 489, "name": "Brigid Delosier", "age": 31, "address": { "number": 6082, "street": "Oak St.", "city": "Portland" }, "interests": [ "Tennis", "Cigars", "Music" ], "children": [ { "name": "Allegra Delosier", "age": null }, { "name": "Yong Delosier", "age": 10 }, { "name": "Steffanie Delosier", "age": 13 } ] }
+{ "cid": 492, "name": "Gene Alcazar", "age": 59, "address": { "number": 9650, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Computers" ], "children": [ { "name": "Olympia Alcazar", "age": null }, { "name": "Mark Alcazar", "age": 37 }, { "name": "Danilo Alcazar", "age": null } ] }
+{ "cid": 493, "name": "Lindsey Trout", "age": 86, "address": { "number": 7619, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Base Jumping", "Skiing" ], "children": [ { "name": "Madlyn Trout", "age": 58 }, { "name": "Amie Trout", "age": 72 } ] }
+{ "cid": 494, "name": "Delma Deever", "age": 84, "address": { "number": 5044, "street": "7th St.", "city": "Seattle" }, "interests": [ "Computers", "Basketball", "Squash" ], "children": [  ] }
+{ "cid": 496, "name": "Lonna Starkweather", "age": 80, "address": { "number": 1162, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Bass", "Running" ], "children": [ { "name": "Matilda Starkweather", "age": null } ] }
+{ "cid": 497, "name": "Chantay Balak", "age": null, "address": null, "interests": [ "Bass", "Fishing" ], "children": [ { "name": "John Balak", "age": null }, { "name": "Thu Balak", "age": 38 } ] }
+{ "cid": 498, "name": "Arleen Sultzer", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Norine Sultzer", "age": 29 } ] }
+{ "cid": 499, "name": "Carlita Tarlton", "age": 43, "address": { "number": 9148, "street": "Main St.", "city": "Sunnyvale" }, "interests": [ "Computers", "Base Jumping", "Video Games" ], "children": [  ] }
+{ "cid": 500, "name": "Tierra Bjorklund", "age": null, "address": null, "interests": [ "Puzzles", "Skiing" ], "children": [ { "name": "Avelina Bjorklund", "age": 54 }, { "name": "Mallory Bjorklund", "age": null } ] }
+{ "cid": 501, "name": "Alyce Coant", "age": null, "address": null, "interests": [ "Music", "Base Jumping" ], "children": [ { "name": "Elyse Coant", "age": 50 } ] }
+{ "cid": 502, "name": "Lawana Mulik", "age": 82, "address": { "number": 3071, "street": "Park St.", "city": "Portland" }, "interests": [ "Cigars", "Cigars" ], "children": [ { "name": "Carrie Mulik", "age": null }, { "name": "Sharlene Mulik", "age": 33 }, { "name": "Leone Mulik", "age": 46 } ] }
+{ "cid": 503, "name": "Phyliss Cassani", "age": null, "address": null, "interests": [ "Squash", "Tennis" ], "children": [ { "name": "Rolando Cassani", "age": 44 }, { "name": "Rikki Cassani", "age": 18 }, { "name": "Monty Cassani", "age": 40 } ] }
+{ "cid": 504, "name": "Marla Kolenda", "age": 57, "address": { "number": 464, "street": "View St.", "city": "San Jose" }, "interests": [ "Coffee" ], "children": [ { "name": "Iliana Kolenda", "age": 34 }, { "name": "Ammie Kolenda", "age": 20 }, { "name": "Candi Kolenda", "age": 23 }, { "name": "Lyla Kolenda", "age": 23 } ] }
+{ "cid": 505, "name": "Mike Runk", "age": null, "address": null, "interests": [ "Databases", "Computers", "Running", "Video Games" ], "children": [ { "name": "Lashawn Runk", "age": 21 } ] }
+{ "cid": 506, "name": "Jonna Kolbusz", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Debrah Kolbusz", "age": null }, { "name": "Hugh Kolbusz", "age": null } ] }
+{ "cid": 507, "name": "Yuk Flanegan", "age": null, "address": null, "interests": [ "Puzzles", "Puzzles", "Squash" ], "children": [ { "name": "Alexander Flanegan", "age": null } ] }
+{ "cid": 508, "name": "Tiffany Kimmey", "age": 64, "address": { "number": 8625, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Bass", "Walking" ], "children": [  ] }
+{ "cid": 509, "name": "Alvaro Johnke", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Allison Johnke", "age": null }, { "name": "Ellan Johnke", "age": null } ] }
+{ "cid": 510, "name": "Candace Morello", "age": null, "address": null, "interests": [ "Wine", "Base Jumping", "Running" ], "children": [ { "name": "Sandy Morello", "age": 57 }, { "name": "Delois Morello", "age": 15 } ] }
+{ "cid": 512, "name": "Paul Cobian", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Will Cobian", "age": 30 }, { "name": "Conrad Cobian", "age": 35 }, { "name": "Justin Cobian", "age": 11 } ] }
+{ "cid": 513, "name": "Marianna Gortman", "age": 49, "address": { "number": 927, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Databases" ], "children": [  ] }
+{ "cid": 514, "name": "Raleigh Belling", "age": 56, "address": { "number": 7408, "street": "View St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [  ] }
+{ "cid": 515, "name": "Connie Banis", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brittni Banis", "age": null }, { "name": "Deloras Banis", "age": 25 } ] }
+{ "cid": 516, "name": "Taunya Berkbigler", "age": 82, "address": { "number": 5441, "street": "View St.", "city": "Seattle" }, "interests": [ "Databases", "Tennis" ], "children": [ { "name": "Cherry Berkbigler", "age": 27 }, { "name": "Perry Berkbigler", "age": null } ] }
+{ "cid": 517, "name": "Alfonso Bruderer", "age": null, "address": null, "interests": [ "Bass" ], "children": [  ] }
+{ "cid": 518, "name": "Cora Ingargiola", "age": null, "address": null, "interests": [ "Skiing", "Squash", "Movies" ], "children": [ { "name": "Katlyn Ingargiola", "age": null }, { "name": "Mike Ingargiola", "age": null }, { "name": "Lawrence Ingargiola", "age": null }, { "name": "Isabelle Ingargiola", "age": null } ] }
+{ "cid": 519, "name": "Julianna Goodsell", "age": 59, "address": { "number": 5594, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Video Games", "Fishing" ], "children": [  ] }
+{ "cid": 520, "name": "Janay Bernbeck", "age": null, "address": null, "interests": [ "Databases", "Databases" ], "children": [ { "name": "Aurea Bernbeck", "age": null }, { "name": "Tiara Bernbeck", "age": null }, { "name": "Alfredia Bernbeck", "age": 26 } ] }
+{ "cid": 521, "name": "Frankie Hofmann", "age": null, "address": null, "interests": [ "Databases", "Movies" ], "children": [ { "name": "Shirlee Hofmann", "age": 32 }, { "name": "Jacque Hofmann", "age": 23 }, { "name": "Jazmin Hofmann", "age": null }, { "name": "Serena Hofmann", "age": 56 } ] }
+{ "cid": 522, "name": "Daryl Kissack", "age": 86, "address": { "number": 7825, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Squash", "Base Jumping", "Tennis" ], "children": [ { "name": "Darrel Kissack", "age": 21 } ] }
+{ "cid": 523, "name": "Johanne Huls", "age": null, "address": null, "interests": [ "Books", "Bass" ], "children": [ { "name": "Melynda Huls", "age": null }, { "name": "Vicky Huls", "age": 16 }, { "name": "Charlott Huls", "age": null } ] }
+{ "cid": 524, "name": "Rickie Manche", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 525, "name": "Miquel Hodnefield", "age": 12, "address": { "number": 4784, "street": "7th St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Darnell Hodnefield", "age": null }, { "name": "Particia Hodnefield", "age": null } ] }
+{ "cid": 528, "name": "Tamela Witherbee", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Penney Witherbee", "age": null } ] }
+{ "cid": 529, "name": "Cinderella Lewis", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Flor Lewis", "age": null }, { "name": "Alonzo Lewis", "age": 23 } ] }
+{ "cid": 530, "name": "Olevia Sturk", "age": 72, "address": { "number": 1939, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Computers" ], "children": [ { "name": "Cindy Sturk", "age": 18 }, { "name": "Alishia Sturk", "age": null }, { "name": "Sonja Sturk", "age": 51 } ] }
+{ "cid": 531, "name": "Camelia Yoes", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 532, "name": "Tania Fraklin", "age": 38, "address": { "number": 2857, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Squash", "Databases" ], "children": [  ] }
+{ "cid": 533, "name": "Trinity Urquidez", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Corrine Urquidez", "age": 29 }, { "name": "Markita Urquidez", "age": 19 }, { "name": "Danette Urquidez", "age": null } ] }
+{ "cid": 534, "name": "Bridgett Ebel", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 535, "name": "Juana Hirliman", "age": 87, "address": { "number": 6763, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Movies" ], "children": [ { "name": "Ursula Hirliman", "age": 40 }, { "name": "Doretha Hirliman", "age": 30 }, { "name": "Leisha Hirliman", "age": 49 } ] }
+{ "cid": 536, "name": "Wilber Rehrer", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Zulema Rehrer", "age": null }, { "name": "Lavonda Rehrer", "age": null }, { "name": "Stacey Rehrer", "age": 59 } ] }
+{ "cid": 537, "name": "Mara Hugar", "age": null, "address": null, "interests": [ "Fishing", "Skiing", "Skiing" ], "children": [ { "name": "Krista Hugar", "age": null } ] }
+{ "cid": 538, "name": "Mack Vollick", "age": null, "address": null, "interests": [ "Base Jumping", "Fishing", "Walking", "Computers" ], "children": [ { "name": "Gil Vollick", "age": 11 }, { "name": "Marica Vollick", "age": null } ] }
+{ "cid": 539, "name": "Nicky Graceffo", "age": null, "address": null, "interests": [ "Video Games" ], "children": [  ] }
+{ "cid": 540, "name": "Bryanna Herling", "age": 67, "address": { "number": 7682, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Cyrstal Herling", "age": 50 }, { "name": "Vallie Herling", "age": 54 }, { "name": "Doris Herling", "age": null } ] }
+{ "cid": 541, "name": "Sammy Adamitis", "age": 71, "address": { "number": 5593, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Books", "Tennis", "Cooking" ], "children": [  ] }
+{ "cid": 542, "name": "Eveline Smedley", "age": 50, "address": { "number": 5513, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Skiing", "Walking" ], "children": [ { "name": "Lynsey Smedley", "age": 26 } ] }
+{ "cid": 543, "name": "Pearl Nollette", "age": null, "address": null, "interests": [ "Base Jumping", "Running" ], "children": [  ] }
+{ "cid": 544, "name": "Silas Demay", "age": 69, "address": { "number": 447, "street": "Main St.", "city": "Portland" }, "interests": [ "Tennis", "Bass" ], "children": [ { "name": "Latonya Demay", "age": null }, { "name": "Lissette Demay", "age": 37 }, { "name": "Lynell Demay", "age": 42 }, { "name": "Mikel Demay", "age": 17 } ] }
+{ "cid": 545, "name": "Dolores Ferer", "age": null, "address": null, "interests": [ "Coffee", "Bass", "Tennis" ], "children": [ { "name": "Bridgette Ferer", "age": null } ] }
+{ "cid": 547, "name": "Daryl Dambra", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Jacquline Dambra", "age": null }, { "name": "Seymour Dambra", "age": null } ] }
+{ "cid": 548, "name": "Elvia Duchesney", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Arcelia Duchesney", "age": 22 } ] }
+{ "cid": 549, "name": "Kathrin Cruff", "age": 63, "address": { "number": 9002, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Tennis", "Books" ], "children": [ { "name": "Candi Cruff", "age": 49 }, { "name": "Barry Cruff", "age": 17 }, { "name": "Shane Cruff", "age": 18 }, { "name": "Brendon Cruff", "age": null } ] }
+{ "cid": 550, "name": "Aleisha Brehon", "age": 61, "address": { "number": 7835, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Squash" ], "children": [ { "name": "Vito Brehon", "age": null }, { "name": "Matthew Brehon", "age": 32 } ] }
+{ "cid": 552, "name": "Marlena Humann", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 553, "name": "Mina Ciminera", "age": null, "address": null, "interests": [ "Base Jumping", "Databases" ], "children": [ { "name": "Cornelius Ciminera", "age": null }, { "name": "Rozanne Ciminera", "age": null }, { "name": "Byron Ciminera", "age": null } ] }
+{ "cid": 554, "name": "Darci Yafai", "age": 60, "address": { "number": 4694, "street": "Park St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Lecia Yafai", "age": 47 } ] }
+{ "cid": 555, "name": "Agustina Bretthauer", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Arthur Bretthauer", "age": 33 }, { "name": "Titus Bretthauer", "age": 33 }, { "name": "Margret Bretthauer", "age": null } ] }
+{ "cid": 557, "name": "Kaitlyn Hilleman", "age": 61, "address": { "number": 1076, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Corrie Hilleman", "age": 31 }, { "name": "Jovan Hilleman", "age": null }, { "name": "Carmine Hilleman", "age": null } ] }
+{ "cid": 559, "name": "Carolyne Shiroma", "age": null, "address": null, "interests": [ "Movies", "Running" ], "children": [ { "name": "Ying Shiroma", "age": 57 } ] }
+{ "cid": 560, "name": "Karin Dicesare", "age": null, "address": null, "interests": [ "Wine", "Puzzles" ], "children": [  ] }
+{ "cid": 561, "name": "Renetta Cudworth", "age": null, "address": null, "interests": [ "Skiing", "Basketball" ], "children": [  ] }
+{ "cid": 563, "name": "Deirdre Landero", "age": null, "address": null, "interests": [ "Books", "Fishing", "Video Games" ], "children": [ { "name": "Norman Landero", "age": 59 }, { "name": "Jennine Landero", "age": 45 }, { "name": "Rutha Landero", "age": 19 }, { "name": "Jackie Landero", "age": 29 } ] }
+{ "cid": 564, "name": "Inger Dargin", "age": 56, "address": { "number": 8704, "street": "View St.", "city": "Mountain View" }, "interests": [ "Wine", "Running", "Computers" ], "children": [  ] }
+{ "cid": 565, "name": "Shantell Rima", "age": 82, "address": { "number": 205, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Boyce Rima", "age": 67 }, { "name": "Woodrow Rima", "age": 18 }, { "name": "Helene Rima", "age": null }, { "name": "David Rima", "age": null } ] }
+{ "cid": 566, "name": "Asley Grow", "age": null, "address": null, "interests": [ "Coffee", "Books", "Tennis" ], "children": [ { "name": "Dale Grow", "age": null } ] }
+{ "cid": 567, "name": "Peggie Madhavan", "age": null, "address": null, "interests": [ "Computers", "Bass" ], "children": [  ] }
+{ "cid": 569, "name": "Beata Diles", "age": 88, "address": { "number": 2198, "street": "Park St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Myrtice Diles", "age": 46 }, { "name": "Stella Diles", "age": null }, { "name": "Rowena Diles", "age": 26 } ] }
+{ "cid": 570, "name": "Lee Basora", "age": null, "address": null, "interests": [ "Squash", "Cigars" ], "children": [  ] }
+{ "cid": 571, "name": "Lenita Tentler", "age": null, "address": null, "interests": [ "Running", "Fishing" ], "children": [ { "name": "Damian Tentler", "age": 16 }, { "name": "Camellia Tentler", "age": null }, { "name": "Vern Tentler", "age": 15 } ] }
+{ "cid": 572, "name": "Darcy Polycarpe", "age": 35, "address": { "number": 8051, "street": "View St.", "city": "Mountain View" }, "interests": [ "Computers", "Coffee", "Walking", "Walking" ], "children": [ { "name": "Kenneth Polycarpe", "age": null } ] }
+{ "cid": 573, "name": "Tyree Ketcher", "age": null, "address": null, "interests": [ "Computers", "Walking" ], "children": [ { "name": "Aleisha Ketcher", "age": null }, { "name": "Vonda Ketcher", "age": null }, { "name": "Cyndy Ketcher", "age": 13 }, { "name": "Chassidy Ketcher", "age": 30 } ] }
+{ "cid": 574, "name": "Camellia Toxey", "age": 52, "address": { "number": 5437, "street": "Hill St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Deandrea Toxey", "age": null }, { "name": "Danille Toxey", "age": null } ] }
+{ "cid": 577, "name": "Alejandro Oblinger", "age": null, "address": null, "interests": [ "Movies", "Movies" ], "children": [ { "name": "Tenesha Oblinger", "age": 56 }, { "name": "Loni Oblinger", "age": 12 }, { "name": "Sherryl Oblinger", "age": null } ] }
+{ "cid": 578, "name": "Dolly Delphia", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Sharron Delphia", "age": null }, { "name": "Shemeka Delphia", "age": null }, { "name": "Rachael Delphia", "age": null } ] }
+{ "cid": 579, "name": "Sabra Yuenger", "age": 45, "address": { "number": 2681, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Eddie Yuenger", "age": null } ] }
+{ "cid": 581, "name": "Leigha Finkenbinder", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lorine Finkenbinder", "age": 29 }, { "name": "Stephanie Finkenbinder", "age": 28 } ] }
+{ "cid": 582, "name": "Suzie Ocallahan", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Tamra Ocallahan", "age": null } ] }
+{ "cid": 583, "name": "Bev Yerena", "age": null, "address": null, "interests": [ "Puzzles", "Wine" ], "children": [ { "name": "Larhonda Yerena", "age": 45 }, { "name": "Josefina Yerena", "age": null }, { "name": "Sydney Yerena", "age": 42 } ] }
+{ "cid": 584, "name": "Bailey Janes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marylou Janes", "age": null }, { "name": "Andra Janes", "age": null } ] }
+{ "cid": 585, "name": "Young Drube", "age": 21, "address": { "number": 6960, "street": "View St.", "city": "Seattle" }, "interests": [ "Basketball", "Fishing", "Walking" ], "children": [ { "name": "Irwin Drube", "age": null }, { "name": "Gustavo Drube", "age": null } ] }
+{ "cid": 586, "name": "Jeannine Donnerberg", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Mike Donnerberg", "age": null } ] }
+{ "cid": 587, "name": "Santos Monterio", "age": 36, "address": { "number": 4454, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Databases", "Music", "Cooking" ], "children": [ { "name": "Lashonda Monterio", "age": null } ] }
+{ "cid": 588, "name": "Debora Laughinghouse", "age": 87, "address": { "number": 5099, "street": "View St.", "city": "San Jose" }, "interests": [ "Tennis", "Walking", "Databases" ], "children": [ { "name": "Frederica Laughinghouse", "age": 59 }, { "name": "Johnie Laughinghouse", "age": 12 }, { "name": "Numbers Laughinghouse", "age": 73 } ] }
+{ "cid": 589, "name": "Rebeca Blackwell", "age": 66, "address": { "number": 5708, "street": "View St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 590, "name": "Joye Burton", "age": null, "address": null, "interests": [ "Bass", "Base Jumping" ], "children": [ { "name": "Noemi Burton", "age": 19 }, { "name": "Hulda Burton", "age": null }, { "name": "Cleotilde Burton", "age": null }, { "name": "Dara Burton", "age": null } ] }
+{ "cid": 591, "name": "Matthew Tenhaeff", "age": null, "address": null, "interests": [ "Databases", "Video Games" ], "children": [ { "name": "Jan Tenhaeff", "age": 25 }, { "name": "Nana Tenhaeff", "age": null }, { "name": "Laticia Tenhaeff", "age": null }, { "name": "Ara Tenhaeff", "age": 44 } ] }
+{ "cid": 592, "name": "Rachelle Spare", "age": 13, "address": { "number": 8088, "street": "Oak St.", "city": "Portland" }, "interests": [ "Squash", "Puzzles" ], "children": [ { "name": "Theo Spare", "age": null }, { "name": "Shizue Spare", "age": null } ] }
+{ "cid": 593, "name": "Danial Pittillo", "age": 87, "address": { "number": 815, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Base Jumping" ], "children": [ { "name": "Neva Pittillo", "age": 28 }, { "name": "Brooks Pittillo", "age": null }, { "name": "Randell Pittillo", "age": 52 }, { "name": "Allyson Pittillo", "age": 51 } ] }
+{ "cid": 594, "name": "Zenia Corban", "age": null, "address": null, "interests": [ "Puzzles", "Computers", "Video Games", "Cigars" ], "children": [ { "name": "Arielle Corban", "age": null }, { "name": "Arthur Corban", "age": 15 }, { "name": "Taneka Corban", "age": 51 }, { "name": "Claire Corban", "age": null } ] }
+{ "cid": 595, "name": "Samuel Brawdy", "age": 28, "address": { "number": 453, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Books", "Basketball" ], "children": [ { "name": "Marlen Brawdy", "age": 14 }, { "name": "Lorine Brawdy", "age": 13 }, { "name": "Brad Brawdy", "age": null } ] }
+{ "cid": 596, "name": "Juliane Maddy", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Walking", "Basketball" ], "children": [ { "name": "Joannie Maddy", "age": null }, { "name": "Penny Maddy", "age": 35 }, { "name": "Joette Maddy", "age": 35 }, { "name": "Karla Maddy", "age": 54 } ] }
+{ "cid": 597, "name": "Clarine Eutsey", "age": 39, "address": { "number": 9112, "street": "7th St.", "city": "Portland" }, "interests": [ "Video Games", "Cigars", "Walking" ], "children": [  ] }
+{ "cid": 598, "name": "Venus Peat", "age": null, "address": null, "interests": [ "Coffee", "Walking", "Cigars" ], "children": [ { "name": "Antonetta Peat", "age": null }, { "name": "Shane Peat", "age": null } ] }
+{ "cid": 599, "name": "Alva Molaison", "age": 87, "address": { "number": 5974, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Wine", "Squash" ], "children": [ { "name": "Milo Molaison", "age": 39 } ] }
+{ "cid": 600, "name": "Cordell Sherburn", "age": null, "address": null, "interests": [ "Squash", "Skiing", "Skiing" ], "children": [ { "name": "Shenna Sherburn", "age": 22 }, { "name": "Minna Sherburn", "age": 10 }, { "name": "Tari Sherburn", "age": null } ] }
+{ "cid": 601, "name": "Zackary Willier", "age": null, "address": null, "interests": [ "Cooking", "Databases", "Databases" ], "children": [  ] }
+{ "cid": 602, "name": "Clyde Salada", "age": 59, "address": { "number": 8316, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Movies", "Skiing", "Cooking" ], "children": [  ] }
+{ "cid": 603, "name": "Barry Corkum", "age": null, "address": null, "interests": [ "Running", "Running" ], "children": [ { "name": "Charlesetta Corkum", "age": null }, { "name": "Helaine Corkum", "age": null }, { "name": "Erinn Corkum", "age": 28 }, { "name": "Alesia Corkum", "age": 36 } ] }
+{ "cid": 605, "name": "Sue Henriksen", "age": 78, "address": { "number": 7208, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Lauretta Henriksen", "age": null }, { "name": "Leigh Henriksen", "age": 11 } ] }
+{ "cid": 606, "name": "Virgilio Liebelt", "age": 11, "address": { "number": 8348, "street": "Cedar St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Stanford Liebelt", "age": null }, { "name": "Delaine Liebelt", "age": null }, { "name": "Kevin Liebelt", "age": null }, { "name": "Michaele Liebelt", "age": null } ] }
+{ "cid": 607, "name": "Bert Garigliano", "age": 71, "address": { "number": 3881, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Walking", "Wine" ], "children": [ { "name": "Junior Garigliano", "age": 42 }, { "name": "Willa Garigliano", "age": 21 }, { "name": "Carlo Garigliano", "age": null } ] }
+{ "cid": 608, "name": "Bruce Stanley", "age": 39, "address": { "number": 4532, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Tennis" ], "children": [  ] }
+{ "cid": 609, "name": "Mindi Dieudonne", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 610, "name": "Elinor Notoma", "age": 66, "address": { "number": 6763, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Coffee" ], "children": [ { "name": "Dennis Notoma", "age": null }, { "name": "Carol Notoma", "age": 21 } ] }
+{ "cid": 611, "name": "Evelyne Bassette", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Angla Bassette", "age": 13 } ] }
+{ "cid": 612, "name": "Keneth Ganie", "age": 57, "address": { "number": 7712, "street": "Washington St.", "city": "Portland" }, "interests": [ "Cigars", "Base Jumping" ], "children": [ { "name": "Connie Ganie", "age": null }, { "name": "Kamala Ganie", "age": 25 }, { "name": "Beulah Ganie", "age": 15 } ] }
+{ "cid": 613, "name": "Shanelle Leader", "age": null, "address": null, "interests": [ "Databases", "Base Jumping", "Wine", "Fishing" ], "children": [ { "name": "Florencia Leader", "age": null }, { "name": "Herbert Leader", "age": 11 }, { "name": "Jeanna Leader", "age": null } ] }
+{ "cid": 614, "name": "Wallace Chaidy", "age": null, "address": null, "interests": [ "Bass", "Movies", "Music" ], "children": [ { "name": "Refugio Chaidy", "age": null }, { "name": "Hae Chaidy", "age": 55 }, { "name": "Julian Chaidy", "age": null }, { "name": "Tabatha Chaidy", "age": null } ] }
+{ "cid": 615, "name": "Kimber Warnberg", "age": 77, "address": { "number": 1404, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Kristal Warnberg", "age": null } ] }
+{ "cid": 616, "name": "Shanda Dussault", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Darrick Dussault", "age": null } ] }
+{ "cid": 617, "name": "Jacques Gaskill", "age": null, "address": null, "interests": [ "Cigars", "Coffee", "Computers", "Wine" ], "children": [ { "name": "Angelyn Gaskill", "age": null }, { "name": "Jeanett Gaskill", "age": 40 }, { "name": "Emelda Gaskill", "age": 34 } ] }
+{ "cid": 618, "name": "Janella Hurtt", "age": null, "address": null, "interests": [ "Skiing", "Coffee", "Skiing" ], "children": [ { "name": "Lupe Hurtt", "age": 17 }, { "name": "Jae Hurtt", "age": 14 }, { "name": "Evan Hurtt", "age": 45 } ] }
+{ "cid": 619, "name": "Luanne Elmquist", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Burton Elmquist", "age": 11 }, { "name": "Melvin Elmquist", "age": null } ] }
+{ "cid": 620, "name": "Arielle Mackellar", "age": null, "address": null, "interests": [ "Cooking", "Bass" ], "children": [ { "name": "Evelin Mackellar", "age": 17 }, { "name": "Theresa Mackellar", "age": 53 }, { "name": "Ronnie Mackellar", "age": null }, { "name": "Elwanda Mackellar", "age": 54 } ] }
+{ "cid": 621, "name": "Theresa Satterthwaite", "age": 16, "address": { "number": 3249, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine", "Skiing", "Wine", "Fishing" ], "children": [ { "name": "Rickie Satterthwaite", "age": null }, { "name": "Rina Satterthwaite", "age": null } ] }
+{ "cid": 622, "name": "Telma Rives", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Maribeth Rives", "age": 42 }, { "name": "Youlanda Rives", "age": 13 }, { "name": "Trang Rives", "age": null }, { "name": "Hyun Rives", "age": null } ] }
+{ "cid": 624, "name": "Bong Lyall", "age": null, "address": null, "interests": [ "Databases", "Music", "Video Games" ], "children": [  ] }
+{ "cid": 625, "name": "Gale Marrazzo", "age": 25, "address": { "number": 2307, "street": "View St.", "city": "San Jose" }, "interests": [ "Fishing", "Base Jumping", "Walking", "Cooking" ], "children": [ { "name": "Coleman Marrazzo", "age": null }, { "name": "Frances Marrazzo", "age": null }, { "name": "Camellia Marrazzo", "age": 11 } ] }
+{ "cid": 626, "name": "Sydney Josten", "age": 44, "address": { "number": 4815, "street": "Hill St.", "city": "Sunnyvale" }, "interests": [ "Cigars" ], "children": [ { "name": "Basil Josten", "age": 14 }, { "name": "Yasuko Josten", "age": null } ] }
+{ "cid": 627, "name": "Fernande Ede", "age": 75, "address": { "number": 9316, "street": "Cedar St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Rebeca Ede", "age": null }, { "name": "Raymond Ede", "age": 57 } ] }
+{ "cid": 628, "name": "Tomoko Alcantara", "age": 56, "address": { "number": 3556, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Running", "Tennis" ], "children": [ { "name": "Babara Alcantara", "age": 31 }, { "name": "Ilana Alcantara", "age": null }, { "name": "Maren Alcantara", "age": 45 } ] }
+{ "cid": 629, "name": "Mayola Clabo", "age": null, "address": null, "interests": [ "Basketball", "Skiing", "Running" ], "children": [ { "name": "Rigoberto Clabo", "age": 58 } ] }
+{ "cid": 630, "name": "Darla Domenick", "age": 14, "address": { "number": 3315, "street": "Park St.", "city": "San Jose" }, "interests": [ "Databases" ], "children": [ { "name": "Verda Domenick", "age": null } ] }
+{ "cid": 631, "name": "Brook Jenks", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Eldon Jenks", "age": null }, { "name": "Luann Jenks", "age": 53 }, { "name": "Aurora Jenks", "age": 37 } ] }
+{ "cid": 632, "name": "Keeley Goga", "age": null, "address": null, "interests": [ "Books", "Base Jumping" ], "children": [ { "name": "Walter Goga", "age": 39 }, { "name": "Chaya Goga", "age": null }, { "name": "Melodie Goga", "age": null }, { "name": "Isidro Goga", "age": 32 } ] }
+{ "cid": 633, "name": "Shalon Grauberger", "age": 34, "address": { "number": 765, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Base Jumping", "Tennis" ], "children": [ { "name": "Kris Grauberger", "age": 14 }, { "name": "Stuart Grauberger", "age": 12 }, { "name": "Billy Grauberger", "age": null } ] }
+{ "cid": 634, "name": "Katherina Parzych", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Modesta Parzych", "age": null }, { "name": "Darin Parzych", "age": 20 } ] }
+{ "cid": 635, "name": "Angelena Braegelmann", "age": 36, "address": { "number": 4158, "street": "Park St.", "city": "San Jose" }, "interests": [ "Wine", "Skiing" ], "children": [ { "name": "Daisey Braegelmann", "age": 18 }, { "name": "Gaston Braegelmann", "age": 19 }, { "name": "Louella Braegelmann", "age": null }, { "name": "Leonie Braegelmann", "age": null } ] }
+{ "cid": 636, "name": "Babara Shore", "age": 83, "address": { "number": 9452, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Candy Shore", "age": 58 }, { "name": "Nanci Shore", "age": null }, { "name": "Asia Shore", "age": null } ] }
+{ "cid": 639, "name": "Zena Seehusen", "age": 24, "address": { "number": 6303, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Cooking", "Movies", "Music" ], "children": [ { "name": "Hester Seehusen", "age": null }, { "name": "Coreen Seehusen", "age": 12 } ] }
+{ "cid": 640, "name": "Willy Bielak", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 642, "name": "Odell Nova", "age": 25, "address": { "number": 896, "street": "Park St.", "city": "San Jose" }, "interests": [ "Video Games", "Squash", "Music" ], "children": [ { "name": "Leopoldo Nova", "age": null }, { "name": "Rickey Nova", "age": null }, { "name": "Mike Nova", "age": 14 }, { "name": "Tamie Nova", "age": 14 } ] }
+{ "cid": 643, "name": "Juliet Skreen", "age": null, "address": null, "interests": [ "Walking" ], "children": [  ] }
+{ "cid": 644, "name": "Julio Gilly", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [ { "name": "Eleonore Gilly", "age": null } ] }
+{ "cid": 645, "name": "Shawnda Dollinger", "age": 36, "address": { "number": 5980, "street": "Park St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Vicente Dollinger", "age": null }, { "name": "Kerrie Dollinger", "age": 10 }, { "name": "Sima Dollinger", "age": 14 } ] }
+{ "cid": 646, "name": "Pablo Catterton", "age": null, "address": null, "interests": [ "Fishing", "Computers" ], "children": [  ] }
+{ "cid": 647, "name": "Jodi Dearson", "age": null, "address": null, "interests": [ "Fishing", "Movies" ], "children": [  ] }
+{ "cid": 649, "name": "Anisha Sender", "age": null, "address": null, "interests": [ "Tennis", "Databases", "Bass" ], "children": [ { "name": "Viva Sender", "age": 40 }, { "name": "Terica Sender", "age": null } ] }
+{ "cid": 650, "name": "Darrin Orengo", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Linwood Orengo", "age": 39 } ] }
+{ "cid": 651, "name": "Delana Henk", "age": 69, "address": { "number": 5497, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Coffee", "Video Games", "Databases" ], "children": [ { "name": "Loan Henk", "age": null }, { "name": "Teresa Henk", "age": 20 }, { "name": "Randell Henk", "age": null }, { "name": "Micah Henk", "age": null } ] }
+{ "cid": 652, "name": "Armida Moeuy", "age": 34, "address": { "number": 8306, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Running" ], "children": [ { "name": "Sunshine Moeuy", "age": null }, { "name": "Leta Moeuy", "age": 19 } ] }
+{ "cid": 653, "name": "Robbie Rhump", "age": null, "address": null, "interests": [ "Squash", "Computers" ], "children": [ { "name": "Alishia Rhump", "age": 14 }, { "name": "Lyndsay Rhump", "age": 27 } ] }
+{ "cid": 654, "name": "Louis Laubersheimer", "age": 76, "address": { "number": 8010, "street": "7th St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Bass", "Cooking" ], "children": [ { "name": "Jewel Laubersheimer", "age": 22 }, { "name": "Toccara Laubersheimer", "age": 45 }, { "name": "Eve Laubersheimer", "age": null } ] }
+{ "cid": 655, "name": "Shaun Brandenburg", "age": null, "address": null, "interests": [ "Skiing", "Computers", "Base Jumping" ], "children": [ { "name": "Ned Brandenburg", "age": null }, { "name": "Takako Brandenburg", "age": 41 }, { "name": "Astrid Brandenburg", "age": null }, { "name": "Patience Brandenburg", "age": null } ] }
+{ "cid": 656, "name": "Rufus Peaden", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nathanael Peaden", "age": 57 }, { "name": "Jamaal Peaden", "age": null } ] }
+{ "cid": 657, "name": "Rory Teachman", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 658, "name": "Truman Leitner", "age": null, "address": null, "interests": [ "Computers", "Bass", "Walking" ], "children": [  ] }
+{ "cid": 659, "name": "Daniel Groskreutz", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Mariam Groskreutz", "age": 21 }, { "name": "Carlton Groskreutz", "age": null } ] }
+{ "cid": 660, "name": "Israel Aday", "age": null, "address": null, "interests": [ "Wine", "Bass", "Cigars" ], "children": [ { "name": "Mi Aday", "age": null } ] }
+{ "cid": 661, "name": "Lorita Kraut", "age": 43, "address": { "number": 5017, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Tennis", "Movies", "Bass" ], "children": [ { "name": "Mirian Kraut", "age": null } ] }
+{ "cid": 662, "name": "Domonique Corbi", "age": 13, "address": { "number": 7286, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Tennis", "Cooking", "Computers" ], "children": [ { "name": "Katrice Corbi", "age": null }, { "name": "Idalia Corbi", "age": null }, { "name": "Hayley Corbi", "age": null } ] }
+{ "cid": 663, "name": "Riley Noteboom", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Marvis Noteboom", "age": 57 } ] }
+{ "cid": 665, "name": "Garnet Desai", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Aliza Desai", "age": null } ] }
+{ "cid": 666, "name": "Pamila Burzlaff", "age": 68, "address": { "number": 6543, "street": "View St.", "city": "Portland" }, "interests": [ "Squash", "Cigars", "Movies" ], "children": [  ] }
+{ "cid": 667, "name": "Shaniqua Deist", "age": null, "address": null, "interests": [ "Puzzles", "Books", "Cigars" ], "children": [  ] }
+{ "cid": 668, "name": "Dorene Spigelman", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Chiquita Spigelman", "age": 29 }, { "name": "Anisha Spigelman", "age": 34 }, { "name": "Micah Spigelman", "age": 28 } ] }
+{ "cid": 669, "name": "Royal Abke", "age": 60, "address": { "number": 1675, "street": "Main St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Leandra Abke", "age": 25 }, { "name": "Shawanna Abke", "age": null } ] }
+{ "cid": 670, "name": "Angelo Kellar", "age": 22, "address": { "number": 3178, "street": "View St.", "city": "Seattle" }, "interests": [ "Wine", "Music", "Fishing" ], "children": [ { "name": "Zula Kellar", "age": null }, { "name": "Brittaney Kellar", "age": 10 }, { "name": "Fredia Kellar", "age": null } ] }
+{ "cid": 671, "name": "Harley Emami", "age": null, "address": null, "interests": [ "Basketball" ], "children": [ { "name": "Valentine Emami", "age": null }, { "name": "Pearlene Emami", "age": null } ] }
+{ "cid": 672, "name": "Pamelia Repka", "age": 30, "address": { "number": 8837, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Coffee", "Base Jumping" ], "children": [ { "name": "Klara Repka", "age": 19 }, { "name": "Bennett Repka", "age": null }, { "name": "Randy Repka", "age": 13 }, { "name": "Ervin Repka", "age": null } ] }
+{ "cid": 673, "name": "Willard Matuszek", "age": null, "address": null, "interests": [ "Running" ], "children": [ { "name": "Kyong Matuszek", "age": null }, { "name": "Delena Matuszek", "age": null }, { "name": "Toney Matuszek", "age": null }, { "name": "Shayne Matuszek", "age": 19 } ] }
+{ "cid": 675, "name": "Camellia Brickett", "age": null, "address": null, "interests": [ "Running" ], "children": [ { "name": "Leona Brickett", "age": null }, { "name": "Mario Brickett", "age": null }, { "name": "Nadine Brickett", "age": 35 }, { "name": "Marlon Brickett", "age": 31 } ] }
+{ "cid": 676, "name": "Ima Juart", "age": 64, "address": { "number": 2498, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Walking" ], "children": [ { "name": "Cortez Juart", "age": 17 }, { "name": "Guillermo Juart", "age": null }, { "name": "Shelley Juart", "age": 20 }, { "name": "Daryl Juart", "age": null } ] }
+{ "cid": 677, "name": "Brigid Sarabia", "age": 89, "address": { "number": 918, "street": "Park St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Elisa Sarabia", "age": null }, { "name": "Pura Sarabia", "age": 56 } ] }
+{ "cid": 678, "name": "Lekisha Barnell", "age": null, "address": null, "interests": [ "Movies", "Skiing", "Running" ], "children": [ { "name": "August Barnell", "age": null }, { "name": "Tiffany Barnell", "age": 55 }, { "name": "Meghan Barnell", "age": null } ] }
+{ "cid": 680, "name": "Domenica Qunnarath", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 681, "name": "Iliana Nagele", "age": null, "address": null, "interests": [ "Movies", "Running" ], "children": [ { "name": "Sunny Nagele", "age": 55 }, { "name": "Waltraud Nagele", "age": 39 }, { "name": "Darron Nagele", "age": null } ] }
+{ "cid": 682, "name": "Krystle Weingartner", "age": 87, "address": { "number": 5293, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Squash" ], "children": [ { "name": "Bryanna Weingartner", "age": 19 }, { "name": "Rubie Weingartner", "age": 32 }, { "name": "Raye Weingartner", "age": null } ] }
+{ "cid": 683, "name": "Dodie Crall", "age": 37, "address": { "number": 1337, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Cassy Crall", "age": null }, { "name": "Thu Crall", "age": 19 } ] }
+{ "cid": 684, "name": "Elmo Ballenger", "age": 69, "address": { "number": 2657, "street": "Park St.", "city": "Seattle" }, "interests": [ "Wine" ], "children": [ { "name": "Sheena Ballenger", "age": 53 }, { "name": "Abby Ballenger", "age": null }, { "name": "Markus Ballenger", "age": null } ] }
+{ "cid": 685, "name": "Lois Mcglothian", "age": null, "address": null, "interests": [ "Movies", "Skiing" ], "children": [ { "name": "Karon Mcglothian", "age": 35 } ] }
+{ "cid": 686, "name": "Trudi Arnette", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Adrian Arnette", "age": 43 }, { "name": "Hulda Arnette", "age": 34 }, { "name": "Shamika Arnette", "age": null } ] }
+{ "cid": 687, "name": "Adriene Glowinski", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 688, "name": "Maryellen Leriche", "age": null, "address": null, "interests": [ "Music", "Walking", "Skiing" ], "children": [ { "name": "Dorinda Leriche", "age": 27 } ] }
+{ "cid": 689, "name": "Camila Cho", "age": 70, "address": { "number": 7731, "street": "Cedar St.", "city": "Mountain View" }, "interests": [ "Video Games", "Cigars" ], "children": [ { "name": "Myrtie Cho", "age": 57 }, { "name": "Merideth Cho", "age": 45 }, { "name": "Meta Cho", "age": 20 } ] }
+{ "cid": 691, "name": "Sharee Charrier", "age": 17, "address": { "number": 6693, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Puzzles", "Cooking", "Bass" ], "children": [ { "name": "Odessa Charrier", "age": null } ] }
+{ "cid": 692, "name": "Nida Picknell", "age": 24, "address": { "number": 9053, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Skiing", "Music", "Wine", "Base Jumping" ], "children": [ { "name": "Caroyln Picknell", "age": null }, { "name": "Micheline Picknell", "age": 10 } ] }
+{ "cid": 693, "name": "Ela Crisan", "age": null, "address": null, "interests": [ "Movies" ], "children": [  ] }
+{ "cid": 694, "name": "Ariel Soltani", "age": null, "address": null, "interests": [ "Databases", "Music", "Puzzles" ], "children": [ { "name": "Aldo Soltani", "age": null }, { "name": "Anglea Soltani", "age": null } ] }
+{ "cid": 695, "name": "Wyatt Eveleth", "age": 28, "address": { "number": 5421, "street": "View St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Orval Eveleth", "age": null }, { "name": "Beth Eveleth", "age": 11 }, { "name": "Yuki Eveleth", "age": null }, { "name": "Alyse Eveleth", "age": 14 } ] }
+{ "cid": 696, "name": "Nadia Dunklee", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Mendy Dunklee", "age": 17 }, { "name": "Edgar Dunklee", "age": null }, { "name": "Pasquale Dunklee", "age": null }, { "name": "Colin Dunklee", "age": null } ] }
+{ "cid": 697, "name": "Claud Coffel", "age": 72, "address": { "number": 8483, "street": "Cedar St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Katheleen Coffel", "age": 38 }, { "name": "Tashina Coffel", "age": null } ] }
+{ "cid": 698, "name": "Tawanna Zanin", "age": 60, "address": { "number": 7979, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Denny Zanin", "age": 31 }, { "name": "Danial Zanin", "age": 43 }, { "name": "Kenyetta Zanin", "age": null }, { "name": "Aleisha Zanin", "age": null } ] }
+{ "cid": 699, "name": "Lyda Golomb", "age": 46, "address": { "number": 5049, "street": "Main St.", "city": "Seattle" }, "interests": [ "Fishing", "Basketball" ], "children": [ { "name": "Shonta Golomb", "age": null }, { "name": "Lynwood Golomb", "age": 26 }, { "name": "Leonila Golomb", "age": 30 }, { "name": "Alejandrina Golomb", "age": null } ] }
+{ "cid": 700, "name": "Suk Blondin", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Brenton Blondin", "age": null }, { "name": "Charlotte Blondin", "age": null }, { "name": "Eldon Blondin", "age": 10 }, { "name": "Leanne Blondin", "age": null } ] }
+{ "cid": 702, "name": "Lane Krog", "age": 50, "address": { "number": 1646, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Running" ], "children": [ { "name": "Carri Krog", "age": null }, { "name": "Sage Krog", "age": null }, { "name": "Bronwyn Krog", "age": null } ] }
+{ "cid": 703, "name": "Susanne Pettey", "age": null, "address": null, "interests": [ "Squash", "Basketball", "Skiing" ], "children": [ { "name": "Nancey Pettey", "age": 35 }, { "name": "Lawana Pettey", "age": null }, { "name": "Percy Pettey", "age": 25 } ] }
+{ "cid": 704, "name": "Melodee Clemons", "age": null, "address": null, "interests": [ "Base Jumping", "Tennis", "Video Games" ], "children": [ { "name": "Doreatha Clemons", "age": 22 } ] }
+{ "cid": 705, "name": "Sofia Bonniwell", "age": 81, "address": { "number": 767, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Basketball" ], "children": [ { "name": "Douglass Bonniwell", "age": 58 }, { "name": "Jackeline Bonniwell", "age": 16 } ] }
+{ "cid": 706, "name": "Miquel Caesar", "age": 16, "address": { "number": 2176, "street": "Park St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Shaniqua Caesar", "age": null }, { "name": "Ellis Caesar", "age": null }, { "name": "Bruna Caesar", "age": null }, { "name": "Kayleen Caesar", "age": null } ] }
+{ "cid": 708, "name": "Elease Holtmann", "age": 75, "address": { "number": 5295, "street": "Washington St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Leonardo Holtmann", "age": null }, { "name": "Katharine Holtmann", "age": null }, { "name": "Chung Holtmann", "age": 20 }, { "name": "Teodoro Holtmann", "age": 19 } ] }
+{ "cid": 709, "name": "Jazmine Twiddy", "age": null, "address": null, "interests": [ "Puzzles", "Computers", "Wine" ], "children": [ { "name": "Veronika Twiddy", "age": 21 } ] }
+{ "cid": 710, "name": "Arlen Horka", "age": null, "address": null, "interests": [ "Movies", "Coffee", "Walking" ], "children": [ { "name": "Valencia Horka", "age": null }, { "name": "Wesley Horka", "age": null } ] }
+{ "cid": 711, "name": "Agnes Andreas", "age": null, "address": null, "interests": [ "Books" ], "children": [ { "name": "Fairy Andreas", "age": null }, { "name": "Wilhemina Andreas", "age": null }, { "name": "Parthenia Andreas", "age": 53 }, { "name": "Maye Andreas", "age": null } ] }
+{ "cid": 712, "name": "Jack Lamoreux", "age": 32, "address": { "number": 4486, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Rubin Lamoreux", "age": 15 }, { "name": "Jonelle Lamoreux", "age": 10 }, { "name": "Shonna Lamoreux", "age": null }, { "name": "India Lamoreux", "age": 17 } ] }
+{ "cid": 713, "name": "Galina Retterbush", "age": null, "address": null, "interests": [ "Bass", "Squash" ], "children": [ { "name": "Janene Retterbush", "age": null }, { "name": "Toby Retterbush", "age": 15 }, { "name": "Renato Retterbush", "age": null }, { "name": "Annice Retterbush", "age": 22 } ] }
+{ "cid": 715, "name": "Zoraida Scribner", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Ninfa Scribner", "age": 31 } ] }
+{ "cid": 716, "name": "Deirdre Bruderer", "age": null, "address": null, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Coralee Bruderer", "age": null }, { "name": "Mina Bruderer", "age": null }, { "name": "Lindsey Bruderer", "age": 35 }, { "name": "Yi Bruderer", "age": null } ] }
+{ "cid": 717, "name": "Paulette Moccasin", "age": 87, "address": { "number": 1426, "street": "View St.", "city": "Portland" }, "interests": [ "Fishing" ], "children": [ { "name": "Savannah Moccasin", "age": null }, { "name": "Mariela Moccasin", "age": 34 }, { "name": "Isadora Moccasin", "age": null }, { "name": "Vivien Moccasin", "age": 31 } ] }
+{ "cid": 718, "name": "Tandy Trick", "age": 18, "address": { "number": 1215, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Fishing", "Fishing" ], "children": [ { "name": "Edyth Trick", "age": null }, { "name": "Jimmy Trick", "age": null }, { "name": "Jacquline Trick", "age": null }, { "name": "Tyler Trick", "age": null } ] }
+{ "cid": 719, "name": "Antoinette Boursiquot", "age": 47, "address": { "number": 3652, "street": "Cedar St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Dennis Boursiquot", "age": null }, { "name": "Katelyn Boursiquot", "age": null }, { "name": "Gabrielle Boursiquot", "age": null }, { "name": "Deidre Boursiquot", "age": null } ] }
+{ "cid": 721, "name": "Jesica Tinder", "age": 28, "address": { "number": 5526, "street": "7th St.", "city": "Mountain View" }, "interests": [  ], "children": [  ] }
+{ "cid": 723, "name": "Teressa Krol", "age": 22, "address": { "number": 8036, "street": "Park St.", "city": "Seattle" }, "interests": [ "Music" ], "children": [ { "name": "Tuan Krol", "age": null }, { "name": "Judi Krol", "age": null }, { "name": "Maddie Krol", "age": null } ] }
+{ "cid": 724, "name": "Merle Bakula", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Margart Bakula", "age": 49 }, { "name": "Mathew Bakula", "age": 36 } ] }
+{ "cid": 725, "name": "Sallie Calderon", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 726, "name": "Brinda Raudebaugh", "age": 83, "address": { "number": 7179, "street": "View St.", "city": "Mountain View" }, "interests": [  ], "children": [  ] }
+{ "cid": 727, "name": "Valene Resecker", "age": null, "address": null, "interests": [ "Music", "Wine", "Books", "Walking" ], "children": [  ] }
+{ "cid": 728, "name": "Bruno Freeburger", "age": 84, "address": { "number": 2482, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Computers" ], "children": [ { "name": "Shizuko Freeburger", "age": null } ] }
+{ "cid": 730, "name": "Marti Vandoren", "age": null, "address": null, "interests": [ "Skiing", "Bass" ], "children": [ { "name": "Carroll Vandoren", "age": null }, { "name": "Lorretta Vandoren", "age": 30 }, { "name": "Chloe Vandoren", "age": 42 }, { "name": "Ilona Vandoren", "age": null } ] }
+{ "cid": 731, "name": "Yajaira Orto", "age": null, "address": null, "interests": [ "Music", "Databases" ], "children": [ { "name": "Eliz Orto", "age": 17 }, { "name": "Gisela Orto", "age": null } ] }
+{ "cid": 732, "name": "Dania Fabio", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Virgie Fabio", "age": null }, { "name": "Nereida Fabio", "age": 37 } ] }
+{ "cid": 733, "name": "Edie Stager", "age": 26, "address": { "number": 2691, "street": "Park St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Ethyl Stager", "age": 10 } ] }
+{ "cid": 734, "name": "Lera Korn", "age": null, "address": null, "interests": [ "Tennis", "Puzzles", "Cigars" ], "children": [ { "name": "Criselda Korn", "age": 37 } ] }
+{ "cid": 736, "name": "Desmond Branam", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Manuel Branam", "age": 51 } ] }
+{ "cid": 737, "name": "Jeffrey Chesson", "age": 13, "address": { "number": 6833, "street": "Lake St.", "city": "Portland" }, "interests": [ "Tennis", "Computers" ], "children": [ { "name": "Clayton Chesson", "age": null }, { "name": "Yi Chesson", "age": null } ] }
+{ "cid": 738, "name": "Josphine Rohrer", "age": 75, "address": { "number": 862, "street": "Main St.", "city": "Los Angeles" }, "interests": [ "Databases" ], "children": [ { "name": "Marvin Rohrer", "age": 22 }, { "name": "Wyatt Rohrer", "age": null }, { "name": "Deloras Rohrer", "age": null } ] }
+{ "cid": 739, "name": "Libbie Thigpin", "age": null, "address": null, "interests": [ "Databases" ], "children": [  ] }
+{ "cid": 740, "name": "Thomasine Collado", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Tabetha Collado", "age": null }, { "name": "Alline Collado", "age": null }, { "name": "Delisa Collado", "age": null }, { "name": "Jack Collado", "age": 56 } ] }
+{ "cid": 741, "name": "Lesia Risatti", "age": 48, "address": { "number": 7378, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Fishing", "Wine", "Databases" ], "children": [ { "name": "Tangela Risatti", "age": null }, { "name": "Leonel Risatti", "age": 33 }, { "name": "Cythia Risatti", "age": 36 } ] }
+{ "cid": 742, "name": "Andy Schifo", "age": 36, "address": { "number": 4422, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 743, "name": "Nona Debroux", "age": null, "address": null, "interests": [ "Bass" ], "children": [  ] }
+{ "cid": 744, "name": "Crysta Christen", "age": 57, "address": { "number": 439, "street": "Hill St.", "city": "Portland" }, "interests": [ "Basketball", "Squash", "Base Jumping" ], "children": [  ] }
+{ "cid": 745, "name": "Tabatha Hagwell", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Gaynell Hagwell", "age": null } ] }
+{ "cid": 746, "name": "Rosalinda Pola", "age": null, "address": null, "interests": [ "Cooking", "Computers", "Walking", "Cigars" ], "children": [ { "name": "Maribel Pola", "age": 19 }, { "name": "Chaya Pola", "age": null }, { "name": "Shauna Pola", "age": null }, { "name": "Elenora Pola", "age": 22 } ] }
+{ "cid": 747, "name": "Gil Dunnaway", "age": 65, "address": { "number": 3022, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Running", "Squash" ], "children": [ { "name": "Laurice Dunnaway", "age": null } ] }
+{ "cid": 748, "name": "Petra Ganes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Perry Ganes", "age": null }, { "name": "Krista Ganes", "age": 54 }, { "name": "Kayce Ganes", "age": 52 }, { "name": "Eleni Ganes", "age": null } ] }
+{ "cid": 749, "name": "Pearle Mauney", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Delpha Mauney", "age": null }, { "name": "Micki Mauney", "age": 28 }, { "name": "Wayne Mauney", "age": null } ] }
+{ "cid": 750, "name": "Rosaura Gaul", "age": null, "address": null, "interests": [ "Music", "Books", "Tennis" ], "children": [ { "name": "Letisha Gaul", "age": 41 } ] }
+{ "cid": 751, "name": "Lydia Iannelli", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Teri Iannelli", "age": 36 } ] }
+{ "cid": 752, "name": "Maria Lebovic", "age": null, "address": null, "interests": [ "Bass" ], "children": [ { "name": "Thi Lebovic", "age": null }, { "name": "Rosamaria Lebovic", "age": 23 }, { "name": "Brinda Lebovic", "age": 39 } ] }
+{ "cid": 753, "name": "Maris Bannett", "age": null, "address": null, "interests": [ "Fishing", "Cigars", "Running" ], "children": [ { "name": "Libbie Bannett", "age": 11 }, { "name": "Francina Bannett", "age": 21 }, { "name": "Tuyet Bannett", "age": null }, { "name": "Zona Bannett", "age": 32 } ] }
+{ "cid": 754, "name": "Luetta Joern", "age": 25, "address": { "number": 5554, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Hildegarde Joern", "age": null }, { "name": "Lorenza Joern", "age": 13 } ] }
+{ "cid": 755, "name": "Bette Trentz", "age": 57, "address": { "number": 2794, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Christa Trentz", "age": 14 }, { "name": "Jestine Trentz", "age": 22 }, { "name": "Shantel Trentz", "age": 37 }, { "name": "Jacklyn Trentz", "age": null } ] }
+{ "cid": 756, "name": "Marisol Noyes", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Delora Noyes", "age": null }, { "name": "Jonelle Noyes", "age": 44 } ] }
+{ "cid": 758, "name": "Akiko Hoenstine", "age": 56, "address": { "number": 8888, "street": "Lake St.", "city": "Portland" }, "interests": [ "Movies", "Walking" ], "children": [ { "name": "Maren Hoenstine", "age": null }, { "name": "Tyler Hoenstine", "age": null }, { "name": "Jesse Hoenstine", "age": 40 } ] }
+{ "cid": 759, "name": "Alaina Dadds", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Athena Dadds", "age": 36 }, { "name": "Denis Dadds", "age": null }, { "name": "Nathanial Dadds", "age": 42 }, { "name": "Molly Dadds", "age": null } ] }
+{ "cid": 761, "name": "Adele Henrikson", "age": null, "address": null, "interests": [ "Cooking", "Bass" ], "children": [ { "name": "Paulina Henrikson", "age": null }, { "name": "David Henrikson", "age": null }, { "name": "Jose Henrikson", "age": null }, { "name": "Meg Henrikson", "age": null } ] }
+{ "cid": 763, "name": "Candis Deya", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Lise Deya", "age": null }, { "name": "Jeni Deya", "age": 52 }, { "name": "Domonique Deya", "age": 24 }, { "name": "Rubie Deya", "age": null } ] }
+{ "cid": 766, "name": "Tosha Loffredo", "age": 64, "address": { "number": 5580, "street": "View St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [ { "name": "Hellen Loffredo", "age": 32 } ] }
+{ "cid": 767, "name": "Wendi Hoecker", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 768, "name": "Adelina Troendle", "age": null, "address": null, "interests": [ "Computers" ], "children": [ { "name": "Lenna Troendle", "age": 51 }, { "name": "Ines Troendle", "age": 48 }, { "name": "Ora Troendle", "age": null } ] }
+{ "cid": 769, "name": "Isaias Tenny", "age": 71, "address": { "number": 270, "street": "Park St.", "city": "Portland" }, "interests": [ "Wine", "Fishing", "Base Jumping" ], "children": [ { "name": "Theo Tenny", "age": null }, { "name": "Shena Tenny", "age": null }, { "name": "Coralee Tenny", "age": null }, { "name": "Orval Tenny", "age": 39 } ] }
+{ "cid": 770, "name": "Merrill Tilson", "age": null, "address": null, "interests": [ "Computers", "Skiing" ], "children": [ { "name": "Elna Tilson", "age": null } ] }
+{ "cid": 771, "name": "Marisela Tredo", "age": null, "address": null, "interests": [ "Tennis", "Coffee" ], "children": [ { "name": "Ardell Tredo", "age": 21 }, { "name": "Evelynn Tredo", "age": 16 } ] }
+{ "cid": 773, "name": "Leatrice Zysett", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Bee Zysett", "age": 30 }, { "name": "Russ Zysett", "age": 11 }, { "name": "Jeff Zysett", "age": 39 }, { "name": "Herman Zysett", "age": 27 } ] }
+{ "cid": 774, "name": "Nadene Rigel", "age": null, "address": null, "interests": [ "Cigars", "Cigars" ], "children": [ { "name": "Rebbeca Rigel", "age": 33 } ] }
+{ "cid": 776, "name": "Dagmar Sarkis", "age": null, "address": null, "interests": [ "Basketball", "Running", "Wine" ], "children": [ { "name": "Tari Sarkis", "age": null }, { "name": "Rana Sarkis", "age": 56 }, { "name": "Merissa Sarkis", "age": null }, { "name": "Lori Sarkis", "age": 26 } ] }
+{ "cid": 777, "name": "Coralee Vaugh", "age": 51, "address": { "number": 4130, "street": "Hill St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Dean Vaugh", "age": 31 }, { "name": "Stanton Vaugh", "age": 39 }, { "name": "Marti Vaugh", "age": 33 }, { "name": "Eden Vaugh", "age": 27 } ] }
+{ "cid": 778, "name": "Shellie Sario", "age": null, "address": null, "interests": [ "Puzzles" ], "children": [  ] }
+{ "cid": 779, "name": "Vinita Bockskopf", "age": null, "address": null, "interests": [ "Tennis", "Video Games" ], "children": [  ] }
+{ "cid": 780, "name": "Penny Poortinga", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Estella Poortinga", "age": null } ] }
+{ "cid": 781, "name": "Christy Darcangelo", "age": 42, "address": { "number": 2178, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Fishing" ], "children": [ { "name": "Luis Darcangelo", "age": 21 }, { "name": "Omega Darcangelo", "age": 26 }, { "name": "Remedios Darcangelo", "age": 28 }, { "name": "Domenic Darcangelo", "age": 21 } ] }
+{ "cid": 782, "name": "Shameka Haifa", "age": 16, "address": { "number": 9555, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Cigars", "Computers", "Coffee", "Skiing" ], "children": [ { "name": "Dannette Haifa", "age": null } ] }
+{ "cid": 783, "name": "Johnnie Kesby", "age": 56, "address": { "number": 9798, "street": "View St.", "city": "Seattle" }, "interests": [ "Puzzles", "Tennis" ], "children": [  ] }
+{ "cid": 784, "name": "Omar Hasen", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Hugh Hasen", "age": null } ] }
+{ "cid": 785, "name": "Gabriel Breidel", "age": 32, "address": { "number": 9288, "street": "Park St.", "city": "San Jose" }, "interests": [ "Cigars", "Bass" ], "children": [ { "name": "Bernie Breidel", "age": null } ] }
+{ "cid": 786, "name": "Johnsie Maheux", "age": null, "address": null, "interests": [ "Cigars" ], "children": [ { "name": "Danuta Maheux", "age": null } ] }
+{ "cid": 787, "name": "Sara Yerly", "age": 12, "address": { "number": 872, "street": "7th St.", "city": "Seattle" }, "interests": [ "Fishing" ], "children": [ { "name": "Nettie Yerly", "age": null }, { "name": "Regine Yerly", "age": null }, { "name": "Hyo Yerly", "age": null } ] }
+{ "cid": 789, "name": "Carli Notto", "age": null, "address": null, "interests": [ "Cigars" ], "children": [  ] }
+{ "cid": 790, "name": "Dustin Brumble", "age": null, "address": null, "interests": [ "Computers", "Databases", "Tennis" ], "children": [ { "name": "Oda Brumble", "age": null }, { "name": "Jennefer Brumble", "age": 26 }, { "name": "Ricardo Brumble", "age": 37 }, { "name": "Graciela Brumble", "age": 10 } ] }
+{ "cid": 791, "name": "Jame Apresa", "age": 66, "address": { "number": 8417, "street": "Main St.", "city": "San Jose" }, "interests": [ "Running", "Puzzles", "Base Jumping" ], "children": [ { "name": "Awilda Apresa", "age": null }, { "name": "Nelle Apresa", "age": 40 }, { "name": "Terrell Apresa", "age": null }, { "name": "Malia Apresa", "age": 43 } ] }
+{ "cid": 793, "name": "Shondra Gollman", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Paul Gollman", "age": 30 }, { "name": "Katherina Gollman", "age": 53 } ] }
+{ "cid": 794, "name": "Annabel Leins", "age": 75, "address": { "number": 9761, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Bass", "Computers", "Bass", "Cigars" ], "children": [ { "name": "Oswaldo Leins", "age": 21 } ] }
+{ "cid": 795, "name": "Sharilyn Branstad", "age": null, "address": null, "interests": [ "Databases", "Music" ], "children": [ { "name": "Ashlee Branstad", "age": 24 }, { "name": "Bobbye Branstad", "age": 26 }, { "name": "Natalya Branstad", "age": null }, { "name": "Edith Branstad", "age": null } ] }
+{ "cid": 796, "name": "Daniele Brisk", "age": null, "address": null, "interests": [ "Walking", "Bass" ], "children": [  ] }
+{ "cid": 797, "name": "Frederica Kale", "age": 77, "address": { "number": 6861, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Puzzles", "Bass" ], "children": [ { "name": "Shanice Kale", "age": null }, { "name": "Soraya Kale", "age": 64 }, { "name": "Laurena Kale", "age": 57 } ] }
+{ "cid": 799, "name": "Ronny Piefer", "age": 45, "address": { "number": 7724, "street": "7th St.", "city": "Mountain View" }, "interests": [ "Fishing" ], "children": [ { "name": "Chantal Piefer", "age": 24 }, { "name": "Tiffany Piefer", "age": null }, { "name": "Farrah Piefer", "age": 21 }, { "name": "Dee Piefer", "age": null } ] }
+{ "cid": 800, "name": "Karon Johnsen", "age": null, "address": null, "interests": [ "Movies" ], "children": [ { "name": "Roselee Johnsen", "age": 25 } ] }
+{ "cid": 802, "name": "Sang Hollman", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Carman Hollman", "age": null }, { "name": "Kirstie Hollman", "age": 40 }, { "name": "Jacquetta Hollman", "age": null } ] }
+{ "cid": 803, "name": "Yolonda Korf", "age": null, "address": null, "interests": [ "Bass", "Skiing", "Music" ], "children": [ { "name": "Ivette Korf", "age": null }, { "name": "Lashon Korf", "age": null } ] }
+{ "cid": 804, "name": "Joaquina Burlin", "age": 77, "address": { "number": 5479, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Running", "Wine", "Running" ], "children": [  ] }
+{ "cid": 805, "name": "Gaylord Ginder", "age": null, "address": null, "interests": [ "Databases", "Coffee" ], "children": [ { "name": "Lucina Ginder", "age": null }, { "name": "Harriett Ginder", "age": null } ] }
+{ "cid": 806, "name": "Corliss Sharratt", "age": null, "address": null, "interests": [ "Basketball", "Cigars", "Cooking" ], "children": [ { "name": "Albertine Sharratt", "age": null }, { "name": "Nobuko Sharratt", "age": 29 }, { "name": "Neil Sharratt", "age": null } ] }
+{ "cid": 807, "name": "Maryanne Kuzminski", "age": 21, "address": { "number": 1601, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Running" ], "children": [ { "name": "India Kuzminski", "age": null }, { "name": "Adell Kuzminski", "age": null } ] }
+{ "cid": 808, "name": "Brande Decius", "age": null, "address": null, "interests": [ "Basketball", "Fishing", "Puzzles" ], "children": [ { "name": "Li Decius", "age": 56 }, { "name": "Eusebio Decius", "age": 50 }, { "name": "Clementina Decius", "age": 29 } ] }
+{ "cid": 809, "name": "Dagny Mangiaracina", "age": 44, "address": { "number": 5993, "street": "Lake St.", "city": "San Jose" }, "interests": [  ], "children": [ { "name": "Bari Mangiaracina", "age": 31 }, { "name": "Tiara Mangiaracina", "age": 12 }, { "name": "Milly Mangiaracina", "age": null }, { "name": "Chelsie Mangiaracina", "age": null } ] }
+{ "cid": 810, "name": "Myron Dumlao", "age": null, "address": null, "interests": [ "Wine", "Coffee" ], "children": [ { "name": "Josie Dumlao", "age": 36 } ] }
+{ "cid": 811, "name": "Marti Whitmyre", "age": null, "address": null, "interests": [ "Music", "Walking" ], "children": [  ] }
+{ "cid": 812, "name": "Bee Godette", "age": 26, "address": { "number": 1757, "street": "Washington St.", "city": "Portland" }, "interests": [ "Video Games", "Base Jumping", "Tennis" ], "children": [ { "name": "Madaline Godette", "age": 10 }, { "name": "Shasta Godette", "age": 15 }, { "name": "Parthenia Godette", "age": 11 }, { "name": "Priscila Godette", "age": 13 } ] }
+{ "cid": 813, "name": "Leann Domagala", "age": 47, "address": { "number": 4472, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Computers" ], "children": [ { "name": "Alvera Domagala", "age": 36 }, { "name": "Rosalva Domagala", "age": 27 }, { "name": "Eugenia Domagala", "age": null }, { "name": "My Domagala", "age": 32 } ] }
+{ "cid": 814, "name": "Harriette Kasmarek", "age": 68, "address": { "number": 7191, "street": "Washington St.", "city": "Sunnyvale" }, "interests": [ "Music", "Skiing" ], "children": [ { "name": "Melani Kasmarek", "age": 24 }, { "name": "Jesica Kasmarek", "age": 22 } ] }
+{ "cid": 815, "name": "Leigha Bires", "age": 11, "address": { "number": 7263, "street": "Oak St.", "city": "Portland" }, "interests": [ "Running" ], "children": [ { "name": "Val Bires", "age": null } ] }
+{ "cid": 816, "name": "Cheyenne Eddie", "age": null, "address": null, "interests": [ "Walking", "Cooking" ], "children": [ { "name": "Kathe Eddie", "age": null }, { "name": "Charles Eddie", "age": null } ] }
+{ "cid": 818, "name": "Nellie Whetzell", "age": null, "address": null, "interests": [ "Walking" ], "children": [  ] }
+{ "cid": 819, "name": "Twanna Finnley", "age": null, "address": null, "interests": [ "Squash", "Cigars" ], "children": [ { "name": "Reba Finnley", "age": null }, { "name": "Moises Finnley", "age": null } ] }
+{ "cid": 820, "name": "Lacy Caudill", "age": 22, "address": { "number": 8679, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Wine" ], "children": [ { "name": "Sybil Caudill", "age": null } ] }
+{ "cid": 821, "name": "Carole Edlund", "age": 76, "address": { "number": 4008, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Computers", "Cooking", "Running", "Basketball" ], "children": [ { "name": "Garfield Edlund", "age": 54 }, { "name": "Brooks Edlund", "age": null }, { "name": "Gertrudis Edlund", "age": null }, { "name": "Tabitha Edlund", "age": 58 } ] }
+{ "cid": 824, "name": "Vonda Czaplewski", "age": 72, "address": { "number": 4597, "street": "7th St.", "city": "Portland" }, "interests": [ "Skiing" ], "children": [ { "name": "Gaynelle Czaplewski", "age": null }, { "name": "India Czaplewski", "age": null } ] }
+{ "cid": 825, "name": "Kirstie Rinebold", "age": 57, "address": { "number": 9463, "street": "Oak St.", "city": "Portland" }, "interests": [ "Cooking", "Cigars", "Books" ], "children": [ { "name": "Vonda Rinebold", "age": null }, { "name": "Man Rinebold", "age": 21 } ] }
+{ "cid": 826, "name": "Ressie Feenstra", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Sasha Feenstra", "age": null } ] }
+{ "cid": 827, "name": "Clementina Papin", "age": null, "address": null, "interests": [ "Music", "Basketball", "Cigars" ], "children": [ { "name": "Catina Papin", "age": null }, { "name": "Demetrius Papin", "age": 59 }, { "name": "Marylou Papin", "age": 12 }, { "name": "Apryl Papin", "age": 16 } ] }
+{ "cid": 828, "name": "Marcelle Steinhour", "age": null, "address": null, "interests": [ "Running", "Basketball", "Walking" ], "children": [ { "name": "Jimmie Steinhour", "age": 13 }, { "name": "Kirstie Steinhour", "age": 19 } ] }
+{ "cid": 831, "name": "Raina Rys", "age": 62, "address": { "number": 7048, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Walking" ], "children": [ { "name": "Ezra Rys", "age": null }, { "name": "Carl Rys", "age": null }, { "name": "Loraine Rys", "age": null } ] }
+{ "cid": 832, "name": "Alina Hosley", "age": null, "address": null, "interests": [ "Databases", "Databases", "Music" ], "children": [ { "name": "Sebrina Hosley", "age": null }, { "name": "Dyan Hosley", "age": null } ] }
+{ "cid": 833, "name": "Lakisha Petkoff", "age": null, "address": null, "interests": [ "Coffee" ], "children": [ { "name": "Brittanie Petkoff", "age": null }, { "name": "Ashli Petkoff", "age": null } ] }
+{ "cid": 834, "name": "Luvenia Grandstaff", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Joleen Grandstaff", "age": 28 }, { "name": "Elvera Grandstaff", "age": null }, { "name": "Leonia Grandstaff", "age": 35 }, { "name": "Jaclyn Grandstaff", "age": 28 } ] }
+{ "cid": 835, "name": "Raphael Marzili", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Angelic Marzili", "age": 38 } ] }
+{ "cid": 836, "name": "Elden Shumski", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Weldon Shumski", "age": null }, { "name": "Anneliese Shumski", "age": null } ] }
+{ "cid": 837, "name": "Denice Wolken", "age": 28, "address": { "number": 5010, "street": "7th St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Kattie Wolken", "age": null } ] }
+{ "cid": 838, "name": "Karan Aharon", "age": 88, "address": { "number": 8033, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Movies", "Walking" ], "children": [ { "name": "Matha Aharon", "age": 16 } ] }
+{ "cid": 841, "name": "Omar Enwall", "age": null, "address": null, "interests": [ "Skiing", "Skiing", "Books" ], "children": [ { "name": "Kirby Enwall", "age": 31 }, { "name": "Cythia Enwall", "age": 24 }, { "name": "August Enwall", "age": null } ] }
+{ "cid": 843, "name": "Lenny Acerno", "age": 64, "address": { "number": 7656, "street": "Main St.", "city": "Seattle" }, "interests": [ "Base Jumping", "Squash" ], "children": [  ] }
+{ "cid": 844, "name": "Madelene Ten", "age": null, "address": null, "interests": [ "Squash" ], "children": [ { "name": "Johanne Ten", "age": 39 }, { "name": "Lurline Ten", "age": null }, { "name": "Cathy Ten", "age": 49 } ] }
+{ "cid": 845, "name": "Burt Earp", "age": 21, "address": { "number": 7626, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Computers" ], "children": [ { "name": "Denny Earp", "age": null }, { "name": "Blaine Earp", "age": null }, { "name": "Wilson Earp", "age": 10 }, { "name": "Joan Earp", "age": null } ] }
+{ "cid": 846, "name": "Kieth Norlund", "age": 15, "address": { "number": 4039, "street": "Park St.", "city": "Mountain View" }, "interests": [ "Wine", "Walking", "Puzzles" ], "children": [ { "name": "Shawn Norlund", "age": null } ] }
+{ "cid": 847, "name": "Ashton Korba", "age": 25, "address": { "number": 6450, "street": "Park St.", "city": "Sunnyvale" }, "interests": [ "Cigars", "Computers", "Walking", "Video Games" ], "children": [  ] }
+{ "cid": 848, "name": "Myrta Kopf", "age": null, "address": null, "interests": [ "Wine", "Basketball", "Base Jumping" ], "children": [  ] }
+{ "cid": 850, "name": "Garnet Younce", "age": null, "address": null, "interests": [ "Databases", "Video Games", "Books" ], "children": [ { "name": "Syble Younce", "age": 16 } ] }
+{ "cid": 851, "name": "Darrel Machia", "age": 31, "address": { "number": 3290, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Coy Machia", "age": 13 }, { "name": "Janean Machia", "age": 13 }, { "name": "Sandi Machia", "age": 18 } ] }
+{ "cid": 852, "name": "Terrell Ramsay", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 853, "name": "Denisse Peralto", "age": 25, "address": { "number": 3931, "street": "7th St.", "city": "Portland" }, "interests": [ "Tennis", "Walking", "Basketball" ], "children": [ { "name": "Asha Peralto", "age": 14 }, { "name": "Clark Peralto", "age": null }, { "name": "Jessika Peralto", "age": null }, { "name": "Nadene Peralto", "age": null } ] }
+{ "cid": 854, "name": "Angie Oyster", "age": 32, "address": { "number": 8860, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Movies", "Fishing" ], "children": [ { "name": "Hugh Oyster", "age": 10 } ] }
+{ "cid": 855, "name": "Rosette Reen", "age": 57, "address": { "number": 2767, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Basketball" ], "children": [  ] }
+{ "cid": 857, "name": "Kasie Fujioka", "age": null, "address": null, "interests": [ "Skiing", "Cigars" ], "children": [ { "name": "Leontine Fujioka", "age": null }, { "name": "Nga Fujioka", "age": 21 }, { "name": "Nathanael Fujioka", "age": 27 } ] }
+{ "cid": 858, "name": "Maricruz Dittberner", "age": null, "address": null, "interests": [ "Tennis", "Wine", "Cigars", "Video Games" ], "children": [  ] }
+{ "cid": 859, "name": "Mozelle Catillo", "age": 61, "address": { "number": 253, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Databases", "Cooking", "Wine" ], "children": [  ] }
+{ "cid": 860, "name": "Isabelle Sept", "age": 88, "address": { "number": 4382, "street": "Washington St.", "city": "Portland" }, "interests": [ "Puzzles", "Books" ], "children": [  ] }
+{ "cid": 861, "name": "Hugh Mcbrien", "age": null, "address": null, "interests": [ "Skiing", "Cigars", "Cooking" ], "children": [ { "name": "Otha Mcbrien", "age": 38 } ] }
+{ "cid": 862, "name": "Constance Bries", "age": 77, "address": { "number": 2585, "street": "Oak St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Lizzie Bries", "age": 42 }, { "name": "Shenika Bries", "age": null }, { "name": "Phillip Bries", "age": null } ] }
+{ "cid": 864, "name": "Katharyn Zanotti", "age": 62, "address": { "number": 8336, "street": "7th St.", "city": "Sunnyvale" }, "interests": [ "Puzzles" ], "children": [ { "name": "Magan Zanotti", "age": null }, { "name": "Jacinto Zanotti", "age": null } ] }
+{ "cid": 865, "name": "Moon Marino", "age": 43, "address": { "number": 5710, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Skiing" ], "children": [ { "name": "Markita Marino", "age": 10 } ] }
+{ "cid": 866, "name": "Bonita Kauphusman", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 869, "name": "Lino Wooderson", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Nola Wooderson", "age": null }, { "name": "Leticia Wooderson", "age": 36 }, { "name": "Bernardine Wooderson", "age": null } ] }
+{ "cid": 870, "name": "Natosha Lufsey", "age": null, "address": null, "interests": [ "Cigars", "Walking" ], "children": [ { "name": "Tiffany Lufsey", "age": null } ] }
+{ "cid": 871, "name": "Lona Dacus", "age": null, "address": null, "interests": [ "Base Jumping" ], "children": [ { "name": "Pablo Dacus", "age": null }, { "name": "Darlene Dacus", "age": 45 }, { "name": "Darius Dacus", "age": 31 }, { "name": "Cordia Dacus", "age": null } ] }
+{ "cid": 872, "name": "Michele Herschel", "age": 39, "address": { "number": 4287, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [  ] }
+{ "cid": 875, "name": "Ramon Crepps", "age": null, "address": null, "interests": [ "Coffee", "Movies", "Skiing" ], "children": [ { "name": "Elisha Crepps", "age": null } ] }
+{ "cid": 876, "name": "Chelsie Motten", "age": null, "address": null, "interests": [ "Music", "Squash", "Music", "Walking" ], "children": [ { "name": "Nida Motten", "age": null }, { "name": "Taneka Motten", "age": 10 }, { "name": "Maynard Motten", "age": 57 } ] }
+{ "cid": 877, "name": "Nicki Lipkind", "age": null, "address": null, "interests": [ "Books", "Movies" ], "children": [ { "name": "Yahaira Lipkind", "age": 12 } ] }
+{ "cid": 878, "name": "Migdalia Bisker", "age": 50, "address": { "number": 6699, "street": "Oak St.", "city": "Los Angeles" }, "interests": [ "Computers", "Basketball" ], "children": [ { "name": "Moira Bisker", "age": null }, { "name": "Tanisha Bisker", "age": null } ] }
+{ "cid": 879, "name": "Vinnie Antoniewicz", "age": 45, "address": { "number": 1633, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Cooking", "Puzzles" ], "children": [  ] }
+{ "cid": 880, "name": "Sara Abo", "age": null, "address": null, "interests": [ "Squash" ], "children": [  ] }
+{ "cid": 881, "name": "Leora Chesnutt", "age": 49, "address": { "number": 6487, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Movies" ], "children": [ { "name": "Myrtle Chesnutt", "age": null }, { "name": "Serina Chesnutt", "age": 11 }, { "name": "Jana Chesnutt", "age": 10 } ] }
+{ "cid": 883, "name": "Odilia Bugtong", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Mark Bugtong", "age": 15 }, { "name": "Paula Bugtong", "age": null }, { "name": "Jenee Bugtong", "age": 17 }, { "name": "Lilian Bugtong", "age": 44 } ] }
+{ "cid": 884, "name": "Laila Marta", "age": null, "address": null, "interests": [ "Fishing", "Movies" ], "children": [ { "name": "Carlota Marta", "age": 19 } ] }
+{ "cid": 885, "name": "Les Legere", "age": 87, "address": { "number": 3998, "street": "Cedar St.", "city": "Portland" }, "interests": [ "Bass", "Tennis", "Fishing" ], "children": [ { "name": "Concetta Legere", "age": 45 }, { "name": "Tamica Legere", "age": null }, { "name": "Aurora Legere", "age": null } ] }
+{ "cid": 887, "name": "Jermaine Folz", "age": 35, "address": { "number": 8487, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Computers", "Puzzles", "Cooking" ], "children": [ { "name": "Sharice Folz", "age": null } ] }
+{ "cid": 888, "name": "Natalie Nocella", "age": 66, "address": { "number": 2856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Noel Nocella", "age": 26 }, { "name": "Damon Nocella", "age": 29 }, { "name": "Joesph Nocella", "age": 33 }, { "name": "Nidia Nocella", "age": null } ] }
+{ "cid": 889, "name": "Elvis Schoff", "age": 83, "address": { "number": 6724, "street": "Hill St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Spring Schoff", "age": 43 }, { "name": "Davis Schoff", "age": 55 }, { "name": "Ryann Schoff", "age": 58 }, { "name": "Clarinda Schoff", "age": 11 } ] }
+{ "cid": 890, "name": "Janise Maccarthy", "age": 66, "address": { "number": 7337, "street": "Main St.", "city": "San Jose" }, "interests": [ "Wine", "Computers" ], "children": [  ] }
+{ "cid": 891, "name": "Jesusita Bhatia", "age": 57, "address": { "number": 1476, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Walking" ], "children": [  ] }
+{ "cid": 892, "name": "Madge Hendson", "age": 79, "address": { "number": 8832, "street": "Cedar St.", "city": "San Jose" }, "interests": [ "Databases", "Fishing", "Skiing" ], "children": [ { "name": "Elia Hendson", "age": 48 }, { "name": "Lashawn Hendson", "age": 27 } ] }
+{ "cid": 893, "name": "Norberto Banchero", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 894, "name": "Reginald Julien", "age": 16, "address": { "number": 1107, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Databases", "Wine" ], "children": [ { "name": "Arthur Julien", "age": null }, { "name": "Evia Julien", "age": null } ] }
+{ "cid": 897, "name": "Gerald Roehrman", "age": null, "address": null, "interests": [ "Bass", "Wine" ], "children": [ { "name": "Virgie Roehrman", "age": 28 }, { "name": "Akiko Roehrman", "age": 59 }, { "name": "Robbie Roehrman", "age": 10 }, { "name": "Flavia Roehrman", "age": null } ] }
+{ "cid": 898, "name": "Thao Seufert", "age": 78, "address": { "number": 3529, "street": "Hill St.", "city": "Seattle" }, "interests": [ "Bass", "Squash", "Coffee" ], "children": [ { "name": "Classie Seufert", "age": null } ] }
+{ "cid": 899, "name": "Ada Kamealoha", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Juliann Kamealoha", "age": null }, { "name": "Ilana Kamealoha", "age": 25 }, { "name": "Herminia Kamealoha", "age": 55 }, { "name": "Carli Kamealoha", "age": null } ] }
+{ "cid": 901, "name": "Riva Ziko", "age": null, "address": null, "interests": [ "Running", "Tennis", "Video Games" ], "children": [ { "name": "Leandra Ziko", "age": 49 }, { "name": "Torrie Ziko", "age": null } ] }
+{ "cid": 903, "name": "Elise Morenz", "age": 17, "address": { "number": 8968, "street": "View St.", "city": "Mountain View" }, "interests": [  ], "children": [  ] }
+{ "cid": 904, "name": "Holley Tofil", "age": 51, "address": { "number": 8946, "street": "Oak St.", "city": "Mountain View" }, "interests": [ "Music", "Squash" ], "children": [ { "name": "Kristal Tofil", "age": null } ] }
+{ "cid": 905, "name": "Pandora Azzarella", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lane Azzarella", "age": null }, { "name": "Joi Azzarella", "age": 19 } ] }
+{ "cid": 907, "name": "Princess Sudol", "age": 73, "address": { "number": 9770, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Base Jumping" ], "children": [ { "name": "Bronwyn Sudol", "age": 22 }, { "name": "Judith Sudol", "age": null } ] }
+{ "cid": 908, "name": "Ferdinand Auila", "age": 82, "address": { "number": 1071, "street": "Lake St.", "city": "Portland" }, "interests": [ "Base Jumping", "Running", "Wine" ], "children": [ { "name": "Ai Auila", "age": 69 }, { "name": "Laurel Auila", "age": null } ] }
+{ "cid": 909, "name": "Mariko Sharar", "age": null, "address": null, "interests": [ "Squash", "Movies", "Computers" ], "children": [  ] }
+{ "cid": 910, "name": "Everette Moe", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Berna Moe", "age": 56 }, { "name": "Harold Moe", "age": 28 }, { "name": "See Moe", "age": 20 } ] }
+{ "cid": 911, "name": "Eileen Bartolomeo", "age": 20, "address": { "number": 8915, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [  ] }
+{ "cid": 912, "name": "Alessandra Kaskey", "age": 52, "address": { "number": 6906, "street": "View St.", "city": "Los Angeles" }, "interests": [ "Skiing", "Walking", "Basketball" ], "children": [ { "name": "Mack Kaskey", "age": null } ] }
+{ "cid": 913, "name": "Evelynn Fague", "age": 42, "address": { "number": 5729, "street": "7th St.", "city": "Seattle" }, "interests": [ "Books", "Databases", "Cooking" ], "children": [  ] }
+{ "cid": 914, "name": "Hunter Flournoy", "age": null, "address": null, "interests": [ "Cooking", "Squash" ], "children": [ { "name": "Christopher Flournoy", "age": 59 }, { "name": "Earnestine Flournoy", "age": null } ] }
+{ "cid": 916, "name": "Kris Mcmarlin", "age": null, "address": null, "interests": [ "Movies", "Music", "Puzzles" ], "children": [  ] }
+{ "cid": 917, "name": "Jerri Blachowski", "age": null, "address": null, "interests": [ "Skiing" ], "children": [ { "name": "Chet Blachowski", "age": 43 }, { "name": "Mallory Blachowski", "age": null }, { "name": "Akilah Blachowski", "age": null } ] }
+{ "cid": 919, "name": "Fairy Wansley", "age": 45, "address": { "number": 9020, "street": "Park St.", "city": "Los Angeles" }, "interests": [ "Wine", "Walking", "Databases", "Video Games" ], "children": [ { "name": "Marvella Wansley", "age": null }, { "name": "Hisako Wansley", "age": null }, { "name": "Shaunta Wansley", "age": null }, { "name": "Gemma Wansley", "age": 21 } ] }
+{ "cid": 920, "name": "Mirtha Dellbringge", "age": null, "address": null, "interests": [ "Walking", "Basketball", "Basketball" ], "children": [ { "name": "Morgan Dellbringge", "age": 51 }, { "name": "Alease Dellbringge", "age": 35 } ] }
+{ "cid": 921, "name": "Mario Nolden", "age": 17, "address": { "number": 3977, "street": "Cedar St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Gertrude Nolden", "age": null }, { "name": "Ray Nolden", "age": null }, { "name": "Inocencia Nolden", "age": null } ] }
+{ "cid": 922, "name": "Shanice Lingle", "age": 26, "address": { "number": 4753, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Sandie Lingle", "age": 12 }, { "name": "Nia Lingle", "age": 13 }, { "name": "Marilyn Lingle", "age": 15 } ] }
+{ "cid": 923, "name": "Bobbi Ursino", "age": null, "address": null, "interests": [ "Movies", "Books", "Walking" ], "children": [ { "name": "Shon Ursino", "age": null }, { "name": "Lorean Ursino", "age": null } ] }
+{ "cid": 924, "name": "Kathleen Lash", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Clementina Lash", "age": 58 }, { "name": "Zula Lash", "age": null }, { "name": "Mellissa Lash", "age": 54 } ] }
+{ "cid": 925, "name": "Quintin Kizzie", "age": null, "address": null, "interests": [ "Computers", "Tennis", "Bass", "Movies" ], "children": [ { "name": "Julius Kizzie", "age": 11 }, { "name": "Melissia Kizzie", "age": null }, { "name": "Olga Kizzie", "age": 42 } ] }
+{ "cid": 927, "name": "Lillia Hartlein", "age": 55, "address": { "number": 5856, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Base Jumping", "Coffee", "Cigars" ], "children": [ { "name": "Nicky Hartlein", "age": null }, { "name": "Cassaundra Hartlein", "age": 10 }, { "name": "Micheline Hartlein", "age": 26 }, { "name": "Anton Hartlein", "age": 32 } ] }
+{ "cid": 928, "name": "Maddie Diclaudio", "age": 33, "address": { "number": 4674, "street": "Washington St.", "city": "San Jose" }, "interests": [ "Base Jumping", "Databases", "Bass" ], "children": [ { "name": "Dominique Diclaudio", "age": 12 } ] }
+{ "cid": 929, "name": "Jean Guitierrez", "age": 75, "address": { "number": 9736, "street": "Lake St.", "city": "Mountain View" }, "interests": [ "Wine", "Wine", "Fishing" ], "children": [  ] }
+{ "cid": 930, "name": "Kathie Gier", "age": 37, "address": { "number": 5075, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Onie Gier", "age": 16 } ] }
+{ "cid": 931, "name": "Octavia Koiner", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Ardath Koiner", "age": 32 }, { "name": "Milly Koiner", "age": null }, { "name": "Arlinda Koiner", "age": null }, { "name": "Debby Koiner", "age": null } ] }
+{ "cid": 932, "name": "Kraig Bomia", "age": null, "address": null, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 933, "name": "Eartha Hershberger", "age": 81, "address": { "number": 7013, "street": "Cedar St.", "city": "Los Angeles" }, "interests": [ "Puzzles" ], "children": [ { "name": "Waneta Hershberger", "age": null }, { "name": "Katherine Hershberger", "age": 67 }, { "name": "Johnnie Hershberger", "age": 25 }, { "name": "Jovan Hershberger", "age": 30 } ] }
+{ "cid": 934, "name": "Dessie Lockmiller", "age": 70, "address": { "number": 4313, "street": "Lake St.", "city": "San Jose" }, "interests": [ "Coffee", "Puzzles" ], "children": [  ] }
+{ "cid": 935, "name": "Sharita Aspegren", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Russell Aspegren", "age": 35 }, { "name": "Bernardina Aspegren", "age": null }, { "name": "Isobel Aspegren", "age": 11 }, { "name": "Reva Aspegren", "age": null } ] }
+{ "cid": 937, "name": "Annika Pauline", "age": 78, "address": { "number": 8563, "street": "Hill St.", "city": "Los Angeles" }, "interests": [  ], "children": [ { "name": "Mikki Pauline", "age": 34 } ] }
+{ "cid": 938, "name": "Parthenia Dromgoole", "age": 36, "address": { "number": 527, "street": "Lake St.", "city": "Sunnyvale" }, "interests": [ "Fishing" ], "children": [  ] }
+{ "cid": 940, "name": "Kitty Nalepka", "age": null, "address": null, "interests": [ "Movies", "Wine", "Basketball" ], "children": [ { "name": "Kendra Nalepka", "age": null } ] }
+{ "cid": 941, "name": "Jamey Jakobson", "age": null, "address": null, "interests": [ "Books", "Cooking", "Video Games" ], "children": [ { "name": "Elmer Jakobson", "age": 14 }, { "name": "Minh Jakobson", "age": 30 } ] }
+{ "cid": 942, "name": "Emerson Keblish", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Leonora Keblish", "age": null } ] }
+{ "cid": 943, "name": "Kathryne Blacock", "age": 82, "address": { "number": 3510, "street": "Oak St.", "city": "Sunnyvale" }, "interests": [ "Running", "Bass", "Music" ], "children": [  ] }
+{ "cid": 944, "name": "Johana Hisman", "age": null, "address": null, "interests": [ "Wine" ], "children": [ { "name": "Kirstin Hisman", "age": 43 }, { "name": "Darwin Hisman", "age": 29 } ] }
+{ "cid": 945, "name": "Hildegard Dedinas", "age": 70, "address": { "number": 3273, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Renato Dedinas", "age": 35 } ] }
+{ "cid": 946, "name": "Taylor Parrigan", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Salome Parrigan", "age": 50 }, { "name": "Gary Parrigan", "age": 25 }, { "name": "Harold Parrigan", "age": null } ] }
+{ "cid": 948, "name": "Thad Scialpi", "age": 22, "address": { "number": 8731, "street": "Washington St.", "city": "Portland" }, "interests": [ "Base Jumping", "Tennis", "Wine" ], "children": [ { "name": "Harlan Scialpi", "age": 10 }, { "name": "Lucile Scialpi", "age": 11 }, { "name": "Audria Scialpi", "age": null } ] }
+{ "cid": 949, "name": "Elissa Rogue", "age": null, "address": null, "interests": [ "Fishing", "Music" ], "children": [ { "name": "Noriko Rogue", "age": 41 }, { "name": "Lavona Rogue", "age": 39 } ] }
+{ "cid": 950, "name": "Young Bayn", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Evangeline Bayn", "age": 38 }, { "name": "Darcy Bayn", "age": 45 }, { "name": "Rosita Bayn", "age": null }, { "name": "Austin Bayn", "age": 46 } ] }
+{ "cid": 951, "name": "Janine Martorano", "age": 65, "address": { "number": 6420, "street": "7th St.", "city": "Los Angeles" }, "interests": [ "Books", "Music" ], "children": [ { "name": "Idella Martorano", "age": null } ] }
+{ "cid": 955, "name": "Liliana Stenkamp", "age": null, "address": null, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 956, "name": "Laquanda Bynoe", "age": 79, "address": { "number": 6122, "street": "Main St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Joel Bynoe", "age": null }, { "name": "Brian Bynoe", "age": 61 }, { "name": "Shana Bynoe", "age": null } ] }
+{ "cid": 957, "name": "Lucius Schurr", "age": 75, "address": { "number": 3918, "street": "Main St.", "city": "Mountain View" }, "interests": [  ], "children": [ { "name": "Willetta Schurr", "age": 22 }, { "name": "Andre Schurr", "age": null }, { "name": "Merrilee Schurr", "age": 32 } ] }
+{ "cid": 958, "name": "Ricardo Pezzica", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Delois Pezzica", "age": 11 } ] }
+{ "cid": 960, "name": "Lenore Limardi", "age": null, "address": null, "interests": [ "Music" ], "children": [ { "name": "Kris Limardi", "age": 12 } ] }
+{ "cid": 961, "name": "Mirian Herpolsheimer", "age": null, "address": null, "interests": [ "Music", "Fishing", "Computers" ], "children": [ { "name": "Larissa Herpolsheimer", "age": 41 }, { "name": "Markus Herpolsheimer", "age": null }, { "name": "Natacha Herpolsheimer", "age": null } ] }
+{ "cid": 962, "name": "Taryn Coley", "age": null, "address": null, "interests": [ "Running", "Basketball", "Cooking" ], "children": [  ] }
+{ "cid": 963, "name": "Mila Ditmars", "age": 29, "address": { "number": 5850, "street": "View St.", "city": "Sunnyvale" }, "interests": [ "Music" ], "children": [  ] }
+{ "cid": 964, "name": "Stephany Soders", "age": null, "address": null, "interests": [ "Tennis", "Wine", "Computers" ], "children": [  ] }
+{ "cid": 965, "name": "Mellie Risen", "age": null, "address": null, "interests": [ "Tennis" ], "children": [ { "name": "Coreen Risen", "age": 36 }, { "name": "Faith Risen", "age": 34 }, { "name": "Crystle Risen", "age": 54 } ] }
+{ "cid": 966, "name": "Brigitte Quimby", "age": 13, "address": { "number": 203, "street": "Main St.", "city": "Mountain View" }, "interests": [ "Skiing", "Tennis" ], "children": [ { "name": "Ilona Quimby", "age": null }, { "name": "Shaunte Quimby", "age": null }, { "name": "Lorie Quimby", "age": null } ] }
+{ "cid": 968, "name": "Alix Levier", "age": 44, "address": { "number": 7241, "street": "Hill St.", "city": "Los Angeles" }, "interests": [ "Databases", "Fishing", "Wine" ], "children": [ { "name": "Florentina Levier", "age": null }, { "name": "Hyon Levier", "age": null }, { "name": "Dannielle Levier", "age": null } ] }
+{ "cid": 970, "name": "Pia Sudderth", "age": null, "address": null, "interests": [ "Databases" ], "children": [ { "name": "Ernestina Sudderth", "age": 15 }, { "name": "Larue Sudderth", "age": 46 }, { "name": "Toshia Sudderth", "age": 27 } ] }
+{ "cid": 974, "name": "Alexis Malcomson", "age": null, "address": null, "interests": [ "Movies", "Books" ], "children": [ { "name": "Kerri Malcomson", "age": null } ] }
+{ "cid": 975, "name": "Gary Whitemore", "age": null, "address": null, "interests": [  ], "children": [  ] }
+{ "cid": 976, "name": "Madalyn Nidiffer", "age": 35, "address": { "number": 7635, "street": "Main St.", "city": "San Jose" }, "interests": [ "Coffee", "Wine", "Music" ], "children": [ { "name": "Tricia Nidiffer", "age": 10 }, { "name": "Kevin Nidiffer", "age": 24 }, { "name": "Elyse Nidiffer", "age": null } ] }
+{ "cid": 978, "name": "Rudy Watsky", "age": 32, "address": { "number": 2754, "street": "Oak St.", "city": "Seattle" }, "interests": [ "Cooking" ], "children": [  ] }
+{ "cid": 979, "name": "Yoko Bailony", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Vivienne Bailony", "age": null }, { "name": "Lori Bailony", "age": 47 } ] }
+{ "cid": 980, "name": "Harley Lappe", "age": 56, "address": { "number": 647, "street": "Hill St.", "city": "Mountain View" }, "interests": [ "Books", "Cigars", "Basketball" ], "children": [ { "name": "Maxwell Lappe", "age": null }, { "name": "Gemma Lappe", "age": 32 }, { "name": "Ester Lappe", "age": 40 }, { "name": "Myles Lappe", "age": 36 } ] }
+{ "cid": 981, "name": "Lilliam Lopus", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Tracey Lopus", "age": null } ] }
+{ "cid": 982, "name": "Jude Brandsrud", "age": 41, "address": { "number": 7133, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Bass", "Skiing" ], "children": [ { "name": "Scottie Brandsrud", "age": null }, { "name": "Gennie Brandsrud", "age": 10 }, { "name": "Agnes Brandsrud", "age": null }, { "name": "Clarinda Brandsrud", "age": 17 } ] }
+{ "cid": 984, "name": "Janett Kitchens", "age": 66, "address": { "number": 7558, "street": "View St.", "city": "Mountain View" }, "interests": [ "Coffee", "Movies", "Squash" ], "children": [ { "name": "Grayce Kitchens", "age": 14 }, { "name": "Dwayne Kitchens", "age": null }, { "name": "Wilber Kitchens", "age": 51 }, { "name": "Nancey Kitchens", "age": null } ] }
+{ "cid": 985, "name": "Arnette Farlow", "age": 23, "address": { "number": 7843, "street": "Main St.", "city": "Portland" }, "interests": [ "Running", "Databases" ], "children": [ { "name": "Lora Farlow", "age": 12 }, { "name": "Arlen Farlow", "age": 11 }, { "name": "Rodney Farlow", "age": null }, { "name": "Tori Farlow", "age": 11 } ] }
+{ "cid": 986, "name": "Tennille Wikle", "age": 78, "address": { "number": 3428, "street": "View St.", "city": "Portland" }, "interests": [ "Movies", "Databases", "Wine" ], "children": [ { "name": "Lourie Wikle", "age": null }, { "name": "Laure Wikle", "age": null } ] }
+{ "cid": 987, "name": "Sharolyn Demchak", "age": 36, "address": { "number": 4672, "street": "Lake St.", "city": "San Jose" }, "interests": [  ], "children": [  ] }
+{ "cid": 988, "name": "Dagmar Plasky", "age": 89, "address": { "number": 1219, "street": "Park St.", "city": "Portland" }, "interests": [  ], "children": [ { "name": "Dann Plasky", "age": 59 }, { "name": "Raye Plasky", "age": null }, { "name": "Sammie Plasky", "age": 36 }, { "name": "Kasi Plasky", "age": 24 } ] }
+{ "cid": 991, "name": "Leonel Toepperwein", "age": 62, "address": { "number": 8356, "street": "Washington St.", "city": "Seattle" }, "interests": [ "Coffee", "Books" ], "children": [ { "name": "Sean Toepperwein", "age": null }, { "name": "Charline Toepperwein", "age": 49 }, { "name": "Hattie Toepperwein", "age": 22 }, { "name": "Melida Toepperwein", "age": null } ] }
+{ "cid": 992, "name": "Staci Alexandropoul", "age": null, "address": null, "interests": [ "Databases", "Movies", "Tennis" ], "children": [ { "name": "Casimira Alexandropoul", "age": null }, { "name": "Kena Alexandropoul", "age": 54 }, { "name": "Ellie Alexandropoul", "age": null }, { "name": "Ambrose Alexandropoul", "age": null } ] }
+{ "cid": 993, "name": "Shawn Irie", "age": null, "address": null, "interests": [ "Fishing", "Cigars" ], "children": [ { "name": "Tonette Irie", "age": null } ] }
+{ "cid": 994, "name": "Isa Gravelle", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Lashonda Gravelle", "age": null }, { "name": "Carry Gravelle", "age": 58 } ] }
+{ "cid": 995, "name": "Kiersten Basila", "age": null, "address": null, "interests": [  ], "children": [ { "name": "Norman Basila", "age": 17 }, { "name": "Reginia Basila", "age": null }, { "name": "Gilberto Basila", "age": null }, { "name": "Elvira Basila", "age": 49 } ] }
+{ "cid": 996, "name": "Elouise Wider", "age": null, "address": null, "interests": [ "Coffee", "Computers", "Base Jumping" ], "children": [  ] }
+{ "cid": 997, "name": "Yesenia Gao", "age": 38, "address": { "number": 5990, "street": "View St.", "city": "Portland" }, "interests": [ "Computers", "Computers", "Puzzles", "Puzzles" ], "children": [ { "name": "Jared Gao", "age": 11 }, { "name": "Sang Gao", "age": null }, { "name": "Jeanne Gao", "age": 13 }, { "name": "Lavona Gao", "age": 23 } ] }
+{ "cid": 998, "name": "Barry Schmaus", "age": 65, "address": { "number": 4894, "street": "View St.", "city": "Sunnyvale" }, "interests": [  ], "children": [ { "name": "Ma Schmaus", "age": 40 }, { "name": "Lashawn Schmaus", "age": 13 }, { "name": "Georgianne Schmaus", "age": 38 } ] }
+{ "cid": 999, "name": "Bo Chaim", "age": 59, "address": { "number": 8050, "street": "View St.", "city": "Seattle" }, "interests": [  ], "children": [ { "name": "Zandra Chaim", "age": 42 }, { "name": "Theda Chaim", "age": 14 }, { "name": "Sharika Chaim", "age": 22 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.adm
new file mode 100644
index 0000000..6d89122
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-edit-distance/fuzzy-inverted-index-olist-edit-distance.1.adm
@@ -0,0 +1,8 @@
+{ "cid": 11, "name": "Meta Simek", "age": 13, "address": { "number": 4384, "street": "7th St.", "city": "San Jose" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Oretha Simek", "age": null }, { "name": "Terence Simek", "age": null } ] }
+{ "cid": 132, "name": "Cindi Turntine", "age": 64, "address": { "number": 9432, "street": "Park St.", "city": "Portland" }, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Howard Turntine", "age": null } ] }
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [  ] }
+{ "cid": 389, "name": "Loraine Morfee", "age": 72, "address": { "number": 2945, "street": "Lake St.", "city": "Seattle" }, "interests": [ "Wine", "Walking" ], "children": [ { "name": "Berry Morfee", "age": 30 } ] }
+{ "cid": 573, "name": "Tyree Ketcher", "age": null, "address": null, "interests": [ "Computers", "Walking" ], "children": [ { "name": "Aleisha Ketcher", "age": null }, { "name": "Vonda Ketcher", "age": null }, { "name": "Cyndy Ketcher", "age": 13 }, { "name": "Chassidy Ketcher", "age": 30 } ] }
+{ "cid": 658, "name": "Truman Leitner", "age": null, "address": null, "interests": [ "Computers", "Bass", "Walking" ], "children": [  ] }
+{ "cid": 716, "name": "Deirdre Bruderer", "age": null, "address": null, "interests": [ "Computers", "Wine" ], "children": [ { "name": "Coralee Bruderer", "age": null }, { "name": "Mina Bruderer", "age": null }, { "name": "Lindsey Bruderer", "age": 35 }, { "name": "Yi Bruderer", "age": null } ] }
+{ "cid": 838, "name": "Karan Aharon", "age": 88, "address": { "number": 8033, "street": "Washington St.", "city": "Portland" }, "interests": [ "Computers", "Movies", "Walking" ], "children": [ { "name": "Matha Aharon", "age": 16 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.adm
new file mode 100644
index 0000000..71bb9d7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-olist-jaccard/fuzzy-inverted-index-olist-jaccard.1.adm
@@ -0,0 +1 @@
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": [ "Computers", "Wine", "Databases", "Walking" ], "children": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.adm
new file mode 100644
index 0000000..fd1b75e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-ulist-jaccard/fuzzy-inverted-index-ulist-jaccard.1.adm
@@ -0,0 +1 @@
+{ "cid": 153, "name": "Randy Hueso", "age": 11, "address": { "number": 1957, "street": "Oak St.", "city": "San Jose" }, "interests": {{ "Computers", "Wine", "Databases", "Walking" }}, "children": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.adm
new file mode 100644
index 0000000..8a99b26
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-contains/fuzzy-inverted-index-word-contains.1.adm
@@ -0,0 +1,3 @@
+{ "id": 4, "dblpid": "books/acm/kim95/ChristodoulakisK95", "title": "Multimedia Information Systems  Issues and Approaches.", "authors": "Stavros Christodoulakis Leonidas Koveos", "misc": "2002-01-03 318-337 1995 Modern Database Systems db/books/collections/kim95.html#ChristodoulakisK95" }
+{ "id": 89, "dblpid": "conf/icip/SchonfeldL98", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-11-05 123-127 1998 ICIP (3) db/conf/icip/icip1998-3.html#SchonfeldL98" }
+{ "id": 90, "dblpid": "conf/hicss/SchonfeldL99", "title": "VORTEX  Video Retrieval and Tracking from Compressed Multimedia Databases ¾ Visual Search Engine.", "authors": "Dan Schonfeld Dan Lelescu", "misc": "2002-01-03 1999 HICSS http //computer.org/proceedings/hicss/0001/00013/00013006abs.htm db/conf/hicss/hicss1999-3.html#SchonfeldL99" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.adm
new file mode 100644
index 0000000..5bf6ae0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/index-selection/fuzzy-inverted-index-word-jaccard/fuzzy-inverted-index-word-jaccard.1.adm
@@ -0,0 +1 @@
+{ "id": 9, "dblpid": "books/acm/kim95/Kaiser95", "title": "Cooperative Transactions for Multiuser Environments.", "authors": "Gail E. Kaiser", "misc": "2002-01-03 409-433 1995 Modern Database Systems db/books/collections/kim95.html#Kaiser95" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query_issue285-2/query_issue285-2.1.adm b/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query_issue285-2/query_issue285-2.1.adm
new file mode 100644
index 0000000..3005398
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query_issue285-2/query_issue285-2.1.adm
@@ -0,0 +1,10 @@
+{ "aid": 1, "bids": [  ] }
+{ "aid": 2, "bids": [  ] }
+{ "aid": 3, "bids": [  ] }
+{ "aid": 4, "bids": [  ] }
+{ "aid": 5, "bids": [ 98 ] }
+{ "aid": 6, "bids": [  ] }
+{ "aid": 7, "bids": [  ] }
+{ "aid": 8, "bids": [  ] }
+{ "aid": 9, "bids": [  ] }
+{ "aid": 10, "bids": [  ] }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query_issue285/query_issue285.1.adm b/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query_issue285/query_issue285.1.adm
new file mode 100644
index 0000000..62c427a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/leftouterjoin/query_issue285/query_issue285.1.adm
@@ -0,0 +1,100 @@
+{ "aid": 1, "bids": [  ] }
+{ "aid": 2, "bids": [  ] }
+{ "aid": 3, "bids": [  ] }
+{ "aid": 4, "bids": [  ] }
+{ "aid": 5, "bids": [ 98 ] }
+{ "aid": 6, "bids": [  ] }
+{ "aid": 7, "bids": [  ] }
+{ "aid": 8, "bids": [  ] }
+{ "aid": 9, "bids": [  ] }
+{ "aid": 10, "bids": [  ] }
+{ "aid": 11, "bids": [  ] }
+{ "aid": 12, "bids": [  ] }
+{ "aid": 13, "bids": [  ] }
+{ "aid": 14, "bids": [  ] }
+{ "aid": 15, "bids": [  ] }
+{ "aid": 16, "bids": [  ] }
+{ "aid": 17, "bids": [  ] }
+{ "aid": 18, "bids": [  ] }
+{ "aid": 19, "bids": [  ] }
+{ "aid": 20, "bids": [  ] }
+{ "aid": 21, "bids": [  ] }
+{ "aid": 22, "bids": [  ] }
+{ "aid": 23, "bids": [  ] }
+{ "aid": 24, "bids": [  ] }
+{ "aid": 25, "bids": [  ] }
+{ "aid": 26, "bids": [  ] }
+{ "aid": 27, "bids": [  ] }
+{ "aid": 28, "bids": [  ] }
+{ "aid": 29, "bids": [  ] }
+{ "aid": 30, "bids": [  ] }
+{ "aid": 31, "bids": [  ] }
+{ "aid": 32, "bids": [  ] }
+{ "aid": 33, "bids": [  ] }
+{ "aid": 34, "bids": [ 57 ] }
+{ "aid": 35, "bids": [  ] }
+{ "aid": 36, "bids": [  ] }
+{ "aid": 37, "bids": [  ] }
+{ "aid": 38, "bids": [  ] }
+{ "aid": 39, "bids": [  ] }
+{ "aid": 40, "bids": [  ] }
+{ "aid": 41, "bids": [  ] }
+{ "aid": 42, "bids": [  ] }
+{ "aid": 43, "bids": [  ] }
+{ "aid": 44, "bids": [  ] }
+{ "aid": 45, "bids": [  ] }
+{ "aid": 46, "bids": [  ] }
+{ "aid": 47, "bids": [  ] }
+{ "aid": 48, "bids": [  ] }
+{ "aid": 49, "bids": [  ] }
+{ "aid": 50, "bids": [  ] }
+{ "aid": 51, "bids": [  ] }
+{ "aid": 52, "bids": [  ] }
+{ "aid": 53, "bids": [  ] }
+{ "aid": 54, "bids": [ 91 ] }
+{ "aid": 55, "bids": [  ] }
+{ "aid": 56, "bids": [  ] }
+{ "aid": 57, "bids": [  ] }
+{ "aid": 58, "bids": [  ] }
+{ "aid": 59, "bids": [  ] }
+{ "aid": 60, "bids": [  ] }
+{ "aid": 61, "bids": [  ] }
+{ "aid": 62, "bids": [  ] }
+{ "aid": 63, "bids": [  ] }
+{ "aid": 64, "bids": [  ] }
+{ "aid": 65, "bids": [  ] }
+{ "aid": 66, "bids": [  ] }
+{ "aid": 67, "bids": [  ] }
+{ "aid": 68, "bids": [ 57 ] }
+{ "aid": 69, "bids": [ 57 ] }
+{ "aid": 70, "bids": [  ] }
+{ "aid": 71, "bids": [  ] }
+{ "aid": 72, "bids": [  ] }
+{ "aid": 73, "bids": [  ] }
+{ "aid": 74, "bids": [  ] }
+{ "aid": 75, "bids": [  ] }
+{ "aid": 76, "bids": [  ] }
+{ "aid": 77, "bids": [  ] }
+{ "aid": 78, "bids": [  ] }
+{ "aid": 79, "bids": [  ] }
+{ "aid": 80, "bids": [  ] }
+{ "aid": 81, "bids": [  ] }
+{ "aid": 82, "bids": [  ] }
+{ "aid": 83, "bids": [  ] }
+{ "aid": 84, "bids": [  ] }
+{ "aid": 85, "bids": [  ] }
+{ "aid": 86, "bids": [  ] }
+{ "aid": 87, "bids": [  ] }
+{ "aid": 88, "bids": [  ] }
+{ "aid": 89, "bids": [  ] }
+{ "aid": 90, "bids": [  ] }
+{ "aid": 91, "bids": [  ] }
+{ "aid": 92, "bids": [  ] }
+{ "aid": 93, "bids": [  ] }
+{ "aid": 94, "bids": [  ] }
+{ "aid": 95, "bids": [  ] }
+{ "aid": 96, "bids": [  ] }
+{ "aid": 97, "bids": [  ] }
+{ "aid": 98, "bids": [  ] }
+{ "aid": 99, "bids": [  ] }
+{ "aid": 100, "bids": [  ] }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/load/issue289_query/issue289_query.1.adm b/asterix-app/src/test/resources/runtimets/results/load/issue289_query/issue289_query.1.adm
new file mode 100644
index 0000000..9a03714
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/load/issue289_query/issue289_query.1.adm
@@ -0,0 +1 @@
+10
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/scan/alltypes_02.adm b/asterix-app/src/test/resources/runtimets/results/scan/alltypes_02.adm
deleted file mode 100644
index 64aee7c..0000000
--- a/asterix-app/src/test/resources/runtimets/results/scan/alltypes_02.adm
+++ /dev/null
@@ -1 +0,0 @@
-{ "id": 10, "name": "Nancy", "age": 32.5f, "salary": 12.0d, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2") }
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/circle-intersect-circle/circle-intersect-circle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/circle-intersect-circle/circle-intersect-circle.1.adm
index c294669..af533e3 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/circle-intersect-circle/circle-intersect-circle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/circle-intersect-circle/circle-intersect-circle.1.adm
@@ -1,19 +1,6 @@
 { "id": 1 }
 { "id": 2 }
 { "id": 3 }
-{ "id": 4 }
-{ "id": 5 }
 { "id": 6 }
-{ "id": 7 }
-{ "id": 8 }
 { "id": 9 }
-{ "id": 10 }
-{ "id": 11 }
-{ "id": 12 }
-{ "id": 13 }
-{ "id": 14 }
-{ "id": 15 }
-{ "id": 16 }
-{ "id": 17 }
-{ "id": 18 }
-{ "id": 19 }
\ No newline at end of file
+{ "id": 11 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm
index 1778736..6a2a2c6 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm
@@ -16,4 +16,6 @@
 16
 17
 18
-19
\ No newline at end of file
+19
+20
+21
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/distance-between-points/distance-between-points.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/distance-between-points/distance-between-points.1.adm
index 2dc063d..be5e2b4 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/distance-between-points/distance-between-points.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/distance-between-points/distance-between-points.1.adm
@@ -16,4 +16,6 @@
 { "id": 16, "distance": 3.605551275463989d }
 { "id": 17, "distance": 8.112336284942828d }
 { "id": 18, "distance": 3.605551275463989d }
-{ "id": 19, "distance": 3.605551275463989d }
\ No newline at end of file
+{ "id": 19, "distance": 3.605551275463989d }
+{ "id": 20, "distance": 5.0d }
+{ "id": 21, "distance": 5.0d }
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-circle/line-intersect-circle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-circle/line-intersect-circle.1.adm
index b61f96b..3ed757e 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-circle/line-intersect-circle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-circle/line-intersect-circle.1.adm
@@ -1,5 +1,7 @@
 { "id": 2 }
 { "id": 3 }
+{ "id": 6 }
+{ "id": 7 }
 { "id": 10 }
 { "id": 11 }
 { "id": 12 }
@@ -8,4 +10,5 @@
 { "id": 15 }
 { "id": 16 }
 { "id": 18 }
-{ "id": 19 }
\ No newline at end of file
+{ "id": 19 }
+{ "id": 21 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-rectangle/line-intersect-rectangle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-rectangle/line-intersect-rectangle.1.adm
index bea5d1d..59506d2 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-rectangle/line-intersect-rectangle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/line-intersect-rectangle/line-intersect-rectangle.1.adm
@@ -10,4 +10,5 @@
 { "id": 15 }
 { "id": 16 }
 { "id": 18 }
-{ "id": 19 }
\ No newline at end of file
+{ "id": 19 }
+{ "id": 21 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/point-in-circle/point-in-circle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/point-in-circle/point-in-circle.1.adm
index ad75861..f8d6259 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/point-in-circle/point-in-circle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/point-in-circle/point-in-circle.1.adm
@@ -3,4 +3,6 @@
 { "id": 15 }
 { "id": 16 }
 { "id": 18 }
-{ "id": 19 }
\ No newline at end of file
+{ "id": 19 }
+{ "id": 20 }
+{ "id": 21 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/point-in-polygon/point-in-polygon.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/point-in-polygon/point-in-polygon.1.adm
index 2ffad7c..d22217a 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/point-in-polygon/point-in-polygon.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/point-in-polygon/point-in-polygon.1.adm
@@ -1 +1,2 @@
-{ "id": 12 }
\ No newline at end of file
+{ "id": 12 }
+{ "id": 20 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/point-in-rectangle/point-in-rectangle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/point-in-rectangle/point-in-rectangle.1.adm
index bc9537a..424f77a 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/point-in-rectangle/point-in-rectangle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/point-in-rectangle/point-in-rectangle.1.adm
@@ -1,2 +1,4 @@
 { "id": 1 }
-{ "id": 10 }
\ No newline at end of file
+{ "id": 10 }
+{ "id": 20 }
+{ "id": 21 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/point-on-line/point-on-line.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/point-on-line/point-on-line.1.adm
index d5bf8d4..f737ee8 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/point-on-line/point-on-line.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/point-on-line/point-on-line.1.adm
@@ -1,4 +1,5 @@
 { "id": 1 }
 { "id": 9 }
 { "id": 10 }
-{ "id": 17 }
\ No newline at end of file
+{ "id": 17 }
+{ "id": 21 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-circle/polygon-intersect-circle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-circle/polygon-intersect-circle.1.adm
index e5293c9..bb9a070 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-circle/polygon-intersect-circle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-circle/polygon-intersect-circle.1.adm
@@ -7,4 +7,5 @@
 { "id": 16 }
 { "id": 17 }
 { "id": 18 }
-{ "id": 19 }
\ No newline at end of file
+{ "id": 19 }
+{ "id": 20 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-rectangle/polygon-intersect-rectangle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-rectangle/polygon-intersect-rectangle.1.adm
index a602b8d..aa83a89 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-rectangle/polygon-intersect-rectangle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/polygon-intersect-rectangle/polygon-intersect-rectangle.1.adm
@@ -5,4 +5,6 @@
 { "id": 6 }
 { "id": 9 }
 { "id": 12 }
-{ "id": 17 }
\ No newline at end of file
+{ "id": 17 }
+{ "id": 20 }
+{ "id": 21 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-circle/rectangle-intersect-circle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-circle/rectangle-intersect-circle.1.adm
index 62f32d0..260b1bf 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-circle/rectangle-intersect-circle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-circle/rectangle-intersect-circle.1.adm
@@ -16,3 +16,5 @@
 { "id": 17 }
 { "id": 18 }
 { "id": 19 }
+{ "id": 20 }
+{ "id": 21 }
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-rectangle/rectangle-intersect-rectangle.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-rectangle/rectangle-intersect-rectangle.1.adm
index e18550c..fed2dc7 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-rectangle/rectangle-intersect-rectangle.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/rectangle-intersect-rectangle/rectangle-intersect-rectangle.1.adm
@@ -1,2 +1,3 @@
 { "id": 1 }
-{ "id": 7 }
\ No newline at end of file
+{ "id": 7 }
+{ "id": 21 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index a883531..f6b3192 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -743,6 +743,11 @@
     -->
   </test-group>
   <test-group name="dml">
+     <test-case FilePath="dml">
+      <compilation-unit name="query-issue288">
+        <output-dir compare="Text">query-issue288</output-dir>
+      </compilation-unit>
+    </test-case>
     <test-case FilePath="dml">
       <compilation-unit name="query-issue205">
         <output-dir compare="Text">query-issue205</output-dir>
@@ -925,6 +930,86 @@
         <output-dir compare="Text">scan-insert-rtree-secondary-index</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-fuzzy-ngram-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-fuzzy-word-secondary-index">
+        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-insert-inverted-index-fuzzy-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-insert-inverted-index-fuzzy-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-fuzzy-ngram-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-ngram-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-fuzzy-word-secondary-index">
+        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-word-secondary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-ngram-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="scan-delete-inverted-index-fuzzy-word-secondary-index-nullable">
+        <output-dir compare="Text">scan-delete-inverted-index-fuzzy-word-secondary-index-nullable</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="employee">
     <test-case FilePath="employee">
@@ -1666,6 +1751,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="btree-index-composite-key-mixed-intervals">
+        <output-dir compare="Text">btree-index-composite-key-mixed-intervals</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="btree-index-rewrite-multiple">
         <output-dir compare="Text">btree-index-rewrite-multiple</output-dir>
       </compilation-unit>
@@ -1676,6 +1766,56 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-ngram-contains">
+        <output-dir compare="Text">fuzzy-inverted-index-ngram-contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-ngram-edit-distance-panic">
+        <output-dir compare="Text">fuzzy-inverted-index-ngram-edit-distance-panic</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-ngram-edit-distance">
+        <output-dir compare="Text">fuzzy-inverted-index-ngram-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-ngram-jaccard">
+        <output-dir compare="Text">fuzzy-inverted-index-ngram-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-olist-edit-distance-panic">
+        <output-dir compare="Text">fuzzy-inverted-index-olist-edit-distance-panic</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-olist-edit-distance">
+        <output-dir compare="Text">fuzzy-inverted-index-olist-edit-distance</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-olist-jaccard">
+        <output-dir compare="Text">fuzzy-inverted-index-olist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-ulist-jaccard">
+        <output-dir compare="Text">fuzzy-inverted-index-ulist-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-word-contains">
+        <output-dir compare="Text">fuzzy-inverted-index-word-contains</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
+      <compilation-unit name="fuzzy-inverted-index-word-jaccard">
+        <output-dir compare="Text">fuzzy-inverted-index-word-jaccard</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="index-selection">
       <compilation-unit name="inverted-index-ngram-contains">
         <output-dir compare="Text">inverted-index-ngram-contains</output-dir>
       </compilation-unit>
@@ -3911,6 +4051,11 @@
         <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error> 
       </compilation-unit>
     </test-case>
+    <test-case FilePath="load">
+      <compilation-unit name="issue289_query">
+        <output-dir compare="Text">issue289_query</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="hints">
     <test-case FilePath="hints">
@@ -4025,5 +4170,17 @@
       </compilation-unit>
   	</test-case>
   </test-group>
+  <test-group name="leftouterjoin">
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query_issue285">
+        <output-dir compare="Text">query_issue285</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="leftouterjoin">
+      <compilation-unit name="query_issue285-2">
+        <output-dir compare="Text">query_issue285-2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>
 
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index dca639b..8e3e203 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -260,7 +260,7 @@
     datasetName = nameComponents.second;
    }
     
-    (<LEFTPAREN>)? query = Query() (<RIGHTPAREN>)? (";")?
+    query = Query() (";")?
    {return new InsertStatement(dataverseName, datasetName, query,  getVarCounter());}
 }
 
@@ -399,9 +399,10 @@
   	("," <IDENTIFIER> { cis.addFieldExpr(token.image); })*
   <RIGHTPAREN>
     ("type"
-  		("btree" { cis.setIndexType(IndexType.BTREE); }
-  		| "keyword" { cis.setIndexType(IndexType.WORD_INVIX); }  		
+  		("btree" { cis.setIndexType(IndexType.BTREE); }  		
   		| "rtree" { cis.setIndexType(IndexType.RTREE); }
+  		| "keyword" { cis.setIndexType(IndexType.WORD_INVIX); }
+  		| "fuzzy keyword" { cis.setIndexType(IndexType.FUZZY_WORD_INVIX); }
   		| "ngram"
   		  <LEFTPAREN>
   		  (<INTEGER_LITERAL>
@@ -410,7 +411,16 @@
   		      cis.setGramLength(Integer.valueOf(token.image));
   		    }
   		  )
-  		  <RIGHTPAREN>	  		  
+  		  <RIGHTPAREN>
+  		| "fuzzy ngram"
+  		  <LEFTPAREN>
+  		  (<INTEGER_LITERAL>
+  		    {
+  		      cis.setIndexType(IndexType.FUZZY_NGRAM_INVIX);
+  		      cis.setGramLength(Integer.valueOf(token.image));
+  		    }
+  		  )
+  		  <RIGHTPAREN>
 		)
   	";"  	
   	| ";"
diff --git a/asterix-common/pom.xml b/asterix-common/pom.xml
index 87a36ab..abd78c4 100644
--- a/asterix-common/pom.xml
+++ b/asterix-common/pom.xml
@@ -6,9 +6,7 @@
 		<groupId>edu.uci.ics.asterix</groupId>
 		<version>0.0.4-SNAPSHOT</version>
 	</parent>
-	<groupId>edu.uci.ics.asterix</groupId>
 	<artifactId>asterix-common</artifactId>
-	<version>0.0.4-SNAPSHOT</version>
 	<build>
 		<plugins>
 			<plugin>
@@ -40,6 +38,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.4-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 00a2651..0f1de56 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
@@ -14,12 +14,10 @@
  */
 package edu.uci.ics.asterix.common.api;
 
-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.api.application.ICCApplicationContext;
-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;
 
 /*
@@ -48,13 +46,8 @@
     }
 
     @Override
-    public IIndexRegistryProvider<IIndex> getIndexRegistryProvider() {
-        return AsterixIndexRegistryProvider.INSTANCE;
-    }
-
-    @Override
     public IStorageManagerInterface getStorageManagerInterface() {
-        return AsterixStorageManagerInterface.INSTANCE;
+        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
     }
 
     @Override
@@ -62,4 +55,8 @@
         return appCtx;
     }
 
+    @Override
+    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
+        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+    }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
index e860a0a..cbe96f7 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/DatasetConfig.java
@@ -41,7 +41,9 @@
         BTREE,        
         RTREE,
         WORD_INVIX,
-        NGRAM_INVIX
+        NGRAM_INVIX,
+        FUZZY_WORD_INVIX,
+        FUZZY_NGRAM_INVIX
     }
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
index 11bdb67..1183b65 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
@@ -35,6 +35,10 @@
 
     public static int DEFAULT_INPUT_DATA_COLUMN = 0;
 
+    public static int DEFAULT_INDEX_MEM_PAGE_SIZE = 32768;
+
+    public static int DEFAULT_INDEX_MEM_NUM_PAGES = 1000;
+
     public static int getFrameSize() {
         int frameSize = GlobalConfig.DEFAULT_FRAME_SIZE;
         String frameSizeStr = System.getProperty(GlobalConfig.FRAME_SIZE_PROPERTY);
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 f84f294..21aa203 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
@@ -5,11 +5,24 @@
 
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTrackerFactory;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepository;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceRepositoryFactory;
+import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 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.ILSMIOOperationCallbackProvider;
+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.ILSMOperationTrackerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.ConstantMergePolicy;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
 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.DelayPageCleanerPolicy;
@@ -20,15 +33,30 @@
 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.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 static final int DEFAULT_MAX_OPEN_FILES = Integer.MAX_VALUE;
     private final INCApplicationContext ncApplicationContext;
 
-    private IndexRegistry<IIndex> indexRegistry;
+    private IIndexLifecycleManager indexLifecycleManager;
     private IFileMapManager fileMapManager;
     private IBufferCache bufferCache;
-    private TransactionProvider provider;
+    private TransactionSubsystem txnSubsystem;
+
+    private ILSMMergePolicy mergePolicy;
+    private ILSMOperationTrackerFactory lsmBTreeOpTrackerFactory;
+    private ILSMOperationTrackerFactory lsmRTreeOpTrackerFactory;
+    private ILSMOperationTrackerFactory lsmInvertedIndexOpTrackerFactory;
+    private ILSMIOOperationScheduler lsmIOScheduler;
+    private PersistentLocalResourceRepository localResourceRepository;
+    private ResourceIdFactory resourceIdFactory;
+    private IIOManager ioManager;
 
     public AsterixAppRuntimeContext(INCApplicationContext ncApplicationContext) {
         this.ncApplicationContext = ncApplicationContext;
@@ -38,21 +66,30 @@
         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();
+        ioManager = ncApplicationContext.getRootContext().getIOManager();
+        indexLifecycleManager = new IndexLifecycleManager(DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET);
+        IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProviderForRecovery(
+                this);
+        txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider);
         IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000);
-        bufferCache = new BufferCache(ioMgr, allocator, prs, pcp, fileMapManager, pageSize, numPages, Integer.MAX_VALUE);
+        bufferCache = new BufferCache(ioManager, allocator, prs, pcp, fileMapManager, pageSize, numPages,
+                DEFAULT_MAX_OPEN_FILES);
 
-        // Initialize the index registry
-        indexRegistry = new IndexRegistry<IIndex>();
+        lsmIOScheduler = SynchronousScheduler.INSTANCE;
+        mergePolicy = new ConstantMergePolicy(3);
+        lsmBTreeOpTrackerFactory = new IndexOperationTrackerFactory(LSMBTreeIOOperationCallbackFactory.INSTANCE);
+        lsmRTreeOpTrackerFactory = new IndexOperationTrackerFactory(LSMRTreeIOOperationCallbackFactory.INSTANCE);
+        lsmInvertedIndexOpTrackerFactory = new IndexOperationTrackerFactory(
+                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE);
 
-        // Initialize the transaction sub-system
-        provider = new TransactionProvider(ncApplicationContext.getNodeId());
+        ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
+                ioManager);
+        localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
+                .createRepository();
+        resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
     }
 
     private int getBufferCachePageSize() {
@@ -110,12 +147,59 @@
         return fileMapManager;
     }
 
-    public IndexRegistry<IIndex> getIndexRegistry() {
-        return indexRegistry;
+    public TransactionSubsystem getTransactionSubsystem() {
+        return txnSubsystem;
     }
 
-    public TransactionProvider getTransactionProvider() {
-        return provider;
+    public IIndexLifecycleManager getIndexLifecycleManager() {
+        return indexLifecycleManager;
     }
 
+    public ILSMMergePolicy getLSMMergePolicy() {
+        return mergePolicy;
+    }
+
+    public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory() {
+        return lsmBTreeOpTrackerFactory;
+    }
+
+    public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory() {
+        return lsmRTreeOpTrackerFactory;
+    }
+
+    public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory() {
+        return lsmInvertedIndexOpTrackerFactory;
+    }
+
+    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider() {
+        return AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER;
+    }
+
+    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
+        return AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER;
+    }
+
+    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
+        return AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER;
+    }
+
+    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
+        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+    }
+
+    public ILSMIOOperationScheduler getLSMIOScheduler() {
+        return lsmIOScheduler;
+    }
+
+    public ILocalResourceRepository getLocalResourceRepository() {
+        return localResourceRepository;
+    }
+
+    public ResourceIdFactory getResourceIdFactory() {
+        return resourceIdFactory;
+    }
+
+    public IIOManager getIOManager() {
+        return ioManager;
+    }
 }
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContextProviderForRecovery.java
new file mode 100644
index 0000000..8f2b96f
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixAppRuntimeContextProviderForRecovery.java
@@ -0,0 +1,104 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+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.ILSMOperationTrackerFactory;
+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 class AsterixAppRuntimeContextProviderForRecovery implements IAsterixAppRuntimeContextProvider {
+
+    private final AsterixAppRuntimeContext asterixAppRuntimeContext;
+
+    public AsterixAppRuntimeContextProviderForRecovery(AsterixAppRuntimeContext asterixAppRuntimeContext) {
+        this.asterixAppRuntimeContext = asterixAppRuntimeContext;
+    }
+
+    @Override
+    public IBufferCache getBufferCache() {
+        return asterixAppRuntimeContext.getBufferCache();
+    }
+
+    @Override
+    public IFileMapProvider getFileMapManager() {
+        return asterixAppRuntimeContext.getFileMapManager();
+    }
+
+    @Override
+    public TransactionSubsystem getTransactionSubsystem() {
+        return asterixAppRuntimeContext.getTransactionSubsystem();
+    }
+
+    @Override
+    public IIndexLifecycleManager getIndexLifecycleManager() {
+        return asterixAppRuntimeContext.getIndexLifecycleManager();
+    }
+
+    @Override
+    public ILSMMergePolicy getLSMMergePolicy() {
+        return asterixAppRuntimeContext.getLSMMergePolicy();
+    }
+
+    @Override
+    public ILSMIOOperationScheduler getLSMIOScheduler() {
+        return asterixAppRuntimeContext.getLSMIOScheduler();
+    }
+
+    @Override
+    public ILocalResourceRepository getLocalResourceRepository() {
+        return asterixAppRuntimeContext.getLocalResourceRepository();
+    }
+
+    @Override
+    public ResourceIdFactory getResourceIdFactory() {
+        return asterixAppRuntimeContext.getResourceIdFactory();
+    }
+
+    @Override
+    public IIOManager getIOManager() {
+        return asterixAppRuntimeContext.getIOManager();
+    }
+
+    @Override
+    public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory() {
+        return asterixAppRuntimeContext.getLSMBTreeOperationTrackerFactory();
+    }
+
+    @Override
+    public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory() {
+        return asterixAppRuntimeContext.getLSMRTreeOperationTrackerFactory();
+    }
+
+    @Override
+    public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory() {
+        return asterixAppRuntimeContext.getLSMInvertedIndexOperationTrackerFactory();
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider() {
+        return asterixAppRuntimeContext.getLSMBTreeIOOperationCallbackProvider();
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
+        return asterixAppRuntimeContext.getLSMRTreeIOOperationCallbackProvider();
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
+        return asterixAppRuntimeContext.getLSMInvertedIndexIOOperationCallbackProvider();
+    }
+
+    @Override
+    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
+        return asterixAppRuntimeContext.getNoOpIOOperationCallbackProvider();
+    }
+
+}
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..fad0a38
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixRuntimeComponentsProvider.java
@@ -0,0 +1,96 @@
+package edu.uci.ics.asterix.common.context;
+
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+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.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+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.ILSMIndex;
+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.ILSMOperationTrackerFactory;
+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 class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
+        ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider, ILSMOperationTrackerFactory,
+        ILSMIOOperationCallbackProvider {
+    private static final long serialVersionUID = 1L;
+
+    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+
+    public static final AsterixRuntimeComponentsProvider LSMBTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
+            LSMBTreeIOOperationCallbackFactory.INSTANCE);
+    public static final AsterixRuntimeComponentsProvider LSMRTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
+            LSMRTreeIOOperationCallbackFactory.INSTANCE);
+    public static final AsterixRuntimeComponentsProvider LSMINVERTEDINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(
+            LSMInvertedIndexIOOperationCallbackFactory.INSTANCE);
+    public static final AsterixRuntimeComponentsProvider NOINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(null);
+
+    private AsterixRuntimeComponentsProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+    }
+
+    @Override
+    public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
+        return new IndexOperationTracker(index, ioOpCallbackFactory);
+    }
+
+    @Override
+    public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
+        return ((IndexOperationTracker) index.getOperationTracker()).getIOOperationCallback();
+    }
+
+    @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/context/TransactionSubsystemProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java
new file mode 100644
index 0000000..85e5375
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/TransactionSubsystemProvider.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2009-2010 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.common.context;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * The purpose of this provider is to work around a cyclic dependency between asterix-common and asterix-transactions.
+ * The operation callbacks would depend on the AsterixAppRuntimeContext to get the transaction subsystem,
+ * while at the same time the AsterixAppRuntimeContext depends on asterix-transactions for the TransactionSubsystem.
+ */
+public class TransactionSubsystemProvider implements ITransactionSubsystemProvider {
+    @Override
+    public TransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx) {
+        AsterixAppRuntimeContext runtimeCtx = (AsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
+                .getApplicationObject();
+        return runtimeCtx.getTransactionSubsystem();
+    }
+}
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 032e832..401af28 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
@@ -15,8 +15,7 @@
 package edu.uci.ics.asterix.common.dataflow;
 
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
-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;
 
 /**
@@ -24,13 +23,7 @@
  * ICCApplicationContext implementation.
  */
 public interface IAsterixApplicationContextInfo {
-
-    /**
-     * Returns an instance of the implementation for IIndexRegistryProvider.
-     * 
-     * @return IIndexRegistryProvider implementation instance
-     */
-    public IIndexRegistryProvider<IIndex> getIndexRegistryProvider();
+    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider();
 
     /**
      * Returns an instance of the implementation for IStorageManagerInterface.
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
index d3c338d..66c0102 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
@@ -41,7 +41,7 @@
 public class EventDriver {
 
     public static final String CLIENT_NODE_ID = "client_node";
-    public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null);
+    public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null, null);
 
     private static String eventsDir;
     private static Events events;
@@ -141,9 +141,9 @@
         args.add(scriptDirSuffix);
         Node clientNode = new Node();
         clientNode.setId("client");
-        clientNode.setIp("127.0.0.1");
+        clientNode.setClusterIp("127.0.0.1");
         for (Node node : cluster.getNode()) {
-            args.add(node.getIp());
+            args.add(node.getClusterIp());
         }
         EventUtil.executeLocalScript(clientNode, eventsDir + "/" + "events" + "/" + "prepare.sh", args);
     }
@@ -153,11 +153,10 @@
         args.add(scriptDirSuffix);
         Node clientNode = new Node();
         clientNode.setId("client");
-        clientNode.setIp("127.0.0.1");
+        clientNode.setClusterIp("127.0.0.1");
         for (Node node : cluster.getNode()) {
-            args.add(node.getIp());
+            args.add(node.getClusterIp());
         }
         EventUtil.executeLocalScript(clientNode, eventsDir + "/" + "events" + "/" + "cleanup.sh", args);
     }
-
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
index 0157f65..6d89c88 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
@@ -18,6 +18,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.StringWriter;
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -44,21 +45,28 @@
         List<String> pargs = new ArrayList<String>();
         pargs.add("/bin/bash");
         pargs.add(client.getEventsDir() + File.separator + "scripts" + File.separator + EXECUTE_SCRIPT);
-        StringBuffer envBuffer = new StringBuffer(IP_LOCATION + "=" + node.getIp() + " ");
+        StringBuffer envBuffer = new StringBuffer(IP_LOCATION + "=" + node.getClusterIp() + " ");
         if (!node.getId().equals(EventDriver.CLIENT_NODE_ID) && cluster.getEnv() != null) {
             for (Property p : cluster.getEnv().getProperty()) {
                 if (p.getKey().equals("JAVA_HOME")) {
                     String val = node.getJavaHome() == null ? p.getValue() : node.getJavaHome();
                     envBuffer.append(p.getKey() + "=" + val + " ");
                 } else if (p.getKey().equals("JAVA_OPTS")) {
-                    String val = "\"" + "-Xmx"
-                            + (node.getJavaHeap() == null ? cluster.getJavaHeap() : node.getJavaHeap());
-                    if (node.getDebug() != null) {
-                        val = val + " " + "-Xdebug -Xrunjdwp:transport=dt_socket,address=" + node.getDebug().intValue()
-                                + "," + "server=y,suspend=n";
+                    StringBuilder builder = new StringBuilder();
+                    builder.append("\"");
+                    String javaOpts = (node.getJavaOpts() == null ? cluster.getJavaOpts() : node.getJavaOpts());
+                    if (javaOpts != null) {
+                        builder.append(javaOpts);
                     }
-                    val = val + "\"";
-                    envBuffer.append(p.getKey() + "=" + val + " ");
+                    if (cluster.isDebugEnabled() != null && cluster.isDebugEnabled().booleanValue()) {
+                        BigInteger debugPort = node.getDebug() == null ? cluster.getDebug() : node.getDebug();
+                        if (debugPort != null) {
+                            builder.append("-Xdebug -Xrunjdwp:transport=dt_socket,address=" + debugPort.intValue()
+                                    + "," + "server=y,suspend=n");
+                        }
+                    }
+                    builder.append("\"");
+                    envBuffer.append(p.getKey() + "=" + builder + " ");
                 } else {
                     envBuffer.append(p.getKey() + "=" + p.getValue() + " ");
                 }
@@ -66,6 +74,7 @@
             }
             pargs.add(cluster.getUsername() == null ? System.getProperty("user.name") : cluster.getUsername());
         }
+
         StringBuffer argBuffer = new StringBuffer();
         if (args != null && args.size() > 0) {
             for (String arg : args) {
@@ -74,7 +83,7 @@
         }
 
         ProcessBuilder pb = new ProcessBuilder(pargs);
-        pb.environment().put(IP_LOCATION, node.getIp());
+        pb.environment().put(IP_LOCATION, node.getClusterIp());
         pb.environment().put(CLUSTER_ENV, envBuffer.toString());
         pb.environment().put(SCRIPT, script);
         pb.environment().put(ARGS, argBuffer.toString());
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
index bf31e9f..e6c58eb 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
@@ -181,16 +181,16 @@
 		}
 
 		if (nodeid.equals(cluster.getMasterNode().getId())) {
-			String ram = cluster.getMasterNode().getJavaHeap() == null ? cluster
-					.getJavaHeap() : cluster.getMasterNode().getJavaHeap();
+			String javaOpts = cluster.getMasterNode().getJavaOpts() == null ? cluster
+					.getJavaOpts() : cluster.getMasterNode().getJavaOpts();
 			String logDir = cluster.getMasterNode().getLogdir() == null ? cluster
 					.getLogdir() : cluster.getMasterNode().getLogdir();
 			String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster
 					.getJavaHome() : cluster.getMasterNode().getJavaHome();
 			BigInteger debug = cluster.getMasterNode().getDebug();
 			return new Node(cluster.getMasterNode().getId(), cluster
-					.getMasterNode().getIp(), ram, javaHome, logDir, null,
-					debug);
+					.getMasterNode().getClusterIp(), javaHome, javaOpts,
+					logDir, null, null, debug);
 		}
 
 		List<Node> nodeList = cluster.getNode();
@@ -218,7 +218,7 @@
 		pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
 		StringBuffer argBuffer = new StringBuffer();
 		String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION
-				+ "=" + node.getIp();
+				+ "=" + node.getClusterIp();
 		if (args != null) {
 			for (String arg : args) {
 				argBuffer.append(arg + " ");
@@ -226,7 +226,7 @@
 		}
 		ProcessBuilder pb = new ProcessBuilder(pargs);
 		pb.environment().putAll(EventDriver.getEnvironment());
-		pb.environment().put(IP_LOCATION, node.getIp());
+		pb.environment().put(IP_LOCATION, node.getClusterIp());
 		pb.environment().put(CLUSTER_ENV, env);
 		pb.environment().put(SCRIPT, script);
 		pb.environment().put(ARGS, argBuffer.toString());
@@ -243,7 +243,7 @@
 		}
 		ProcessBuilder pb = new ProcessBuilder(pargs);
 		pb.environment().putAll(EventDriver.getEnvironment());
-		pb.environment().put(IP_LOCATION, node.getIp());
+		pb.environment().put(IP_LOCATION, node.getClusterIp());
 		pb.start();
 	}
 
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
index 4cd4b82..6f4336d 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
@@ -38,126 +38,123 @@
 
 public class EventrixClient {
 
-	private static final Logger LOGGER = Logger.getLogger(EventrixClient.class
-			.getName());
+    private static final Logger LOGGER = Logger.getLogger(EventrixClient.class.getName());
 
-	private EventTask[] tasks;
-	private boolean dryRun = false;
-	private LinkedBlockingQueue<EventTaskReport> msgInbox = new LinkedBlockingQueue<EventTaskReport>();
-	private AtomicInteger pendingTasks = new AtomicInteger(0);
-	private final Cluster cluster;
-	private IPatternListener listener;
-	private IOutputHandler outputHandler;
-	private Events events;
-	private String eventsDir;
+    private EventTask[] tasks;
+    private boolean dryRun = false;
+    private LinkedBlockingQueue<EventTaskReport> msgInbox = new LinkedBlockingQueue<EventTaskReport>();
+    private AtomicInteger pendingTasks = new AtomicInteger(0);
+    private final Cluster cluster;
+    private IPatternListener listener;
+    private IOutputHandler outputHandler;
+    private Events events;
+    private String eventsDir;
 
-	public EventrixClient(String eventsDir, Cluster cluster, boolean dryRun,
-			IOutputHandler outputHandler) throws Exception {
-		this.eventsDir = eventsDir;
-		this.events = initializeEvents();
-		this.cluster = cluster;
-		this.dryRun = dryRun;
-		this.outputHandler = outputHandler;
-		if (!dryRun) {
-			initializeCluster(eventsDir);
-		}
-	}
+    public EventrixClient(String eventsDir, Cluster cluster, boolean dryRun, IOutputHandler outputHandler)
+            throws Exception {
+        this.eventsDir = eventsDir;
+        this.events = initializeEvents();
+        this.cluster = cluster;
+        this.dryRun = dryRun;
+        this.outputHandler = outputHandler;
+        if (!dryRun) {
+            initializeCluster(eventsDir);
+        }
+    }
 
-	public void submit(Patterns patterns) throws Exception {
-		initTasks(patterns);
-		try {
-			waitForCompletion();
-		} catch (InterruptedException ie) {
-			LOGGER.info("Interrupted exception :" + ie);
-		} catch (Exception e) {
-			throw e;
-		}
+    public void submit(Patterns patterns) throws Exception {
+        initTasks(patterns);
+        try {
+            waitForCompletion();
+        } catch (InterruptedException ie) {
+            LOGGER.info("Interrupted exception :" + ie);
+        } catch (Exception e) {
+            throw e;
+        }
 
-	}
+    }
 
-	public void submit(Patterns patterns, IPatternListener listener)
-			throws Exception {
-		this.listener = listener;
-		initTasks(patterns);
-	}
+    public void submit(Patterns patterns, IPatternListener listener) throws Exception {
+        this.listener = listener;
+        initTasks(patterns);
+    }
 
-	private void initTasks(Patterns patterns) {
-		tasks = new EventTask[patterns.getPattern().size()];
-		pendingTasks.set(tasks.length);
-		int index = 0;
-		for (Pattern pattern : patterns.getPattern()) {
-			tasks[index] = new EventTask(pattern, this);
-			tasks[index].start();
-			index++;
-		}
-	}
+    private void initTasks(Patterns patterns) {
+        tasks = new EventTask[patterns.getPattern().size()];
+        pendingTasks.set(tasks.length);
+        int index = 0;
+        for (Pattern pattern : patterns.getPattern()) {
+            tasks[index] = new EventTask(pattern, this);
+            tasks[index].start();
+            index++;
+        }
+    }
 
-	public Cluster getCluster() {
-		return cluster;
-	}
+    public Cluster getCluster() {
+        return cluster;
+    }
 
-	public boolean isDryRun() {
-		return dryRun;
-	}
+    public boolean isDryRun() {
+        return dryRun;
+    }
 
-	public Events getEvents() {
-		return events;
-	}
+    public Events getEvents() {
+        return events;
+    }
 
-	public String getEventsDir() {
-		return eventsDir;
-	}
+    public String getEventsDir() {
+        return eventsDir;
+    }
 
-	public synchronized void notifyCompletion(EventTaskReport report) {
+    public synchronized void notifyCompletion(EventTaskReport report) {
 
-		if (report.isSuccess()) {
-			if (listener != null) {
-				pendingTasks.decrementAndGet();
-				listener.eventCompleted(report);
-				if (pendingTasks.get() == 0) {
-					listener.jobCompleted();
-				}
-			} else {
-				try {
-					msgInbox.put(report);
-				} catch (InterruptedException e) {
-				}
-			}
-		} else {
-			for (EventTask t : tasks) {
-				if (t.getState() == EventTask.State.INITIALIZED
-						|| t.getState() == EventTask.State.IN_PROGRESS) {
-					t.cancel();
-				}
-			}
-			if (listener != null) {
-				listener.jobFailed(report);
-			} else {
-				try {
-					msgInbox.put(report);
-				} catch (InterruptedException e) {
-				}
-			}
-		}
-	}
+        if (report.isSuccess()) {
+            if (listener != null) {
+                pendingTasks.decrementAndGet();
+                listener.eventCompleted(report);
+                if (pendingTasks.get() == 0) {
+                    listener.jobCompleted();
+                }
+            } else {
+                try {
+                    msgInbox.put(report);
+                } catch (InterruptedException e) {
+                }
+            }
+        } else {
+            for (EventTask t : tasks) {
+                if (t.getState() == EventTask.State.INITIALIZED || t.getState() == EventTask.State.IN_PROGRESS) {
+                    t.cancel();
+                }
+            }
+            if (listener != null) {
+                listener.jobFailed(report);
+            } else {
+                try {
+                    msgInbox.put(report);
+                } catch (InterruptedException e) {
+                }
+            }
+        }
+    }
 
-	private void waitForCompletion() throws Exception {
-		while (true) {
-			EventTaskReport report = msgInbox.take();
-			if (report.isSuccess()) {
-				if (pendingTasks.decrementAndGet() == 0) {
-					break;
-				}
-			} else {
-				throw new RuntimeException(report.getException().getMessage());
-			}
-		}
-	}
+    private void waitForCompletion() throws Exception {
+        while (true) {
+            EventTaskReport report = msgInbox.take();
+            if (report.isSuccess()) {
+                if (pendingTasks.decrementAndGet() == 0) {
+                    break;
+                }
+            } else {
+                throw new RuntimeException(report.getException().getMessage());
+            }
+        }
+    }
 
-	private void initializeCluster(String eventsDir) throws Exception {
-		Patterns patterns = initPattern(eventsDir);
-		submit(patterns);
-	}
+    private void initializeCluster(String eventsDir) throws Exception {
+        Patterns patterns = initPattern(eventsDir);
+        submit(patterns);
+    }
 
 	private Patterns initPattern(String eventsDir) {
 		Nodeid nodeid = new Nodeid(new Value(null,
@@ -167,37 +164,35 @@
 		String username = cluster.getUsername() == null ? System
 				.getProperty("user.name") : cluster.getUsername();
 		patternList.add(getDirectoryTransferPattern(username, eventsDir,
-				nodeid, cluster.getMasterNode().getIp(), workingDir));
+				nodeid, cluster.getMasterNode().getClusterIp(), workingDir));
 
 		if (!cluster.getWorkingDir().isNFS()) {
 			for (Node node : cluster.getNode()) {
 				patternList.add(getDirectoryTransferPattern(username,
-						eventsDir, nodeid, node.getIp(), workingDir));
+						eventsDir, nodeid, node.getClusterIp(), workingDir));
 			}
 		}
 		Patterns patterns = new Patterns(patternList);
 		return patterns;
 	}
 
-	private Pattern getDirectoryTransferPattern(String username, String src,
-			Nodeid srcNode, String destNodeIp, String destDir) {
-		String pargs = username + " " + src + " " + destNodeIp + " " + destDir;
-		Event event = new Event("directory_transfer", srcNode, pargs);
-		return new Pattern(null, 1, null, event);
-	}
+    private Pattern getDirectoryTransferPattern(String username, String src, Nodeid srcNode, String destNodeIp,
+            String destDir) {
+        String pargs = username + " " + src + " " + destNodeIp + " " + destDir;
+        Event event = new Event("directory_transfer", srcNode, pargs);
+        return new Pattern(null, 1, null, event);
+    }
 
-	public IOutputHandler getErrorHandler() {
-		return outputHandler;
-	}
+    public IOutputHandler getErrorHandler() {
+        return outputHandler;
+    }
 
-	private Events initializeEvents() throws JAXBException,
-			FileNotFoundException {
-		File file = new File(eventsDir + File.separator + "events"
-				+ File.separator + "events.xml");
-		JAXBContext eventCtx = JAXBContext.newInstance(Events.class);
-		Unmarshaller unmarshaller = eventCtx.createUnmarshaller();
-		events = (Events) unmarshaller.unmarshal(file);
-		return events;
-	}
+    private Events initializeEvents() throws JAXBException, FileNotFoundException {
+        File file = new File(eventsDir + File.separator + "events" + File.separator + "events.xml");
+        JAXBContext eventCtx = JAXBContext.newInstance(Events.class);
+        Unmarshaller unmarshaller = eventCtx.createUnmarshaller();
+        events = (Events) unmarshaller.unmarshal(file);
+        return events;
+    }
 
 }
diff --git a/asterix-events/src/main/resources/events/backup/backup.sh b/asterix-events/src/main/resources/events/backup/backup.sh
index e10a4c9..556ca39 100755
--- a/asterix-events/src/main/resources/events/backup/backup.sh
+++ b/asterix-events/src/main/resources/events/backup/backup.sh
@@ -1,35 +1,72 @@
 WORKING_DIR=$1
 ASTERIX_INSTANCE_NAME=$2
-ASTERIX_DATA_DIR=$3
-BACKUP_ID=$4
-BACKUP_DIR=$5
-BACKUP_TYPE=$6
-NODE_ID=$7
+ASTERIX_IODEVICES=$3
+NODE_STORE=$4
+ASTERIX_ROOT_METADATA_DIR=$5
+TXN_LOG_DIR_NAME=$6
+BACKUP_ID=$7
+BACKUP_DIR=$8
+BACKUP_TYPE=$9
+NODE_ID=${10}
 
-nodeStores=$(echo $ASTERIX_DATA_DIR | tr "," "\n")
+nodeIODevices=$(echo $ASTERIX_IODEVICES | tr "," "\n")
 
 if [ $BACKUP_TYPE == "hdfs" ];
 then
-  HDFS_URL=$8
-  HADOOP_VERSION=$9
+  HDFS_URL=${11}
+  HADOOP_VERSION=${12}
   export HADOOP_HOME=$WORKING_DIR/hadoop-$HADOOP_VERSION
-  for nodeStore in $nodeStores
+  index=1
+  for nodeIODevice in $nodeIODevices
   do
-    MANGLED_DIR_NAME=`echo $nodeStores | tr / _`
-    NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$MANGLED_DIR_NAME
-    echo "$HADOOP_HOME/bin/hadoop fs -copyFromLocal $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/ $HDFS_URL/$NODE_BACKUP_DIR/" >> ~/backup.log
-    $HADOOP_HOME/bin/hadoop fs -copyFromLocal $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/ $HDFS_URL/$NODE_BACKUP_DIR/
+    STORE_DIR=$nodeIODevice/$NODE_STORE
+    TXN_LOG_DIR=$nodeIODevice/$TXN_LOG_DIR_NAME
+    NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/
+   
+    # make the destination directory 
+    $HADOOP_HOME/bin/hadoop fs -mkdir $STORE_DIR $HDFS_URL/$NODE_BACKUP_DIR
+
+    # copy store directory
+    $HADOOP_HOME/bin/hadoop fs -copyFromLocal $STORE_DIR $HDFS_URL/$NODE_BACKUP_DIR/
+
+    # copy asterix root metadata directory and log directory from the primary(first) iodevice
+    if [ $index -eq 1 ];
+    then
+      # copy asterix root metadata directory
+      $HADOOP_HOME/bin/hadoop fs -copyFromLocal $nodeIODevice/$ASTERIX_ROOT_METADATA_DIR $HDFS_URL/$NODE_BACKUP_DIR/
+
+      # copy log directory 
+      $HADOOP_HOME/bin/hadoop fs -copyFromLocal $TXN_LOG_DIR $HDFS_URL/$NODE_BACKUP_DIR/
+    fi
+
+    index=`expr $index + 1`
   done
 else 
-  for nodeStore in $nodeStores
+  index=1
+  for nodeIODevice in $nodeIODevices
   do
-    MANGLED_DIR_NAME=`echo $nodeStores | tr / _`
-    NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$MANGLED_DIR_NAME
+    STORE_DIR=$nodeIODevice/$NODE_STORE
+    TXN_LOG_DIR=$nodeIODevice/$TXN_LOG_DIR_NAME
+    NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID
+
+    # create the backup directory, if it does not exists
     if [ ! -d $NODE_BACKUP_DIR ];
     then
       mkdir -p $NODE_BACKUP_DIR
     fi
-    echo "cp -r  $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/* $NODE_BACKUP_DIR/" >> ~/backup.log
-    cp -r  $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/* $NODE_BACKUP_DIR/
+
+    # copy store directory
+    cp -r $STORE_DIR $NODE_BACKUP_DIR/
+
+    # copy asterix root metadata directory and log directory from the primary(first) iodevice
+    if [ $index -eq 1 ];
+    then
+      cp -r $nodeIODevice/$ASTERIX_ROOT_METADATA_DIR  $NODE_BACKUP_DIR/
+
+      # copy log directory
+      cp -r $TXN_LOG_DIR $NODE_BACKUP_DIR/
+    fi
+
+    index=`expr $index + 1`
   done
 fi
diff --git a/asterix-events/src/main/resources/events/node_join/nc_join.sh b/asterix-events/src/main/resources/events/node_join/nc_join.sh
index 5aa86db..6b5280b 100755
--- a/asterix-events/src/main/resources/events/node_join/nc_join.sh
+++ b/asterix-events/src/main/resources/events/node_join/nc_join.sh
@@ -1,7 +1,8 @@
 CC_HOST=$1
 NC_ID=$2
+IO_DEVICES=$3
 if [ ! -d $LOG_DIR ]; 
 then 
   mkdir -p $LOG_DIR
 fi
-$ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cc-host $CC_HOST -cc-port 1099 -cluster-net-ip-address $IP_LOCATION  -data-ip-address $IP_LOCATION -result-ip-address $IP_LOCATION &> $LOG_DIR/${NC_ID}.log
+$ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cc-host $CC_HOST -cc-port 1099 -cluster-net-ip-address $IP_LOCATION  -data-ip-address $IP_LOCATION -iodevices $IO_DEVICES -result-ip-address $IP_LOCATION &> $LOG_DIR/${NC_ID}.log
diff --git a/asterix-events/src/main/resources/events/restore/restore.sh b/asterix-events/src/main/resources/events/restore/restore.sh
index 515efc1..6396eec 100755
--- a/asterix-events/src/main/resources/events/restore/restore.sh
+++ b/asterix-events/src/main/resources/events/restore/restore.sh
@@ -1,32 +1,75 @@
 WORKING_DIR=$1
 ASTERIX_INSTANCE_NAME=$2
-ASTERIX_DATA_DIR=$3
-BACKUP_ID=$4
-BACKUP_DIR=$5
-BACKUP_TYPE=$6
-NODE_ID=$7
-HDFS_URL=$8
-HADOOP_VERSION=$9
+ASTERIX_IODEVICES=$3
+NODE_STORE=$4
+ASTERIX_ROOT_METADATA_DIR=$5
+TXN_LOG_DIR_NAME=$6
+BACKUP_ID=$7
+BACKUP_DIR=$8
+BACKUP_TYPE=$9
+NODE_ID=${10}
+HDFS_URL=${11}
+HADOOP_VERSION=${12}
+HADOOP_HOME=$WORKING_DIR/hadoop-$HADOOP_VERSION
 
-export HADOOP_HOME=$WORKING_DIR/hadoop-$HADOOP_VERSION
-nodeStores=$(echo $ASTERIX_DATA_DIR | tr "," "\n")
+iodevices=$(echo $ASTERIX_IODEVICES | tr "," "\n")
 
-for nodeStore in $nodeStores
+index=1
+for iodevice in $iodevices
 do
-  MANGLED_BACKUP_DIR=`echo $nodeStore | tr / _`
-  NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$MANGLED_BACKUP_DIR
-  DEST_DIR=$nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME
-  if [ ! -d $DEST_DIR ]
-  then 
-    mkdir -p $DEST_DIR
-  else 
-    rm -rf $DEST_DIR/*
+  NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID
+
+  # remove the existing store directory
+  STORE_DIR=$iodevice/$NODE_STORE
+
+  # if STORE_DIR ends with a /, remove it
+  if [ ${STORE_DIR:${#STORE_DIR}-1} == "/" ];
+  then
+    STORE_DIR=${STORE_DIR:0:${#STORE_DIR}-1}
   fi
- 
+
+  DEST_STORE_DIR=${STORE_DIR%/*}
+  SOURCE_STORE_DIR=${NODE_STORE##*/}
+  rm -rf $DEST_STORE_DIR/$SOURCE_STORE_DIR
+
+  # remove the existing log directory
+  DEST_LOG_DIR=$iodevice/$TXN_LOG_DIR_NAME/
+  rm -rf $DEST_LOG_DIR
+
+  # remove the existing asterix metadata directory
+  rm -rf $iodevice/$ASTERIX_ROOT_METADATA_DIR
+
   if [ $BACKUP_TYPE == "hdfs" ];
   then
-    $HADOOP_HOME/bin/hadoop fs -copyToLocal $HDFS_URL/$NODE_BACKUP_DIR/*  $DEST_DIR/ 
+      # RESTORE FROM HDFS BACKUP
+
+      # copy store directory
+      $HADOOP_HOME/bin/hadoop fs -copyToLocal $HDFS_URL/$NODE_BACKUP_DIR/$SOURCE_STORE_DIR  $DEST_STORE_DIR/ 
+
+      # copy asterix metadata root directory and txn log directory
+      if [ $index -eq 1 ];
+      then
+        $HADOOP_HOME/bin/hadoop fs -copyToLocal $HDFS_URL/$NODE_BACKUP_DIR/$ASTERIX_ROOT_METADATA_DIR $iodevice/
+
+        # copy transaction logs directory
+        $HADOOP_HOME/bin/hadoop fs -copyToLocal $HDFS_URL/$NODE_BACKUP_DIR/$TXN_LOG_DIR_NAME $iodevice/
+      fi
+
   else
-    cp  -r $NODE_BACKUP_DIR/*  $DEST_DIR/ 
+
+      # RESTORE FROM LOCAL BACKUP
+      # copy store directory
+      cp  -r $NODE_BACKUP_DIR/$SOURCE_STORE_DIR  $DEST_STORE_DIR/ 
+
+      # copy asterix metadata root directory and txn log directory
+      if [ $index -eq 1 ];
+      then
+        cp -r $NODE_BACKUP_DIR/$ASTERIX_ROOT_METADATA_DIR $iodevice/
+
+        # copy transaction logs directory
+        cp -r $NODE_BACKUP_DIR/$TXN_LOG_DIR_NAME $iodevice/
+      fi
+
   fi
+  index=`expr $index + 1`
 done
diff --git a/asterix-events/src/main/resources/schema/cluster.xsd b/asterix-events/src/main/resources/schema/cluster.xsd
index 38afda3..0e1adce 100644
--- a/asterix-events/src/main/resources/schema/cluster.xsd
+++ b/asterix-events/src/main/resources/schema/cluster.xsd
@@ -3,19 +3,21 @@
 
 <!-- definition of simple types --> 
 <xs:element name="name" type="xs:string"/>
-<xs:element name="java_heap" type="xs:string"/>
+<xs:element name="java_opts" type="xs:string"/>
 <xs:element name="logdir" type="xs:string"/>
 <xs:element name="id" type="xs:string"/>
-<xs:element name="ip" type="xs:string"/>
+<xs:element name="client-ip" type="xs:string"/>
 <xs:element name="cluster-ip" type="xs:string"/>
 <xs:element name="key" type="xs:string"/>
 <xs:element name="value" type="xs:string"/>
 <xs:element name="dir" type="xs:string"/>
 <xs:element name="NFS" type="xs:boolean"/>
 <xs:element name="store" type="xs:string"/>
+<xs:element name="iodevices" type="xs:string"/>
 <xs:element name="java_home" type="xs:string"/>
 <xs:element name="username" type="xs:string"/>
 <xs:element name="debug" type="xs:integer"/>
+<xs:element name="debugEnabled" type="xs:boolean"/>
 
 <!-- definition of complex elements -->
 <xs:element name="workingDir">
@@ -31,10 +33,10 @@
   <xs:complexType>
     <xs:sequence>
       <xs:element ref="cl:id"/>
-      <xs:element ref="cl:ip"/>
+      <xs:element ref="cl:client-ip"/>
       <xs:element ref="cl:cluster-ip"/>
       <xs:element ref="cl:java_home" minOccurs="0"/>
-      <xs:element ref="cl:java_heap" minOccurs="0"/>
+      <xs:element ref="cl:java_opts" minOccurs="0"/>
       <xs:element ref="cl:logdir" minOccurs="0"/>
       <xs:element ref="cl:debug" minOccurs="0"/>
     </xs:sequence>
@@ -62,11 +64,12 @@
   <xs:complexType>
     <xs:sequence>
       <xs:element ref="cl:id"/>
-      <xs:element ref="cl:ip"/>
-      <xs:element ref="cl:java_heap" minOccurs="0"/>
+      <xs:element ref="cl:cluster-ip"/>
       <xs:element ref="cl:java_home" minOccurs="0"/>
+      <xs:element ref="cl:java_opts" minOccurs="0"/>
       <xs:element ref="cl:logdir" minOccurs="0"/>
       <xs:element ref="cl:store" minOccurs="0"/>
+      <xs:element ref="cl:iodevices" minOccurs="0"/>
       <xs:element ref="cl:debug" minOccurs="0"/>
     </xs:sequence>
   </xs:complexType>
@@ -78,11 +81,14 @@
       <xs:element ref="cl:name"/>
       <xs:element ref="cl:username"/>
       <xs:element ref="cl:env" minOccurs="0"/>
-      <xs:element ref="cl:java_heap" minOccurs="0"/>
       <xs:element ref="cl:java_home" minOccurs="0"/>
+      <xs:element ref="cl:java_opts" minOccurs="0"/>
       <xs:element ref="cl:logdir" minOccurs="0"/>
       <xs:element ref="cl:store" minOccurs="0"/>
+      <xs:element ref="cl:iodevices" minOccurs="0"/>
       <xs:element ref="cl:workingDir"/>
+      <xs:element ref="cl:debugEnabled" minOccurs="0"/>
+      <xs:element ref="cl:debug" minOccurs="0"/>
       <xs:element ref="cl:master-node"/>
       <xs:element ref="cl:node" maxOccurs="unbounded"/>
     </xs:sequence>
diff --git a/asterix-hyracks-glue/pom.xml b/asterix-hyracks-glue/pom.xml
deleted file mode 100644
index e28b2af..0000000
--- a/asterix-hyracks-glue/pom.xml
+++ /dev/null
@@ -1,53 +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.7</source>
-					<target>1.7</target>
-					<fork>true</fork>
-				</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-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java
index af7dbb7..66a0697 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java
@@ -28,14 +28,10 @@
     public void execute(String[] args) throws Exception {
         String[] cmdArgs = new String[args.length - 1];
         System.arraycopy(args, 1, cmdArgs, 0, cmdArgs.length);
-        if (cmdArgs.length >= 1 && cmdArgs[0].equals("-help")) {
-            System.out.println(getUsageDescription());
-        } else {
-            config = getCommandConfig();
-            CmdLineParser parser = new CmdLineParser(config);
-            parser.parseArgument(cmdArgs);
-            execCommand();
-        }
+        config = getCommandConfig();
+        CmdLineParser parser = new CmdLineParser(config);
+        parser.parseArgument(cmdArgs);
+        execCommand();
     }
 
     abstract protected void execCommand() throws Exception;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommandConfig.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommandConfig.java
deleted file mode 100644
index f28dfd9..0000000
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommandConfig.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package edu.uci.ics.asterix.installer.command;
-
-import org.kohsuke.args4j.Option;
-
-public class AbstractCommandConfig implements CommandConfig {
-
-    @Option(name = "-help", required = false, usage = "Help")
-    public boolean help = false;
-
-    @Override
-    public boolean helpMode() {
-        return help;
-    }
-
-}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
index e52308e..5ef7449 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
@@ -52,14 +52,13 @@
     protected String getUsageDescription() {
         return "\nAlter the instance's configuration settings."
                 + "\nPrior to running this command, the instance is required to be INACTIVE state."
-                + "\n\nAvailable arguments/options" 
-                + "\n-n name of the ASTERIX instance"
+                + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance"
                 + "\n-conf path to the ASTERIX configuration file.";
     }
 
 }
 
-class AlterConfig extends AbstractCommandConfig {
+class AlterConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
index cd4cc5c..297a168 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
@@ -31,6 +31,8 @@
 
 public class BackupCommand extends AbstractCommand {
 
+    public static final String ASTERIX_ROOT_METADATA_DIR = "asterix_root_metadata";
+
     @Override
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig();
@@ -59,15 +61,14 @@
                 + "\nfailure or accidental execution of a DDL statement (drop dataverse/dataset),"
                 + "\nyou may need to recover the lost data. The backup command allows you to take a"
                 + "\nbackup of the data stored with an ASTERIX instance. "
-                + "\nThe backed up snapshot is stored in HDFS." 
-                + "\n\nAvailable arguments/options:"
+                + "\nThe backed up snapshot is stored in HDFS." + "\n\nAvailable arguments/options:"
                 + "\n-n name of the Asterix instance";
 
     }
 
 }
 
-class BackupConfig extends AbstractCommandConfig {
+class BackupConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of the Asterix instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java
index 94aa935..55b0c3b 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java
@@ -1,20 +1,5 @@
-/*
- * 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.installer.command;
 
-public interface CommandConfig {
+public abstract class CommandConfig {
 
-    public boolean helpMode();
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
index 8a9347a..7fc3884 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
@@ -55,6 +55,9 @@
             case SHUTDOWN:
                 cmd = new ShutdownCommand();
                 break;
+            case HELP:
+                cmd = new HelpCommand();
+                break;
         }
         cmd.execute(args);
     }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java
index 6b0b53d..d22e6ce 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java
@@ -26,9 +26,11 @@
         String workingDir = InstallerDriver.getManagixHome() + File.separator + "clusters" + File.separator + "local"
                 + File.separator + "working_dir";
         cluster.setWorkingDir(new WorkingDir(workingDir, true));
-        cluster.setStore(workingDir + File.separator + "storage");
+        cluster.setIodevices(workingDir);
+        cluster.setStore("storage");
         cluster.setLogdir(workingDir + File.separator + "logs");
         cluster.setJavaHome(System.getenv("JAVA_HOME"));
+        cluster.setJavaOpts("-Xmx1024m");
 
         Marshaller marshaller = ctx.createMarshaller();
         marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
@@ -40,7 +42,11 @@
         Configuration configuration = (Configuration) unmarshaller.unmarshal(new File(installerConfPath));
 
         configuration.getBackup().setBackupDir(workingDir + File.separator + "backup");
-        configuration.getZookeeper().setHomeDir(workingDir + File.separator + "zookeeper");
+        configuration.getZookeeper().setHomeDir(
+                InstallerDriver.getManagixHome() + File.separator + InstallerDriver.MANAGIX_INTERNAL_DIR
+                        + File.separator + "zookeeper_home");
+        configuration.getZookeeper().getServers().setJavaHome(System.getenv("JAVA_HOME"));
+
         marshaller = ctx.createMarshaller();
         marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
         marshaller.marshal(configuration, new FileOutputStream(installerConfPath));
@@ -50,7 +56,7 @@
     @Override
     protected String getUsageDescription() {
         return "\nAuto-generates the ASTERIX installer configruation settings and ASTERIX cluster "
-                + "\n configuration settings for a single node setup.";
+                + "\nconfiguration settings for a single node setup.";
     }
 
     @Override
@@ -60,6 +66,6 @@
 
 }
 
-class ConfigureConfig extends AbstractCommandConfig {
+class ConfigureConfig extends CommandConfig {
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
index 5ddc9e9..fd0af04 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
@@ -57,11 +57,6 @@
 		Unmarshaller unmarshaller = ctx.createUnmarshaller();
 		cluster = (Cluster) unmarshaller.unmarshal(new File(
 				createConfig.clusterPath));
-		cluster.setWorkingDir(new WorkingDir(cluster.getWorkingDir().getDir()
-				+ File.separator + asterixInstanceName, cluster.getWorkingDir()
-				.isNFS()));
-		cluster.setLogdir(cluster.getLogdir() + File.separator
-				+ asterixInstanceName);
 		AsterixInstance asterixInstance = InstallerUtil.createAsterixInstance(
 				asterixInstanceName, cluster);
 		InstallerUtil.evaluateConflictWithOtherInstances(asterixInstance);
@@ -69,12 +64,15 @@
 		List<Property> clusterProperties = new ArrayList<Property>();
 		clusterProperties.add(new Property("ASTERIX_HOME", cluster
 				.getWorkingDir().getDir() + File.separator + "asterix"));
-		clusterProperties.add(new Property("JAVA_OPTS", "-Xmx"
-				+ cluster.getJavaHeap()));
+		StringBuilder javaOpts = new StringBuilder();
+		if (cluster.getJavaOpts() != null) {
+			javaOpts.append(cluster.getJavaOpts());
+		}
+		clusterProperties.add(new Property("JAVA_OPTS", javaOpts.toString()));
 		clusterProperties.add(new Property("CLUSTER_NET_IP", cluster
 				.getMasterNode().getClusterIp()));
 		clusterProperties.add(new Property("CLIENT_NET_IP", cluster
-				.getMasterNode().getIp()));
+				.getMasterNode().getClientIp()));
 		clusterProperties.add(new Property("LOG_DIR", cluster.getLogdir()));
 		clusterProperties.add(new Property("JAVA_HOME", cluster.getJavaHome()));
 		clusterProperties.add(new Property("WORKING_DIR", cluster
@@ -123,7 +121,7 @@
 
 }
 
-class CreateConfig extends AbstractCommandConfig {
+class CreateConfig extends CommandConfig {
 
 	@Option(name = "-n", required = true, usage = "Name of Asterix Instance")
 	public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
index a2788b4..2279d8f 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
@@ -26,41 +26,37 @@
 
 public class DeleteCommand extends AbstractCommand {
 
-	@Override
-	protected void execCommand() throws Exception {
-		InstallerDriver.initConfig();
-		String asterixInstanceName = ((DeleteConfig) config).name;
-		AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(
-				asterixInstanceName, State.INACTIVE);
-		PatternCreator pc = new PatternCreator();
-		Patterns patterns = pc.createDeleteInstancePattern(instance);
-		InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig();
+        String asterixInstanceName = ((DeleteConfig) config).name;
+        AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
+        PatternCreator pc = new PatternCreator();
+        Patterns patterns = pc.createDeleteInstancePattern(instance);
+        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
 
-		patterns = pc.createRemoveAsterixWorkingDirPattern(instance);
-		InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
-		ServiceProvider.INSTANCE.getLookupService().removeAsterixInstance(
-				asterixInstanceName);
-		LOGGER.info("Deleted Asterix instance: " + asterixInstanceName);
-	}
+        patterns = pc.createRemoveAsterixWorkingDirPattern(instance);
+        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+        ServiceProvider.INSTANCE.getLookupService().removeAsterixInstance(asterixInstanceName);
+        LOGGER.info("Deleted Asterix instance: " + asterixInstanceName);
+    }
 
-	@Override
-	protected CommandConfig getCommandConfig() {
-		return new DeleteConfig();
-	}
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new DeleteConfig();
+    }
 
-	@Override
-	protected String getUsageDescription() {
-		return "\nPermanently deletes an ASTERIX instance." + "\n"
-				+ "The instance must be in the INACTIVE state."
-				+ "\n\nAvailable arguments/options"
-				+ "\n-n name of the ASTERIX instance.";
-	}
+    @Override
+    protected String getUsageDescription() {
+        return "\nPermanently deletes an ASTERIX instance." + "\n" + "The instance must be in the INACTIVE state."
+                + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance.";
+    }
 
 }
 
-class DeleteConfig extends AbstractCommandConfig {
+class DeleteConfig extends CommandConfig {
 
-	@Option(name = "-n", required = true, usage = "Name of Asterix Instance")
-	public String name;
+    @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+    public String name;
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
index bec82d5..9306a56 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
@@ -29,77 +29,63 @@
 
 public class DescribeCommand extends AbstractCommand {
 
-	@Override
-	protected void execCommand() throws Exception {
-		InstallerDriver.initConfig();
-		String asterixInstanceName = ((DescribeConfig) config).name;
-		boolean adminView = ((DescribeConfig) config).admin;
-		if (asterixInstanceName != null) {
-			InstallerUtil.validateAsterixInstanceExists(asterixInstanceName,
-					State.INACTIVE, State.ACTIVE, State.UNUSABLE);
-			AsterixInstance instance = ServiceProvider.INSTANCE
-					.getLookupService().getAsterixInstance(asterixInstanceName);
-			if (instance != null) {
-				AsterixRuntimeState state = VerificationUtil
-						.getAsterixRuntimeState(instance);
-				boolean expectedRunning = instance.getState().equals(
-						State.UNUSABLE) ? instance.getPreviousState().equals(
-						State.ACTIVE) : !instance.getState().equals(
-						State.INACTIVE);
-				VerificationUtil.updateInstanceWithRuntimeDescription(instance,
-						state, expectedRunning);
-				ServiceProvider.INSTANCE.getLookupService()
-						.updateAsterixInstance(instance);
-				LOGGER.info(instance.getDescription(adminView));
-			} else {
-				throw new InstallerException("Asterix instance by the name "
-						+ asterixInstanceName + " does not exist.");
-			}
-		} else {
-			List<AsterixInstance> asterixInstances = ServiceProvider.INSTANCE
-					.getLookupService().getAsterixInstances();
-			if (asterixInstances.size() > 0) {
-				for (AsterixInstance instance : asterixInstances) {
-					AsterixRuntimeState state = VerificationUtil
-							.getAsterixRuntimeState(instance);
-					boolean expectedRunning = instance.getState().equals(
-							State.UNUSABLE) ? instance.getPreviousState()
-							.equals(State.ACTIVE) : !instance.getState()
-							.equals(State.INACTIVE);
-					VerificationUtil.updateInstanceWithRuntimeDescription(
-							instance, state, expectedRunning);
-					ServiceProvider.INSTANCE.getLookupService()
-							.updateAsterixInstance(instance);
-					LOGGER.info(instance.getDescription(adminView));
-				}
-			} else {
-				LOGGER.info("No Asterix instances found!");
-			}
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig();
+        String asterixInstanceName = ((DescribeConfig) config).name;
+        boolean adminView = ((DescribeConfig) config).admin;
+        if (asterixInstanceName != null) {
+            InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE, State.ACTIVE,
+                    State.UNUSABLE);
+            AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(
+                    asterixInstanceName);
+            if (instance != null) {
+                AsterixRuntimeState state = VerificationUtil.getAsterixRuntimeState(instance);
+                boolean expectedRunning = instance.getState().equals(State.UNUSABLE) ? instance.getPreviousState()
+                        .equals(State.ACTIVE) : !instance.getState().equals(State.INACTIVE);
+                VerificationUtil.updateInstanceWithRuntimeDescription(instance, state, expectedRunning);
+                ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(instance);
+                LOGGER.info(instance.getDescription(adminView));
+            } else {
+                throw new InstallerException("Asterix instance by the name " + asterixInstanceName + " does not exist.");
+            }
+        } else {
+            List<AsterixInstance> asterixInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
+            if (asterixInstances.size() > 0) {
+                for (AsterixInstance instance : asterixInstances) {
+                    AsterixRuntimeState state = VerificationUtil.getAsterixRuntimeState(instance);
+                    boolean expectedRunning = instance.getState().equals(State.UNUSABLE) ? instance.getPreviousState()
+                            .equals(State.ACTIVE) : !instance.getState().equals(State.INACTIVE);
+                    VerificationUtil.updateInstanceWithRuntimeDescription(instance, state, expectedRunning);
+                    ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(instance);
+                    LOGGER.info(instance.getDescription(adminView));
+                }
+            } else {
+                LOGGER.info("No Asterix instances found!");
+            }
 
-		}
-	}
+        }
+    }
 
-	@Override
-	protected CommandConfig getCommandConfig() {
-		return new DescribeConfig();
-	}
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new DescribeConfig();
+    }
 
-	@Override
-	protected String getUsageDescription() {
-		return "\nProvides information about an ASTERIX instance."
-				+ "\n\nUsage arguments/options:-"
-				+ "\n[-n]       Name of the ASTERIX instance."
-				+ "\n[-admin]   Provides a detailed description";
-	}
+    @Override
+    protected String getUsageDescription() {
+        return "\nProvides information about an ASTERIX instance." + "\n\nUsage arguments/options:-"
+                + "\n[-n]       Name of the ASTERIX instance." + "\n[-admin]   Provides a detailed description";
+    }
 
 }
 
-class DescribeConfig extends AbstractCommandConfig {
+class DescribeConfig extends CommandConfig {
 
-	@Option(name = "-n", required = false, usage = "Name of Asterix Instance")
-	public String name;
+    @Option(name = "-n", required = false, usage = "Name of Asterix Instance")
+    public String name;
 
-	@Option(name = "-admin", required = false, usage = "Detailed description")
-	public boolean admin;
+    @Option(name = "-admin", required = false, usage = "Detailed description")
+    public boolean admin;
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
new file mode 100644
index 0000000..a9864a3
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
@@ -0,0 +1,85 @@
+/*
+ * 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.installer.command;
+
+import org.kohsuke.args4j.Option;
+
+public class HelpCommand extends AbstractCommand {
+
+    @Override
+    protected void execCommand() throws Exception {
+        HelpConfig helpConfig = (HelpConfig) config;
+        String command = helpConfig.command;
+        CommandType cmdType = CommandType.valueOf(command.toUpperCase());
+        String helpMessage = null;
+        switch (cmdType) {
+            case CREATE:
+                helpMessage = new CreateCommand().getUsageDescription();
+                break;
+            case CONFIGURE:
+                helpMessage = new ConfigureCommand().getUsageDescription();
+                break;
+            case DELETE:
+                helpMessage = new DeleteCommand().getUsageDescription();
+                break;
+            case DESCRIBE:
+                helpMessage = new DescribeCommand().getUsageDescription();
+                break;
+            case RESTORE:
+                helpMessage = new RestoreCommand().getUsageDescription();
+                break;
+            case START:
+                helpMessage = new StartCommand().getUsageDescription();
+                break;
+            case SHUTDOWN:
+                helpMessage = new ShutdownCommand().getUsageDescription();
+                break;
+            case BACKUP:
+                helpMessage = new BackupCommand().getUsageDescription();
+                break;
+            case STOP:
+                helpMessage = new StopCommand().getUsageDescription();
+                break;
+            case VALIDATE:
+                helpMessage = new ValidateCommand().getUsageDescription();
+                break;
+            default:
+                helpMessage = "Unknown command " + command;
+        }
+
+        System.out.println(helpMessage);
+    }
+
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new HelpConfig();
+    }
+
+    @Override
+    protected String getUsageDescription() {
+        return "\nAlter the instance's configuration settings."
+                + "\nPrior to running this command, the instance is required to be INACTIVE state."
+                + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance"
+                + "\n-conf path to the ASTERIX configuration file.";
+    }
+
+}
+
+class HelpConfig extends CommandConfig {
+
+    @Option(name = "-cmd", required = true, usage = "Name of Asterix Instance")
+    public String command;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
index d27fee9..979c414 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
@@ -27,7 +27,8 @@
         ALTER,
         VALIDATE,
         CONFIGURE,
-        SHUTDOWN
+        SHUTDOWN,
+        HELP
     }
 
     public void execute(String args[]) throws Exception;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
index 49629a4..9ef925a 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
@@ -60,7 +60,7 @@
 
 }
 
-class RestoreConfig extends AbstractCommandConfig {
+class RestoreConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of the Asterix instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
index cbfd5c5..b9dd23d 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
@@ -39,6 +39,6 @@
 
 }
 
-class ShutdownConfig extends AbstractCommandConfig {
+class ShutdownConfig extends CommandConfig {
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
index 8004e9a..38bcf33 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
@@ -61,11 +61,9 @@
     }
 }
 
-class StartConfig extends AbstractCommandConfig {
+class StartConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
     public String name;
 
-  
-
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
index 85fcb68..0e10414 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
@@ -80,7 +80,7 @@
 
 }
 
-class StopConfig extends AbstractCommandConfig {
+class StopConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
index 42c0d38..3d292a9 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
@@ -29,18 +29,15 @@
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.cluster.MasterNode;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
-import edu.uci.ics.asterix.event.schema.pattern.Patterns;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
 import edu.uci.ics.asterix.installer.schema.conf.Configuration;
 import edu.uci.ics.asterix.installer.schema.conf.Zookeeper;
 
 public class ValidateCommand extends AbstractCommand {
 
-	private static final String OK = " [" + "\u2713" + "]";
-	private static final String ERROR = " [" + "x" + "]";
-	private static final String WARNING = " [" + "!" + "]";
+	private static final String OK = " [" + "OK" + "]";
+	private static final String ERROR = " [" + "ERROR" + "]";
+	private static final String WARNING = " [" + "WARNING" + "]";
 
 	@Override
 	protected void execCommand() throws Exception {
@@ -111,18 +108,19 @@
 
 			Set<String> servers = new HashSet<String>();
 			Set<String> serverIds = new HashSet<String>();
-			servers.add(cluster.getMasterNode().getIp());
+			servers.add(cluster.getMasterNode().getClusterIp());
 			serverIds.add(cluster.getMasterNode().getId());
 
 			MasterNode masterNode = cluster.getMasterNode();
-			Node master = new Node(masterNode.getId(), masterNode.getIp(),
-					masterNode.getJavaHeap(), masterNode.getJavaHome(),
-					masterNode.getLogdir(), null, masterNode.getDebug());
+			Node master = new Node(masterNode.getId(),
+					masterNode.getClusterIp(), masterNode.getJavaOpts(),
+					masterNode.getJavaHome(), masterNode.getLogdir(), null,
+					null, null);
 
 			valid = valid & validateNodeConfiguration(master, cluster);
 
 			for (Node node : cluster.getNode()) {
-				servers.add(node.getIp());
+				servers.add(node.getClusterIp());
 				if (serverIds.contains(node.getId())) {
 					valid = false;
 					LOGGER.error("Duplicate node id :" + node.getId() + ERROR);
@@ -141,9 +139,9 @@
 				&& checkTemporaryPath(cluster.getLogdir())) {
 			tempDirs.add("Log directory: " + cluster.getLogdir());
 		}
-		if (cluster.getStore() != null
-				&& checkTemporaryPath(cluster.getStore())) {
-			tempDirs.add("Store directory: " + cluster.getStore());
+		if (cluster.getIodevices() != null
+				&& checkTemporaryPath(cluster.getIodevices())) {
+			tempDirs.add("IO Device: " + cluster.getIodevices());
 		}
 
 		if (tempDirs.size() > 0) {
@@ -159,7 +157,7 @@
 
 	private boolean validateNodeConfiguration(Node node, Cluster cluster) {
 		boolean valid = true;
-		valid = checkNodeReachability(node.getIp());
+		valid = checkNodeReachability(node.getClusterIp());
 		if (node.getJavaHome() == null || node.getJavaHome().length() == 0) {
 			if (cluster.getJavaHome() == null
 					|| cluster.getJavaHome().length() == 0) {
@@ -178,8 +176,8 @@
 			}
 		}
 
-		if (node.getStore() == null || cluster.getStore().length() == 0) {
-			if (cluster.getMasterNode().getId().equals(node.getId())
+		if (node.getStore() == null || node.getStore().length() == 0) {
+			if (!cluster.getMasterNode().getId().equals(node.getId())
 					&& (cluster.getStore() == null || cluster.getStore()
 							.length() == 0)) {
 				valid = false;
@@ -188,14 +186,16 @@
 			}
 		}
 
-		if (node.getJavaHeap() == null || node.getJavaHeap().length() == 0) {
-			if (cluster.getJavaHeap() == null
-					|| cluster.getJavaHeap().length() == 0) {
+		if (node.getIodevices() == null || node.getIodevices().length() == 0) {
+			if (!cluster.getMasterNode().getId().equals(node.getId())
+					&& (cluster.getIodevices() == null || cluster
+							.getIodevices().length() == 0)) {
 				valid = false;
-				LOGGER.fatal("java heap size not defined at cluster/node level for node: "
+				LOGGER.fatal("iodevice(s) not defined at cluster/node level for node: "
 						+ node.getId() + ERROR);
 			}
 		}
+
 		return valid;
 	}
 
@@ -259,7 +259,7 @@
 
 }
 
-class ValidateConfig extends AbstractCommandConfig {
+class ValidateConfig extends CommandConfig {
 
 	@Option(name = "-c", required = false, usage = "Path to the cluster configuration xml")
 	public String cluster;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
index 321fd1a..8083427 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
@@ -30,107 +30,133 @@
 
 public class InstallerDriver {
 
-    public static final String MANAGIX_INTERNAL_DIR = ".installer";
-    public static final String MANAGIX_EVENT_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix";
-    public static final String MANAGIX_EVENT_SCRIPTS_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix"
-            + File.separator + "scripts";
-    public static final String ASTERIX_DIR = "asterix";
-    public static final String EVENTS_DIR = "events";
+	public static final String MANAGIX_INTERNAL_DIR = ".installer";
+	public static final String MANAGIX_EVENT_DIR = MANAGIX_INTERNAL_DIR
+			+ File.separator + "eventrix";
+	public static final String MANAGIX_EVENT_SCRIPTS_DIR = MANAGIX_INTERNAL_DIR
+			+ File.separator + "eventrix" + File.separator + "scripts";
+	public static final String ASTERIX_DIR = "asterix";
+	public static final String EVENTS_DIR = "events";
 
-    private static final Logger LOGGER = Logger.getLogger(InstallerDriver.class.getName());
-    public static final String ENV_MANAGIX_HOME = "MANAGIX_HOME";
-    public static final String MANAGIX_CONF_XML = "conf" + File.separator + "managix-conf.xml";
+	private static final Logger LOGGER = Logger.getLogger(InstallerDriver.class
+			.getName());
+	public static final String ENV_MANAGIX_HOME = "MANAGIX_HOME";
+	public static final String MANAGIX_CONF_XML = "conf" + File.separator
+			+ "managix-conf.xml";
 
-    private static Configuration conf;
-    private static String managixHome;
-    private static String asterixZip;
+	private static Configuration conf;
+	private static String managixHome;
+	private static String asterixZip;
 
-    public static String getAsterixZip() {
-        return asterixZip;
-    }
+	public static String getAsterixZip() {
+		return asterixZip;
+	}
 
-    public static Configuration getConfiguration() {
-        return conf;
-    }
+	public static Configuration getConfiguration() {
+		return conf;
+	}
 
-    public static void initConfig() throws Exception {
-        File configFile = new File(managixHome + File.separator + MANAGIX_CONF_XML);
-        JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
-        Unmarshaller unmarshaller = configCtx.createUnmarshaller();
-        conf = (Configuration) unmarshaller.unmarshal(configFile);
-        asterixZip = initBinary("asterix-server");
+	public static void initConfig() throws Exception {
+		File configFile = new File(managixHome + File.separator
+				+ MANAGIX_CONF_XML);
+		JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
+		Unmarshaller unmarshaller = configCtx.createUnmarshaller();
+		conf = (Configuration) unmarshaller.unmarshal(configFile);
+		asterixZip = initBinary("asterix-server");
 
-        ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
-        if (!lookupService.isRunning(conf)) {
-            lookupService.startService(conf);
-        }
-    }
+		ILookupService lookupService = ServiceProvider.INSTANCE
+				.getLookupService();
+		if (!lookupService.isRunning(conf)) {
+			lookupService.startService(conf);
+		}
+	}
 
-    private static String initBinary(final String fileNamePattern) {
-        String asterixDir = InstallerDriver.getAsterixDir();
-        File file = new File(asterixDir);
-        File[] zipFiles = file.listFiles(new FileFilter() {
-            public boolean accept(File arg0) {
-                return arg0.getAbsolutePath().contains(fileNamePattern) && arg0.isFile();
-            }
-        });
-        if (zipFiles.length == 0) {
-            String msg = " Binary not found at " + asterixDir;
-            LOGGER.log(Level.FATAL, msg);
-            throw new IllegalStateException(msg);
-        }
-        if (zipFiles.length > 1) {
-            String msg = " Multiple binaries found at " + asterixDir;
-            LOGGER.log(Level.FATAL, msg);
-            throw new IllegalStateException(msg);
-        }
+	private static String initBinary(final String fileNamePattern) {
+		String asterixDir = InstallerDriver.getAsterixDir();
+		File file = new File(asterixDir);
+		File[] zipFiles = file.listFiles(new FileFilter() {
+			public boolean accept(File arg0) {
+				return arg0.getAbsolutePath().contains(fileNamePattern)
+						&& arg0.isFile();
+			}
+		});
+		if (zipFiles.length == 0) {
+			String msg = " Binary not found at " + asterixDir;
+			LOGGER.log(Level.FATAL, msg);
+			throw new IllegalStateException(msg);
+		}
+		if (zipFiles.length > 1) {
+			String msg = " Multiple binaries found at " + asterixDir;
+			LOGGER.log(Level.FATAL, msg);
+			throw new IllegalStateException(msg);
+		}
 
-        return zipFiles[0].getAbsolutePath();
-    }
+		return zipFiles[0].getAbsolutePath();
+	}
 
-    public static String getManagixHome() {
-        return managixHome;
-    }
+	public static String getManagixHome() {
+		return managixHome;
+	}
 
-    public static String getAsterixDir() {
-        return managixHome + File.separator + ASTERIX_DIR;
-    }
+	public static String getAsterixDir() {
+		return managixHome + File.separator + ASTERIX_DIR;
+	}
 
-    public static void main(String args[]) {
-        try {
-            if (args.length != 0) {
-                managixHome = System.getenv(ENV_MANAGIX_HOME);
-                CommandHandler cmdHandler = new CommandHandler();
-                cmdHandler.processCommand(args);
-            } else {
-                printUsage();
-            }
-        } catch (IllegalArgumentException iae) {
-            LOGGER.error("Unknown command");
-            printUsage();
-        } catch (Exception e) {
-            LOGGER.error(e.getMessage());
-        }
-    }
+	public static void main(String args[]) {
+		try {
+			if (args.length != 0) {
+				managixHome = System.getenv(ENV_MANAGIX_HOME);
+				CommandHandler cmdHandler = new CommandHandler();
+				cmdHandler.processCommand(args);
+			} else {
+				printUsage();
+			}
+		} catch (IllegalArgumentException iae) {
+			LOGGER.error("Unknown command");
+			printUsage();
+		} catch (Exception e) {
+			LOGGER.error(e.getMessage());
+			if (e.getMessage() == null || e.getMessage().length() == 0) {
+				e.printStackTrace();
+			}
+		}
+	}
 
-    private static void printUsage() {
-        StringBuffer buffer = new StringBuffer("managix <command> <options>" + "\n");
-        buffer.append("Commands" + "\n");
-        buffer.append("create   " + ":" + " Creates a new asterix instance" + "\n");
-        buffer.append("delete   " + ":" + " Deletes an asterix instance" + "\n");
-        buffer.append("start    " + ":" + " Starts an  asterix instance" + "\n");
-        buffer.append("stop     " + ":" + " Stops an asterix instance that is in ACTIVE state" + "\n");
-        buffer.append("backup   " + ":" + " Creates a back up for an existing asterix instance" + "\n");
-        buffer.append("restore  " + ":" + " Restores an asterix instance" + "\n");
-        buffer.append("describe " + ":" + " Describes an existing asterix instance" + "\n");
-        buffer.append("validate " + ":" + " Validates the installer/cluster configuration" + "\n");
-        buffer.append("configure" + ":" + " Auto-generate configuration for local psedu-distributed Asterix instance"
-                + "\n");
-        buffer.append("shutdown " + ":" + " Shutdown the installer service" + "\n");
-        buffer.append("validate " + ":" + " Validates the installer/cluster configuration" + "\n");
-        buffer.append("configure" + ":" + " Auto-generate configuration for local psedu-distributed Asterix instance"
-                + "\n");
-        buffer.append("shutdown " + ":" + " Shutdown the installer service" + "\n");
-        LOGGER.info(buffer.toString());
-    }
+	private static void printUsage() {
+		StringBuffer buffer = new StringBuffer("managix <command> <options>"
+				+ "\n");
+		buffer.append("Commands" + "\n");
+		buffer.append("create   " + ":" + " Creates a new asterix instance"
+				+ "\n");
+		buffer.append("delete   " + ":" + " Deletes an asterix instance" + "\n");
+		buffer.append("start    " + ":" + " Starts an  asterix instance" + "\n");
+		buffer.append("stop     " + ":"
+				+ " Stops an asterix instance that is in ACTIVE state" + "\n");
+		buffer.append("backup   " + ":"
+				+ " Creates a back up for an existing asterix instance" + "\n");
+		buffer.append("restore  " + ":" + " Restores an asterix instance"
+				+ "\n");
+		buffer.append("describe " + ":"
+				+ " Describes an existing asterix instance" + "\n");
+		buffer.append("validate " + ":"
+				+ " Validates the installer/cluster configuration" + "\n");
+		buffer.append("configure"
+				+ ":"
+				+ " Auto-generate configuration for local psedu-distributed Asterix instance"
+				+ "\n");
+		buffer.append("shutdown " + ":" + " Shutdown the installer service"
+				+ "\n");
+		buffer.append("validate " + ":"
+				+ " Validates the installer/cluster configuration" + "\n");
+		buffer.append("configure"
+				+ ":"
+				+ " Auto-generate configuration for local psedu-distributed Asterix instance"
+				+ "\n");
+		buffer.append("shutdown " + ":" + " Shutdown the installer service"
+				+ "\n");
+		buffer.append("help     " + ":"
+				+ " Provides usage description of a command" + "\n");
+
+		LOGGER.info(buffer.toString());
+	}
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
index 24b531a..e2be142 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
@@ -25,9 +25,12 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.StringWriter;
+import java.net.URL;
+import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 import java.util.jar.JarEntry;
@@ -51,6 +54,9 @@
 
 public class InstallerUtil {
 
+    public static final String TXN_LOG_DIR = "txnLogs";
+    public static final String TXN_LOG_DIR_KEY_SUFFIX = "txnLogDir";
+
     public static AsterixInstance createAsterixInstance(String asterixInstanceName, Cluster cluster)
             throws FileNotFoundException, IOException {
         Properties asterixConfProp = new Properties();
@@ -65,20 +71,57 @@
         return instance;
     }
 
-    public static void createAsterixZip(AsterixInstance asterixInstance, boolean newDeployment) throws IOException {
+    public static void createAsterixZip(AsterixInstance asterixInstance, boolean newDeployment) throws IOException,
+            InterruptedException {
+
+        String modifiedZipPath = injectAsterixPropertyFile(InstallerDriver.getAsterixZip(), asterixInstance,
+                newDeployment);
+        injectAsterixLogPropertyFile(modifiedZipPath, asterixInstance);
+    }
+
+    private static String injectAsterixPropertyFile(String origZipFile, AsterixInstance asterixInstance,
+            boolean newDeployment) throws IOException {
         writeAsterixConfigurationFile(asterixInstance, newDeployment);
         String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
-        unzip(InstallerDriver.getAsterixZip(), asterixInstanceDir);
+        unzip(origZipFile, asterixInstanceDir);
         File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
                 + asterixInstance.getAsterixVersion() + ".jar");
-        String origFile = "test.properties";
+        String asterixPropertyFile = "test.properties";
         File replacementFile = new File(asterixInstanceDir + File.separator + "test.properties");
-        replaceInJar(sourceJar, origFile, replacementFile);
+        replaceInJar(sourceJar, asterixPropertyFile, replacementFile);
         new File(asterixInstanceDir + File.separator + "test.properties").delete();
         String asterixZipName = InstallerDriver.getAsterixZip().substring(
                 InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
         zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+        return asterixInstanceDir + File.separator + asterixZipName;
+    }
 
+    private static String injectAsterixLogPropertyFile(String origZipFile, AsterixInstance asterixInstance)
+            throws IOException {
+        String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
+        unzip(origZipFile, asterixInstanceDir);
+        File sourceJar1 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        String txnLogPropertyFile = "log.properties";
+        Properties txnLogProperties = new Properties();
+        URLClassLoader urlClassLoader = new URLClassLoader(new URL[] { sourceJar1.toURI().toURL() });
+        InputStream in = urlClassLoader.getResourceAsStream(txnLogPropertyFile);
+        if (in != null) {
+            txnLogProperties.load(in);
+        }
+
+        writeAsterixLogConfigurationFile(asterixInstance.getName(), asterixInstance.getCluster(), txnLogProperties);
+
+        File sourceJar2 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        File replacementFile = new File(asterixInstanceDir + File.separator + "log.properties");
+        replaceInJar(sourceJar2, txnLogPropertyFile, replacementFile);
+
+        new File(asterixInstanceDir + File.separator + "log.properties").delete();
+        String asterixZipName = InstallerDriver.getAsterixZip().substring(
+                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
+        zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+        return asterixInstanceDir + File.separator + asterixZipName;
     }
 
     public static void addLibraryToAsterixZip(AsterixInstance asterixInstance, String dataverseName,
@@ -111,7 +154,6 @@
 
     public static String getNodeDirectories(String asterixInstanceName, Node node, Cluster cluster) {
         String storeDataSubDir = asterixInstanceName + File.separator + "data" + File.separator;
-        String storeLibrarySubDir = asterixInstanceName + File.separator + "library" + File.separator;
         String[] storeDirs = null;
         StringBuffer nodeDataStore = new StringBuffer();
         String storeDirValue = node.getStore();
@@ -121,7 +163,6 @@
                 throw new IllegalStateException(" Store not defined for node " + node.getId());
             }
             storeDataSubDir = node.getId() + File.separator + storeDataSubDir;
-            storeLibrarySubDir = node.getId() + File.separator + storeLibrarySubDir;
         }
 
         storeDirs = storeDirValue.split(",");
@@ -143,19 +184,40 @@
         conf.append("MetadataNode=" + asterixInstanceName + "_" + metadataNodeId + "\n");
         conf.append("NewUniverse=" + newData + "\n");
 
+        String storeDir = null;
         for (Node node : cluster.getNode()) {
-            String nodeDir = getNodeDirectories(asterixInstance.getName(), node, cluster);
-            conf.append(asterixInstanceName + "_" + node.getId() + ".stores" + "=" + nodeDir + "\n");
+            storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
+            conf.append(asterixInstanceName + "_" + node.getId() + ".stores" + "=" + storeDir + "\n");
         }
+
         Properties asterixConfProp = asterixInstance.getConfiguration();
         String outputDir = asterixConfProp.getProperty("output_dir");
         conf.append("OutputDir=" + outputDir);
+
         File asterixConfDir = new File(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName);
         asterixConfDir.mkdirs();
         dumpToFile(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName + File.separator
                 + "test.properties", conf.toString());
     }
 
+    private static void writeAsterixLogConfigurationFile(String asterixInstanceName, Cluster cluster,
+            Properties logProperties) throws IOException {
+        StringBuffer conf = new StringBuffer();
+        for (Map.Entry<Object, Object> p : logProperties.entrySet()) {
+            conf.append(p.getKey() + "=" + p.getValue() + "\n");
+        }
+
+        for (Node node : cluster.getNode()) {
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            String txnLogDir = iodevices.split(",")[0].trim() + File.separator + InstallerUtil.TXN_LOG_DIR;
+            conf.append(asterixInstanceName + "_" + node.getId() + "." + TXN_LOG_DIR_KEY_SUFFIX + "=" + txnLogDir
+                    + "\n");
+        }
+        dumpToFile(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName + File.separator
+                + "log.properties", conf.toString());
+
+    }
+
     public static Properties getAsterixConfiguration(String asterixConf) throws FileNotFoundException, IOException {
         Properties prop = new Properties();
         prop.load(new FileInputStream(asterixConf));
@@ -178,13 +240,13 @@
             int count;
             byte data[] = new byte[BUFFER_SIZE];
 
-            //write the file to the disk
+            // write the file to the disk
             FileOutputStream fos = new FileOutputStream(dst);
             dest = new BufferedOutputStream(fos, BUFFER_SIZE);
             while ((count = zis.read(data, 0, BUFFER_SIZE)) != -1) {
                 dest.write(data, 0, count);
             }
-            //close the output streams
+            // close the output streams
             dest.flush();
             dest.close();
         }
@@ -301,21 +363,21 @@
     public static void evaluateConflictWithOtherInstances(AsterixInstance instance) throws Exception {
         List<AsterixInstance> existingInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
         List<String> usedIps = new ArrayList<String>();
-        String masterIp = instance.getCluster().getMasterNode().getIp();
+        String masterIp = instance.getCluster().getMasterNode().getClusterIp();
         for (Node node : instance.getCluster().getNode()) {
-            usedIps.add(node.getIp());
+            usedIps.add(node.getClusterIp());
         }
-        usedIps.add(instance.getCluster().getMasterNode().getIp());
+        usedIps.add(instance.getCluster().getMasterNode().getClusterIp());
         boolean conflictFound = false;
         AsterixInstance conflictingInstance = null;
         for (AsterixInstance existing : existingInstances) {
-            conflictFound = existing.getCluster().getMasterNode().getIp().equals(masterIp);
+            conflictFound = existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
             if (conflictFound) {
                 conflictingInstance = existing;
                 break;
             }
             for (Node n : existing.getCluster().getNode()) {
-                if (usedIps.contains(n.getIp())) {
+                if (usedIps.contains(n.getClusterIp())) {
                     conflictFound = true;
                     conflictingInstance = existing;
                     break;
@@ -341,7 +403,7 @@
         }
         ProcessBuilder pb = new ProcessBuilder(pargs);
         pb.environment().putAll(EventDriver.getEnvironment());
-        pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getIp());
+        pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getClusterIp());
         Process p = pb.start();
         BufferedInputStream bis = new BufferedInputStream(p.getInputStream());
         StringWriter writer = new StringWriter();
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
index 7c7a792..9cb1b96e 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
@@ -29,93 +29,110 @@
 
 public class VerificationUtil {
 
-    private static final String VERIFY_SCRIPT_PATH = InstallerDriver.getManagixHome() + File.separator
-            + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "scripts" + File.separator + "verify.sh";
+	private static final String VERIFY_SCRIPT_PATH = InstallerDriver
+			.getManagixHome()
+			+ File.separator
+			+ InstallerDriver.MANAGIX_INTERNAL_DIR
+			+ File.separator
+			+ "scripts"
+			+ File.separator + "verify.sh";
 
-    public static AsterixRuntimeState getAsterixRuntimeState(AsterixInstance instance) throws Exception {
+	public static AsterixRuntimeState getAsterixRuntimeState(
+			AsterixInstance instance) throws Exception {
 
-        Cluster cluster = instance.getCluster();
-        List<String> args = new ArrayList<String>();
-        args.add(instance.getName());
-        args.add(instance.getCluster().getMasterNode().getIp());
-        for (Node node : cluster.getNode()) {
-            args.add(node.getIp());
-            args.add(instance.getName() + "_" + node.getId());
-        }
+		Cluster cluster = instance.getCluster();
+		List<String> args = new ArrayList<String>();
+		args.add(instance.getName());
+		args.add(instance.getCluster().getMasterNode().getClusterIp());
+		for (Node node : cluster.getNode()) {
+			args.add(node.getClusterIp());
+			args.add(instance.getName() + "_" + node.getId());
+		}
+		Thread.sleep(2000);
+		String output = InstallerUtil.executeLocalScript(VERIFY_SCRIPT_PATH,
+				args);
+		boolean ccRunning = true;
+		List<String> failedNCs = new ArrayList<String>();
+		String[] infoFields;
+		ProcessInfo pInfo;
+		List<ProcessInfo> processes = new ArrayList<ProcessInfo>();
 
-        String output = InstallerUtil.executeLocalScript(VERIFY_SCRIPT_PATH, args);
-        boolean ccRunning = true;
-        List<String> failedNCs = new ArrayList<String>();
-        String[] infoFields;
-        ProcessInfo pInfo;
-        List<ProcessInfo> processes = new ArrayList<ProcessInfo>();
+		for (String line : output.split("\n")) {
+			String nodeid = null;
+			infoFields = line.split(":");
+			try {
+				int pid = Integer.parseInt(infoFields[3]);
+				if (infoFields[0].equals("NC")) {
+					nodeid = infoFields[2].split("_")[1];
+				} else {
+					nodeid = instance.getCluster().getMasterNode().getId();
+				}
+				pInfo = new ProcessInfo(infoFields[0], infoFields[1], nodeid,
+						pid);
+				processes.add(pInfo);
+			} catch (Exception e) {
+				if (infoFields[0].equalsIgnoreCase("CC")) {
+					ccRunning = false;
+				} else {
+					failedNCs.add(infoFields[1]);
+				}
+			}
+		}
+		return new AsterixRuntimeState(processes, failedNCs, ccRunning);
+	}
 
-        for (String line : output.split("\n")) {
-            infoFields = line.split(":");
-            try {
-                int pid = Integer.parseInt(infoFields[3]);
-                pInfo = new ProcessInfo(infoFields[0], infoFields[1], pid);
-                processes.add(pInfo);
-            } catch (Exception e) {
-                if (infoFields[0].equalsIgnoreCase("CC")) {
-                    ccRunning = false;
-                } else {
-                    failedNCs.add(infoFields[1]);
-                }
-            }
-        }
-        return new AsterixRuntimeState(processes, failedNCs, ccRunning);
-    }
+	public static void updateInstanceWithRuntimeDescription(
+			AsterixInstance instance, AsterixRuntimeState state,
+			boolean expectedRunning) {
+		StringBuffer summary = new StringBuffer();
+		if (expectedRunning) {
+			if (!state.isCcRunning()) {
+				summary.append("Cluster Controller not running at "
+						+ instance.getCluster().getMasterNode().getId() + "\n");
+				instance.setState(State.UNUSABLE);
+			}
+			if (state.getFailedNCs() != null && !state.getFailedNCs().isEmpty()) {
+				summary.append("Node Controller not running at the following nodes"
+						+ "\n");
+				for (String failedNC : state.getFailedNCs()) {
+					summary.append(failedNC + "\n");
+				}
+				instance.setState(State.UNUSABLE);
+			}
+			if (!(instance.getState().equals(State.UNUSABLE))) {
+				instance.setState(State.ACTIVE);
+			}
+		} else {
+			if (state.getProcesses() != null && state.getProcesses().size() > 0) {
+				summary.append("Following process still running " + "\n");
+				for (ProcessInfo pInfo : state.getProcesses()) {
+					summary.append(pInfo + "\n");
+				}
+				instance.setState(State.UNUSABLE);
+			} else {
+				instance.setState(State.INACTIVE);
+			}
+		}
+		state.setSummary(summary.toString());
+		instance.setAsterixRuntimeStates(state);
+	}
 
-    public static void updateInstanceWithRuntimeDescription(AsterixInstance instance, AsterixRuntimeState state,
-            boolean expectedRunning) {
-        StringBuffer summary = new StringBuffer();
-        if (expectedRunning) {
-            if (!state.isCcRunning()) {
-                summary.append("Cluster Controller not running at " + instance.getCluster().getMasterNode().getIp()
-                        + "\n");
-                instance.setState(State.UNUSABLE);
-            }
-            if (state.getFailedNCs() != null && !state.getFailedNCs().isEmpty()) {
-                summary.append("Node Controller not running at the following nodes" + "\n");
-                for (String failedNC : state.getFailedNCs()) {
-                    summary.append(failedNC + "\n");
-                }
-                instance.setState(State.UNUSABLE);
-            }
-            if (!(instance.getState().equals(State.UNUSABLE))) {
-                instance.setState(State.ACTIVE);
-            }
-        } else {
-            if (state.getProcesses() != null && state.getProcesses().size() > 0) {
-                summary.append("Following process still running " + "\n");
-                for (ProcessInfo pInfo : state.getProcesses()) {
-                    summary.append(pInfo + "\n");
-                }
-                instance.setState(State.UNUSABLE);
-            } else {
-                instance.setState(State.INACTIVE);
-            }
-        }
-        state.setSummary(summary.toString());
-        instance.setAsterixRuntimeStates(state);
-    }
-
-    public static void verifyBackupRestoreConfiguration(String hdfsUrl, String hadoopVersion, String hdfsBackupDir)
-            throws Exception {
-        StringBuffer errorCheck = new StringBuffer();
-        if (hdfsUrl == null || hdfsUrl.length() == 0) {
-            errorCheck.append("\n HDFS Url not configured");
-        }
-        if (hadoopVersion == null || hadoopVersion.length() == 0) {
-            errorCheck.append("\n HDFS version not configured");
-        }
-        if (hdfsBackupDir == null || hdfsBackupDir.length() == 0) {
-            errorCheck.append("\n HDFS backup directory not configured");
-        }
-        if (errorCheck.length() > 0) {
-            throw new Exception("Incomplete hdfs configuration in " + InstallerDriver.getManagixHome() + File.separator
-                    + InstallerDriver.MANAGIX_CONF_XML + errorCheck);
-        }
-    }
+	public static void verifyBackupRestoreConfiguration(String hdfsUrl,
+			String hadoopVersion, String hdfsBackupDir) throws Exception {
+		StringBuffer errorCheck = new StringBuffer();
+		if (hdfsUrl == null || hdfsUrl.length() == 0) {
+			errorCheck.append("\n HDFS Url not configured");
+		}
+		if (hadoopVersion == null || hadoopVersion.length() == 0) {
+			errorCheck.append("\n HDFS version not configured");
+		}
+		if (hdfsBackupDir == null || hdfsBackupDir.length() == 0) {
+			errorCheck.append("\n HDFS backup directory not configured");
+		}
+		if (errorCheck.length() > 0) {
+			throw new Exception("Incomplete hdfs configuration in "
+					+ InstallerDriver.getManagixHome() + File.separator
+					+ InstallerDriver.MANAGIX_CONF_XML + errorCheck);
+		}
+	}
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
index fc53304..f6aeafe 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
@@ -29,8 +29,10 @@
 import edu.uci.ics.asterix.event.schema.pattern.Pattern;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
 import edu.uci.ics.asterix.event.schema.pattern.Value;
+import edu.uci.ics.asterix.installer.command.BackupCommand;
 import edu.uci.ics.asterix.installer.command.StopCommand;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
+import edu.uci.ics.asterix.installer.driver.InstallerUtil;
 import edu.uci.ics.asterix.installer.error.VerificationUtil;
 import edu.uci.ics.asterix.installer.model.AsterixInstance;
 import edu.uci.ics.asterix.installer.model.BackupInfo;
@@ -41,449 +43,412 @@
 
 public class PatternCreator {
 
-	private ILookupService lookupService = ServiceProvider.INSTANCE
-			.getLookupService();
+    private ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
 
-	private void addInitialDelay(Pattern p, int delay, String unit) {
-		Delay d = new Delay(new Value(null, "" + delay), unit);
-		p.setDelay(d);
-	}
+    private void addInitialDelay(Pattern p, int delay, String unit) {
+        Delay d = new Delay(new Value(null, "" + delay), unit);
+        p.setDelay(d);
+    }
 
-	public Patterns getStartAsterixPattern(String asterixInstanceName,
-			Cluster cluster) throws Exception {
-		String ccLocationId = cluster.getMasterNode().getId();
-		String ccLocationIp = cluster.getMasterNode().getIp();
+    public Patterns getStartAsterixPattern(String asterixInstanceName, Cluster cluster) throws Exception {
+        String ccLocationId = cluster.getMasterNode().getId();
+        String ccLocationIp = cluster.getMasterNode().getClusterIp();
 
-		String destDir = cluster.getWorkingDir().getDir() + File.separator
-				+ "asterix";
-		List<Pattern> ps = new ArrayList<Pattern>();
+        String destDir = cluster.getWorkingDir().getDir() + File.separator + "asterix";
+        List<Pattern> ps = new ArrayList<Pattern>();
 
-		Pattern copyHyracks = createCopyHyracksPattern(asterixInstanceName,
-				cluster, ccLocationIp, destDir);
-		ps.add(copyHyracks);
+        Pattern copyHyracks = createCopyHyracksPattern(asterixInstanceName, cluster, ccLocationIp, destDir);
+        ps.add(copyHyracks);
 
-		Pattern createCC = createCCStartPattern(ccLocationId);
-		addInitialDelay(createCC, 2, "sec");
-		ps.add(createCC);
+        Pattern createCC = createCCStartPattern(ccLocationId);
+        addInitialDelay(createCC, 3, "sec");
+        ps.add(createCC);
 
-		boolean copyHyracksToNC = !cluster.getWorkingDir().isNFS();
-		for (Node node : cluster.getNode()) {
-			if (copyHyracksToNC) {
-				Pattern copyHyracksForNC = createCopyHyracksPattern(
-						asterixInstanceName, cluster, node.getIp(), destDir);
-				ps.add(copyHyracksForNC);
-			}
-			Pattern createNC = createNCStartPattern(cluster.getMasterNode()
-					.getIp(), node.getId(),
-					asterixInstanceName + "_" + node.getId());
-			addInitialDelay(createNC, 4, "sec");
-			ps.add(createNC);
-		}
+        boolean copyHyracksToNC = !cluster.getWorkingDir().isNFS();
+        for (Node node : cluster.getNode()) {
+            if (copyHyracksToNC) {
+                Pattern copyHyracksForNC = createCopyHyracksPattern(asterixInstanceName, cluster, node.getClusterIp(),
+                        destDir);
+                ps.add(copyHyracksForNC);
+            }
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            Pattern createNC = createNCStartPattern(cluster.getMasterNode().getClusterIp(), node.getId(),
+                    asterixInstanceName + "_" + node.getId(), iodevices);
+            addInitialDelay(createNC, 5, "sec");
+            ps.add(createNC);
+        }
 
-		Patterns patterns = new Patterns(ps);
-		patterns.getPattern().addAll(
-				createHadoopLibraryTransferPattern(cluster).getPattern());
-		return patterns;
-	}
+        Patterns patterns = new Patterns(ps);
+        patterns.getPattern().addAll(createHadoopLibraryTransferPattern(cluster).getPattern());
+        return patterns;
+    }
 
-	public Patterns getStopCommandPattern(StopCommand stopCommand)
-			throws Exception {
-		List<Pattern> ps = new ArrayList<Pattern>();
-		AsterixInstance asterixInstance = lookupService
-				.getAsterixInstance(stopCommand.getAsterixInstanceName());
-		Cluster cluster = asterixInstance.getCluster();
+    public Patterns getStopCommandPattern(StopCommand stopCommand) throws Exception {
+        List<Pattern> ps = new ArrayList<Pattern>();
+        AsterixInstance asterixInstance = lookupService.getAsterixInstance(stopCommand.getAsterixInstanceName());
+        Cluster cluster = asterixInstance.getCluster();
 
-		String ccLocation = cluster.getMasterNode().getId();
-		Pattern createCC = createCCStopPattern(ccLocation);
-		addInitialDelay(createCC, 5, "sec");
-		ps.add(createCC);
+        String ccLocation = cluster.getMasterNode().getId();
+        Pattern createCC = createCCStopPattern(ccLocation);
+        addInitialDelay(createCC, 5, "sec");
+        ps.add(createCC);
 
-		String asterixInstanceName = stopCommand.getAsterixInstanceName();
-		int nodeControllerIndex = 1;
-		for (Node node : cluster.getNode()) {
-			Pattern createNC = createNCStopPattern(node.getId(),
-					asterixInstanceName + "_" + nodeControllerIndex);
-			ps.add(createNC);
-			nodeControllerIndex++;
-		}
+        String asterixInstanceName = stopCommand.getAsterixInstanceName();
+        int nodeControllerIndex = 1;
+        for (Node node : cluster.getNode()) {
+            Pattern createNC = createNCStopPattern(node.getId(), asterixInstanceName + "_" + nodeControllerIndex);
+            ps.add(createNC);
+            nodeControllerIndex++;
+        }
 
-		Patterns patterns = new Patterns(ps);
-		return patterns;
-	}
+        Patterns patterns = new Patterns(ps);
+        return patterns;
+    }
 
-	public Patterns getBackUpAsterixPattern(AsterixInstance instance,
-			Backup backupConf) throws Exception {
-		BackupType backupType = BackupInfo.getBackupType(backupConf);
-		Patterns patterns = null;
-		switch (backupType) {
-		case HDFS:
-			patterns = getHDFSBackUpAsterixPattern(instance, backupConf);
-			break;
-		case LOCAL:
-			patterns = getLocalBackUpAsterixPattern(instance, backupConf);
-			break;
-		}
-		return patterns;
-	}
+    public Patterns getBackUpAsterixPattern(AsterixInstance instance, Backup backupConf) throws Exception {
+        BackupType backupType = BackupInfo.getBackupType(backupConf);
+        Patterns patterns = null;
+        switch (backupType) {
+            case HDFS:
+                patterns = getHDFSBackUpAsterixPattern(instance, backupConf);
+                break;
+            case LOCAL:
+                patterns = getLocalBackUpAsterixPattern(instance, backupConf);
+                break;
+        }
+        return patterns;
+    }
 
-	public Patterns getRestoreAsterixPattern(AsterixInstance instance,
-			BackupInfo backupInfo) throws Exception {
-		BackupType backupType = backupInfo.getBackupType();
-		Patterns patterns = null;
-		switch (backupType) {
-		case HDFS:
-			patterns = getHDFSRestoreAsterixPattern(instance, backupInfo);
-			break;
-		case LOCAL:
-			patterns = getLocalRestoreAsterixPattern(instance, backupInfo);
-			break;
-		}
-		return patterns;
-	}
+    public Patterns getRestoreAsterixPattern(AsterixInstance instance, BackupInfo backupInfo) throws Exception {
+        BackupType backupType = backupInfo.getBackupType();
+        Patterns patterns = null;
+        switch (backupType) {
+            case HDFS:
+                patterns = getHDFSRestoreAsterixPattern(instance, backupInfo);
+                break;
+            case LOCAL:
+                patterns = getLocalRestoreAsterixPattern(instance, backupInfo);
+                break;
+        }
+        return patterns;
+    }
 
-	private Patterns getHDFSBackUpAsterixPattern(AsterixInstance instance,
-			Backup backupConf) throws Exception {
-		Cluster cluster = instance.getCluster();
-		String clusterStore = instance.getCluster().getStore();
-		String hdfsUrl = backupConf.getHdfs().getUrl();
-		String hadoopVersion = backupConf.getHdfs().getVersion();
-		String hdfsBackupDir = backupConf.getBackupDir();
-		VerificationUtil.verifyBackupRestoreConfiguration(hdfsUrl,
-				hadoopVersion, hdfsBackupDir);
-		String workingDir = cluster.getWorkingDir().getDir();
-		String backupId = "" + instance.getBackupInfo().size();
-		String nodeStore;
-		String pargs;
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		for (Node node : cluster.getNode()) {
-			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			nodeStore = node.getStore() == null ? clusterStore : node
-					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + hdfsBackupDir + " " + "hdfs" + " "
-					+ node.getId() + " " + hdfsUrl + " " + hadoopVersion;
-			Event event = new Event("backup", nodeid, pargs);
-			patternList.add(new Pattern(null, 1, null, event));
-		}
-		return new Patterns(patternList);
-	}
+    private Patterns getHDFSBackUpAsterixPattern(AsterixInstance instance, Backup backupConf) throws Exception {
+        Cluster cluster = instance.getCluster();
+        String hdfsUrl = backupConf.getHdfs().getUrl();
+        String hadoopVersion = backupConf.getHdfs().getVersion();
+        String hdfsBackupDir = backupConf.getBackupDir();
+        VerificationUtil.verifyBackupRestoreConfiguration(hdfsUrl, hadoopVersion, hdfsBackupDir);
+        String workingDir = cluster.getWorkingDir().getDir();
+        String backupId = "" + instance.getBackupInfo().size();
+        String store;
+        String pargs;
+        String iodevices;
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        for (Node node : cluster.getNode()) {
+            Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+            iodevices = node.getIodevices() == null ? instance.getCluster().getIodevices() : node.getIodevices();
+            store = node.getStore() == null ? cluster.getStore() : node.getStore();
+            pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + store + " "
+                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
+                    + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " " + hadoopVersion;
+            Event event = new Event("backup", nodeid, pargs);
+            patternList.add(new Pattern(null, 1, null, event));
+        }
+        return new Patterns(patternList);
+    }
 
-	private Patterns getLocalBackUpAsterixPattern(AsterixInstance instance,
-			Backup backupConf) throws Exception {
-		Cluster cluster = instance.getCluster();
-		String clusterStore = instance.getCluster().getStore();
-		String backupDir = backupConf.getBackupDir();
-		String workingDir = cluster.getWorkingDir().getDir();
-		String backupId = "" + instance.getBackupInfo().size();
-		String nodeStore;
-		String pargs;
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		for (Node node : cluster.getNode()) {
-			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			nodeStore = node.getStore() == null ? clusterStore : node
-					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + backupDir + " " + "local" + " "
-					+ node.getId();
-			Event event = new Event("backup", nodeid, pargs);
-			patternList.add(new Pattern(null, 1, null, event));
-		}
-		return new Patterns(patternList);
-	}
+    private Patterns getLocalBackUpAsterixPattern(AsterixInstance instance, Backup backupConf) throws Exception {
+        Cluster cluster = instance.getCluster();
+        String backupDir = backupConf.getBackupDir();
+        String workingDir = cluster.getWorkingDir().getDir();
+        String backupId = "" + instance.getBackupInfo().size();
+        String iodevices;
+        String store;
+        String pargs;
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        for (Node node : cluster.getNode()) {
+            Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+            iodevices = node.getIodevices() == null ? instance.getCluster().getIodevices() : node.getIodevices();
+            store = node.getStore() == null ? cluster.getStore() : node.getStore();
+            pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + store + " "
+                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
+                    + backupDir + " " + "local" + " " + node.getId();
+            Event event = new Event("backup", nodeid, pargs);
+            patternList.add(new Pattern(null, 1, null, event));
+        }
+        return new Patterns(patternList);
+    }
 
-	public Patterns getHDFSRestoreAsterixPattern(AsterixInstance instance,
-			BackupInfo backupInfo) throws Exception {
-		Cluster cluster = instance.getCluster();
-		String clusterStore = instance.getCluster().getStore();
-		String hdfsUrl = backupInfo.getBackupConf().getHdfs().getUrl();
-		String hadoopVersion = backupInfo.getBackupConf().getHdfs()
-				.getVersion();
-		String hdfsBackupDir = backupInfo.getBackupConf().getBackupDir();
-		VerificationUtil.verifyBackupRestoreConfiguration(hdfsUrl,
-				hadoopVersion, hdfsBackupDir);
-		String workingDir = cluster.getWorkingDir().getDir();
-		int backupId = backupInfo.getId();
-		String nodeStore;
-		String pargs;
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		for (Node node : cluster.getNode()) {
-			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			nodeStore = node.getStore() == null ? clusterStore : node
-					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + hdfsBackupDir + " " + "hdfs" + " "
-					+ node.getId() + " " + hdfsUrl + " " + hadoopVersion;
-			Event event = new Event("restore", nodeid, pargs);
-			patternList.add(new Pattern(null, 1, null, event));
-		}
-		return new Patterns(patternList);
-	}
+    public Patterns getHDFSRestoreAsterixPattern(AsterixInstance instance, BackupInfo backupInfo) throws Exception {
+        Cluster cluster = instance.getCluster();
+        String clusterStore = instance.getCluster().getStore();
+        String hdfsUrl = backupInfo.getBackupConf().getHdfs().getUrl();
+        String hadoopVersion = backupInfo.getBackupConf().getHdfs().getVersion();
+        String hdfsBackupDir = backupInfo.getBackupConf().getBackupDir();
+        VerificationUtil.verifyBackupRestoreConfiguration(hdfsUrl, hadoopVersion, hdfsBackupDir);
+        String workingDir = cluster.getWorkingDir().getDir();
+        int backupId = backupInfo.getId();
+        String nodeStore;
+        String pargs;
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        for (Node node : cluster.getNode()) {
+            Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            nodeStore = node.getStore() == null ? clusterStore : node.getStore();
+            pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + nodeStore + " "
+                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
+                    + " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId() + " " + hdfsUrl + " " + hadoopVersion;
+            Event event = new Event("restore", nodeid, pargs);
+            patternList.add(new Pattern(null, 1, null, event));
+        }
+        return new Patterns(patternList);
+    }
 
-	public Patterns getLocalRestoreAsterixPattern(AsterixInstance instance,
-			BackupInfo backupInfo) throws Exception {
-		Cluster cluster = instance.getCluster();
-		String clusterStore = instance.getCluster().getStore();
-		String backupDir = backupInfo.getBackupConf().getBackupDir();
-		String workingDir = cluster.getWorkingDir().getDir();
-		int backupId = backupInfo.getId();
-		String nodeStore;
-		String pargs;
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		for (Node node : cluster.getNode()) {
-			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			nodeStore = node.getStore() == null ? clusterStore : node
-					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + backupDir + " " + "local" + " "
-					+ node.getId();
-			Event event = new Event("restore", nodeid, pargs);
-			patternList.add(new Pattern(null, 1, null, event));
-		}
-		return new Patterns(patternList);
-	}
+    public Patterns getLocalRestoreAsterixPattern(AsterixInstance instance, BackupInfo backupInfo) throws Exception {
+        Cluster cluster = instance.getCluster();
+        String clusterStore = instance.getCluster().getStore();
+        String backupDir = backupInfo.getBackupConf().getBackupDir();
+        String workingDir = cluster.getWorkingDir().getDir();
+        int backupId = backupInfo.getId();
+        String nodeStore;
+        String pargs;
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        for (Node node : cluster.getNode()) {
+            Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            nodeStore = node.getStore() == null ? clusterStore : node.getStore();
+            pargs = workingDir + " " + instance.getName() + " " + iodevices + " " + nodeStore + " "
+                    + BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + InstallerUtil.TXN_LOG_DIR + " " + backupId + " "
+                    + backupDir + " " + "local" + " " + node.getId();
+            Event event = new Event("restore", nodeid, pargs);
+            patternList.add(new Pattern(null, 1, null, event));
+        }
+        return new Patterns(patternList);
+    }
 
-	public Patterns createHadoopLibraryTransferPattern(Cluster cluster)
-			throws Exception {
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		String workingDir = cluster.getWorkingDir().getDir();
-		String hadoopVersion = InstallerDriver.getConfiguration().getBackup()
-				.getHdfs().getVersion();
-		File hadoopDir = new File(InstallerDriver.getManagixHome()
-				+ File.separator + InstallerDriver.MANAGIX_INTERNAL_DIR
-				+ File.separator + "hadoop-" + hadoopVersion);
-		if (!hadoopDir.exists()) {
-			throw new IllegalStateException("Hadoop version :" + hadoopVersion
-					+ " not supported");
-		}
+    public Patterns createHadoopLibraryTransferPattern(Cluster cluster) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        String workingDir = cluster.getWorkingDir().getDir();
+        String hadoopVersion = InstallerDriver.getConfiguration().getBackup().getHdfs().getVersion();
+        File hadoopDir = new File(InstallerDriver.getManagixHome() + File.separator
+                + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "hadoop-" + hadoopVersion);
+        if (!hadoopDir.exists()) {
+            throw new IllegalStateException("Hadoop version :" + hadoopVersion + " not supported");
+        }
 
-		Nodeid nodeid = new Nodeid(new Value(null,
-				EventDriver.CLIENT_NODE.getId()));
-		String username = cluster.getUsername() != null ? cluster.getUsername()
-				: System.getProperty("user.name");
-		String pargs = username + " " + hadoopDir.getAbsolutePath() + " "
-				+ cluster.getMasterNode().getIp() + " " + workingDir;
-		Event event = new Event("directory_transfer", nodeid, pargs);
-		Pattern p = new Pattern(null, 1, null, event);
-		addInitialDelay(p, 2, "sec");
-		patternList.add(p);
+        Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+        String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
+        String pargs = username + " " + hadoopDir.getAbsolutePath() + " " + cluster.getMasterNode().getClusterIp()
+                + " " + workingDir;
+        Event event = new Event("directory_transfer", nodeid, pargs);
+        Pattern p = new Pattern(null, 1, null, event);
+        addInitialDelay(p, 2, "sec");
+        patternList.add(p);
 
-		boolean copyToNC = !cluster.getWorkingDir().isNFS();
-		if (copyToNC) {
-			for (Node node : cluster.getNode()) {
-				nodeid = new Nodeid(new Value(null, node.getId()));
-				pargs = cluster.getUsername() + " "
-						+ hadoopDir.getAbsolutePath() + " " + node.getIp()
-						+ " " + workingDir;
-				event = new Event("directory_transfer", nodeid, pargs);
-				p = new Pattern(null, 1, null, event);
-				addInitialDelay(p, 2, "sec");
-				patternList.add(p);
-			}
-		}
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+        boolean copyToNC = !cluster.getWorkingDir().isNFS();
+        if (copyToNC) {
+            for (Node node : cluster.getNode()) {
+                nodeid = new Nodeid(new Value(null, node.getId()));
+                pargs = cluster.getUsername() + " " + hadoopDir.getAbsolutePath() + " " + node.getClusterIp() + " "
+                        + workingDir;
+                event = new Event("directory_transfer", nodeid, pargs);
+                p = new Pattern(null, 1, null, event);
+                addInitialDelay(p, 2, "sec");
+                patternList.add(p);
+            }
+        }
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	public Patterns createDeleteInstancePattern(AsterixInstance instance)
-			throws Exception {
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		patternList.addAll(createRemoveAsterixStoragePattern(instance)
-				.getPattern());
-		if (instance.getBackupInfo() != null
-				&& instance.getBackupInfo().size() > 0) {
-			List<BackupInfo> backups = instance.getBackupInfo();
-			Set<String> removedBackupDirsHDFS = new HashSet<String>();
-			Set<String> removedBackupDirsLocal = new HashSet<String>();
+    public Patterns createDeleteInstancePattern(AsterixInstance instance) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        patternList.addAll(createRemoveAsterixStoragePattern(instance).getPattern());
+        if (instance.getBackupInfo() != null && instance.getBackupInfo().size() > 0) {
+            List<BackupInfo> backups = instance.getBackupInfo();
+            Set<String> removedBackupDirsHDFS = new HashSet<String>();
+            Set<String> removedBackupDirsLocal = new HashSet<String>();
 
-			String backupDir;
-			for (BackupInfo binfo : backups) {
-				backupDir = binfo.getBackupConf().getBackupDir();
-				switch (binfo.getBackupType()) {
-				case HDFS:
-					if (removedBackupDirsHDFS.contains(backups)) {
-						continue;
-					}
-					patternList.addAll(createRemoveHDFSBackupPattern(instance,
-							backupDir).getPattern());
-					removedBackupDirsHDFS.add(backupDir);
-					break;
+            String backupDir;
+            for (BackupInfo binfo : backups) {
+                backupDir = binfo.getBackupConf().getBackupDir();
+                switch (binfo.getBackupType()) {
+                    case HDFS:
+                        if (removedBackupDirsHDFS.contains(backups)) {
+                            continue;
+                        }
+                        patternList.addAll(createRemoveHDFSBackupPattern(instance, backupDir).getPattern());
+                        removedBackupDirsHDFS.add(backupDir);
+                        break;
 
-				case LOCAL:
-					if (removedBackupDirsLocal.contains(backups)) {
-						continue;
-					}
-					patternList.addAll(createRemoveLocalBackupPattern(instance,
-							backupDir).getPattern());
-					removedBackupDirsLocal.add(backupDir);
-					break;
-				}
+                    case LOCAL:
+                        if (removedBackupDirsLocal.contains(backups)) {
+                            continue;
+                        }
+                        patternList.addAll(createRemoveLocalBackupPattern(instance, backupDir).getPattern());
+                        removedBackupDirsLocal.add(backupDir);
+                        break;
+                }
 
-			}
-		}
-		patternList.addAll(createRemoveAsterixLogDirPattern(instance)
-				.getPattern());
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+            }
+        }
+        patternList.addAll(createRemoveAsterixLogDirPattern(instance).getPattern());
+        patternList.addAll(createRemoveAsterixRootMetadata(instance).getPattern());
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	private Patterns createRemoveHDFSBackupPattern(AsterixInstance instance,
-			String hdfsBackupDir) throws Exception {
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		Cluster cluster = instance.getCluster();
-		String hdfsUrl = InstallerDriver.getConfiguration().getBackup()
-				.getHdfs().getUrl();
-		String hadoopVersion = InstallerDriver.getConfiguration().getBackup()
-				.getHdfs().getVersion();
-		String workingDir = cluster.getWorkingDir().getDir();
-		Node launchingNode = cluster.getNode().get(0);
-		Nodeid nodeid = new Nodeid(new Value(null, launchingNode.getId()));
-		String pathToDelete = hdfsBackupDir + File.separator
-				+ instance.getName();
-		String pargs = workingDir + " " + hadoopVersion + " " + hdfsUrl + " "
-				+ pathToDelete;
-		Event event = new Event("hdfs_delete", nodeid, pargs);
-		patternList.add(new Pattern(null, 1, null, event));
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+    private Patterns createRemoveHDFSBackupPattern(AsterixInstance instance, String hdfsBackupDir) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
+        String hdfsUrl = InstallerDriver.getConfiguration().getBackup().getHdfs().getUrl();
+        String hadoopVersion = InstallerDriver.getConfiguration().getBackup().getHdfs().getVersion();
+        String workingDir = cluster.getWorkingDir().getDir();
+        Node launchingNode = cluster.getNode().get(0);
+        Nodeid nodeid = new Nodeid(new Value(null, launchingNode.getId()));
+        String pathToDelete = hdfsBackupDir + File.separator + instance.getName();
+        String pargs = workingDir + " " + hadoopVersion + " " + hdfsUrl + " " + pathToDelete;
+        Event event = new Event("hdfs_delete", nodeid, pargs);
+        patternList.add(new Pattern(null, 1, null, event));
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	private Patterns createRemoveLocalBackupPattern(AsterixInstance instance,
-			String localBackupDir) throws Exception {
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		Cluster cluster = instance.getCluster();
+    private Patterns createRemoveLocalBackupPattern(AsterixInstance instance, String localBackupDir) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
 
-		String pathToDelete = localBackupDir + File.separator
-				+ instance.getName();
-		String pargs = pathToDelete;
-		List<String> removedBackupDirs = new ArrayList<String>();
-		for (Node node : cluster.getNode()) {
-			if (removedBackupDirs.contains(node.getIp())) {
-				continue;
-			}
-			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			Event event = new Event("file_delete", nodeid, pargs);
-			patternList.add(new Pattern(null, 1, null, event));
-			removedBackupDirs.add(node.getIp());
-		}
+        String pathToDelete = localBackupDir + File.separator + instance.getName();
+        String pargs = pathToDelete;
+        List<String> removedBackupDirs = new ArrayList<String>();
+        for (Node node : cluster.getNode()) {
+            if (removedBackupDirs.contains(node.getClusterIp())) {
+                continue;
+            }
+            Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+            Event event = new Event("file_delete", nodeid, pargs);
+            patternList.add(new Pattern(null, 1, null, event));
+            removedBackupDirs.add(node.getClusterIp());
+        }
 
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	public Patterns createRemoveAsterixWorkingDirPattern(
-			AsterixInstance instance) throws Exception {
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		Cluster cluster = instance.getCluster();
-		String workingDir = cluster.getWorkingDir().getDir();
-		String pargs = workingDir;
-		Nodeid nodeid = new Nodeid(new Value(null, cluster.getMasterNode()
-				.getId()));
-		Event event = new Event("file_delete", nodeid, pargs);
-		patternList.add(new Pattern(null, 1, null, event));
+    public Patterns createRemoveAsterixWorkingDirPattern(AsterixInstance instance) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
+        String workingDir = cluster.getWorkingDir().getDir();
+        String pargs = workingDir;
+        Nodeid nodeid = new Nodeid(new Value(null, cluster.getMasterNode().getId()));
+        Event event = new Event("file_delete", nodeid, pargs);
+        patternList.add(new Pattern(null, 1, null, event));
 
-		if (!cluster.getWorkingDir().isNFS()) {
-			for (Node node : cluster.getNode()) {
-				nodeid = new Nodeid(new Value(null, node.getId()));
-				event = new Event("file_delete", nodeid, pargs);
-				patternList.add(new Pattern(null, 1, null, event));
-			}
-		}
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+        if (!cluster.getWorkingDir().isNFS()) {
+            for (Node node : cluster.getNode()) {
+                nodeid = new Nodeid(new Value(null, node.getId()));
+                event = new Event("file_delete", nodeid, pargs);
+                patternList.add(new Pattern(null, 1, null, event));
+            }
+        }
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	private Patterns createRemoveAsterixLogDirPattern(AsterixInstance instance)
-			throws Exception {
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		Cluster cluster = instance.getCluster();
-		String pargs = instance.getCluster().getLogdir();
-		Nodeid nodeid = new Nodeid(new Value(null, cluster.getMasterNode()
-				.getId()));
-		Event event = new Event("file_delete", nodeid, pargs);
-		patternList.add(new Pattern(null, 1, null, event));
+    private Patterns createRemoveAsterixRootMetadata(AsterixInstance instance) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
+        Nodeid nodeid = null;
+        String pargs = null;
+        Event event = null;
+        for (Node node : cluster.getNode()) {
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            String primaryIODevice = iodevices.split(",")[0].trim();
+            pargs = primaryIODevice + File.separator + BackupCommand.ASTERIX_ROOT_METADATA_DIR;
+            nodeid = new Nodeid(new Value(null, node.getId()));
+            event = new Event("file_delete", nodeid, pargs);
+            patternList.add(new Pattern(null, 1, null, event));
+        }
 
-		if (!cluster.getWorkingDir().isNFS()) {
-			for (Node node : cluster.getNode()) {
-				nodeid = new Nodeid(new Value(null, node.getId()));
-				event = new Event("file_delete", nodeid, pargs);
-				patternList.add(new Pattern(null, 1, null, event));
-			}
-		}
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	private Patterns createRemoveAsterixStoragePattern(AsterixInstance instance)
-			throws Exception {
-		List<Pattern> patternList = new ArrayList<Pattern>();
-		Cluster cluster = instance.getCluster();
-		String pargs = null;
+    private Patterns createRemoveAsterixLogDirPattern(AsterixInstance instance) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
+        String pargs = instance.getCluster().getLogdir();
+        Nodeid nodeid = new Nodeid(new Value(null, cluster.getMasterNode().getId()));
+        Event event = new Event("file_delete", nodeid, pargs);
+        patternList.add(new Pattern(null, 1, null, event));
 
-		for (Node node : cluster.getNode()) {
-			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			String[] nodeStores;
-			if (node.getStore() != null) {
-				nodeStores = node.getStore().trim().split(",");
-				for (String ns : nodeStores) {
-					pargs = ns + File.separator + instance.getName();
-				}
-			} else {
-				nodeStores = cluster.getStore().trim().split(",");
-				for (String ns : nodeStores) {
-					pargs = ns + File.separator + node.getId() + File.separator
-							+ instance.getName();
-				}
-			}
-			Event event = new Event("file_delete", nodeid, pargs);
-			patternList.add(new Pattern(null, 1, null, event));
-		}
-		Patterns patterns = new Patterns(patternList);
-		return patterns;
-	}
+        for (Node node : cluster.getNode()) {
+            nodeid = new Nodeid(new Value(null, node.getId()));
+            event = new Event("file_delete", nodeid, pargs);
+            patternList.add(new Pattern(null, 1, null, event));
+        }
 
-	private Pattern createCopyHyracksPattern(String instanceName,
-			Cluster cluster, String destinationIp, String destDir) {
-		Nodeid nodeid = new Nodeid(new Value(null,
-				EventDriver.CLIENT_NODE.getId()));
-		String username = cluster.getUsername() != null ? cluster.getUsername()
-				: System.getProperty("user.name");
-		String asterixZipName = InstallerDriver.getAsterixZip()
-				.substring(
-						InstallerDriver.getAsterixZip().lastIndexOf(
-								File.separator) + 1);
-		String fileToTransfer = new File(InstallerDriver.getAsterixDir()
-				+ File.separator + instanceName + File.separator
-				+ asterixZipName).getAbsolutePath();
-		String pargs = username + " " + fileToTransfer + " " + destinationIp
-				+ " " + destDir + " " + "unpack";
-		Event event = new Event("file_transfer", nodeid, pargs);
-		return new Pattern(null, 1, null, event);
-	}
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	private Pattern createCCStartPattern(String hostId) {
-		Nodeid nodeid = new Nodeid(new Value(null, hostId));
-		Event event = new Event("cc_start", nodeid, "");
-		return new Pattern(null, 1, null, event);
-	}
+    private Patterns createRemoveAsterixStoragePattern(AsterixInstance instance) throws Exception {
+        List<Pattern> patternList = new ArrayList<Pattern>();
+        Cluster cluster = instance.getCluster();
+        String pargs = null;
 
-	public Pattern createCCStopPattern(String hostId) {
-		Nodeid nodeid = new Nodeid(new Value(null, hostId));
-		Event event = new Event("cc_failure", nodeid, null);
-		return new Pattern(null, 1, null, event);
-	}
+        for (Node node : cluster.getNode()) {
+            Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+            String[] nodeIODevices;
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            nodeIODevices = iodevices.trim().split(",");
+            for (String nodeIODevice : nodeIODevices) {
+                String nodeStore = node.getStore() == null ? cluster.getStore() : node.getStore();
+                pargs = nodeIODevice.trim() + File.separator + nodeStore;
+                Event event = new Event("file_delete", nodeid, pargs);
+                patternList.add(new Pattern(null, 1, null, event));
+            }
+        }
+        Patterns patterns = new Patterns(patternList);
+        return patterns;
+    }
 
-	public Pattern createNCStartPattern(String ccHost, String hostId,
-			String nodeControllerId) {
-		Nodeid nodeid = new Nodeid(new Value(null, hostId));
-		Event event = new Event("node_join", nodeid, ccHost + " "
-				+ nodeControllerId);
-		return new Pattern(null, 1, null, event);
-	}
+    private Pattern createCopyHyracksPattern(String instanceName, Cluster cluster, String destinationIp, String destDir) {
+        Nodeid nodeid = new Nodeid(new Value(null, EventDriver.CLIENT_NODE.getId()));
+        String username = cluster.getUsername() != null ? cluster.getUsername() : System.getProperty("user.name");
+        String asterixZipName = InstallerDriver.getAsterixZip().substring(
+                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
+        String fileToTransfer = new File(InstallerDriver.getAsterixDir() + File.separator + instanceName
+                + File.separator + asterixZipName).getAbsolutePath();
+        String pargs = username + " " + fileToTransfer + " " + destinationIp + " " + destDir + " " + "unpack";
+        Event event = new Event("file_transfer", nodeid, pargs);
+        return new Pattern(null, 1, null, event);
+    }
 
-	public Pattern createNCStopPattern(String hostId, String nodeControllerId) {
-		Nodeid nodeid = new Nodeid(new Value(null, hostId));
-		Event event = new Event("node_failure", nodeid, nodeControllerId);
-		return new Pattern(null, 1, null, event);
-	}
+    private Pattern createCCStartPattern(String hostId) {
+        Nodeid nodeid = new Nodeid(new Value(null, hostId));
+        Event event = new Event("cc_start", nodeid, "");
+        return new Pattern(null, 1, null, event);
+    }
+
+    public Pattern createCCStopPattern(String hostId) {
+        Nodeid nodeid = new Nodeid(new Value(null, hostId));
+        Event event = new Event("cc_failure", nodeid, null);
+        return new Pattern(null, 1, null, event);
+    }
+
+    public Pattern createNCStartPattern(String ccHost, String hostId, String nodeControllerId, String iodevices) {
+        Nodeid nodeid = new Nodeid(new Value(null, hostId));
+        String pargs = ccHost + " " + nodeControllerId + " " + iodevices;
+        Event event = new Event("node_join", nodeid, pargs);
+        return new Pattern(null, 1, null, event);
+    }
+
+    public Pattern createNCStopPattern(String hostId, String nodeControllerId) {
+        Nodeid nodeid = new Nodeid(new Value(null, hostId));
+        Event event = new Event("node_failure", nodeid, nodeControllerId);
+        return new Pattern(null, 1, null, event);
+    }
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
index d3ba758..9f8b28d 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
@@ -25,157 +25,159 @@
 
 public class AsterixInstance implements Serializable {
 
-    private static final long serialVersionUID = 2874439550187520449L;
+	private static final long serialVersionUID = 2874439550187520449L;
 
-    public enum State {
-        ACTIVE,
-        INACTIVE,
-        UNUSABLE
-    }
+	public enum State {
+		ACTIVE, INACTIVE, UNUSABLE
+	}
 
-    private final Cluster cluster;
-    private final String name;
-    private final Date createdTimestamp;
-    private Date stateChangeTimestamp;
-    private Date modifiedTimestamp;
-    private Properties configuration;
-    private State state;
-    private final String metadataNodeId;
-    private final String asterixVersion;
-    private final List<BackupInfo> backupInfo;
-    private final String webInterfaceUrl;
-    private AsterixRuntimeState runtimeState;
-    private State previousState;
+	private final Cluster cluster;
+	private final String name;
+	private final Date createdTimestamp;
+	private Date stateChangeTimestamp;
+	private Date modifiedTimestamp;
+	private Properties configuration;
+	private State state;
+	private final String metadataNodeId;
+	private final String asterixVersion;
+	private final List<BackupInfo> backupInfo;
+	private final String webInterfaceUrl;
+	private AsterixRuntimeState runtimeState;
+	private State previousState;
 
-    public AsterixInstance(String name, Cluster cluster, Properties configuration, String metadataNodeId,
-            String asterixVersion) {
-        this.name = name;
-        this.cluster = cluster;
-        this.configuration = configuration;
-        this.metadataNodeId = metadataNodeId;
-        this.state = State.ACTIVE;
-        this.previousState = State.UNUSABLE;
-        this.asterixVersion = asterixVersion;
-        this.createdTimestamp = new Date();
-        this.backupInfo = new ArrayList<BackupInfo>();
-        this.webInterfaceUrl = "http://" + cluster.getMasterNode().getIp() + ":" + 19001;
-    }
+	public AsterixInstance(String name, Cluster cluster,
+			Properties configuration, String metadataNodeId,
+			String asterixVersion) {
+		this.name = name;
+		this.cluster = cluster;
+		this.configuration = configuration;
+		this.metadataNodeId = metadataNodeId;
+		this.state = State.ACTIVE;
+		this.previousState = State.UNUSABLE;
+		this.asterixVersion = asterixVersion;
+		this.createdTimestamp = new Date();
+		this.backupInfo = new ArrayList<BackupInfo>();
+		this.webInterfaceUrl = "http://"
+				+ cluster.getMasterNode().getClusterIp() + ":" + 19001;
+	}
 
-    public Date getModifiedTimestamp() {
-        return stateChangeTimestamp;
-    }
+	public Date getModifiedTimestamp() {
+		return stateChangeTimestamp;
+	}
 
-    public Properties getConfiguration() {
-        return configuration;
-    }
+	public Properties getConfiguration() {
+		return configuration;
+	}
 
-    public void setConfiguration(Properties properties) {
-        this.configuration = properties;
-    }
+	public void setConfiguration(Properties properties) {
+		this.configuration = properties;
+	}
 
-    public State getState() {
-        return state;
-    }
+	public State getState() {
+		return state;
+	}
 
-    public void setState(State state) {
-        this.previousState = this.state;
-        this.state = state;
-    }
+	public void setState(State state) {
+		this.previousState = this.state;
+		this.state = state;
+	}
 
-    public Cluster getCluster() {
-        return cluster;
-    }
+	public Cluster getCluster() {
+		return cluster;
+	}
 
-    public String getName() {
-        return name;
-    }
+	public String getName() {
+		return name;
+	}
 
-    public Date getCreatedTimestamp() {
-        return createdTimestamp;
-    }
+	public Date getCreatedTimestamp() {
+		return createdTimestamp;
+	}
 
-    public Date getStateChangeTimestamp() {
-        return stateChangeTimestamp;
-    }
+	public Date getStateChangeTimestamp() {
+		return stateChangeTimestamp;
+	}
 
-    public void setStateChangeTimestamp(Date stateChangeTimestamp) {
-        this.stateChangeTimestamp = stateChangeTimestamp;
-    }
+	public void setStateChangeTimestamp(Date stateChangeTimestamp) {
+		this.stateChangeTimestamp = stateChangeTimestamp;
+	}
 
-    public void setModifiedTimestamp(Date modifiedTimestamp) {
-        this.modifiedTimestamp = modifiedTimestamp;
-    }
+	public void setModifiedTimestamp(Date modifiedTimestamp) {
+		this.modifiedTimestamp = modifiedTimestamp;
+	}
 
-    public String getMetadataNodeId() {
-        return metadataNodeId;
-    }
+	public String getMetadataNodeId() {
+		return metadataNodeId;
+	}
 
-    public String getAsterixVersion() {
-        return asterixVersion;
-    }
+	public String getAsterixVersion() {
+		return asterixVersion;
+	}
 
-    public String getDescription(boolean detailed) {
-        StringBuffer buffer = new StringBuffer();
-        buffer.append("Name:" + name + "\n");
-        buffer.append("Created:" + createdTimestamp + "\n");
-        buffer.append("Web-Url:" + webInterfaceUrl + "\n");
-        buffer.append("State:" + state);
-        if (!state.equals(State.UNUSABLE) && stateChangeTimestamp != null) {
-            buffer.append(" (" + stateChangeTimestamp + ")" + "\n");
-        } else {
-            buffer.append("\n");
-        }
-        if (modifiedTimestamp != null) {
-            buffer.append("Last modified timestamp:" + modifiedTimestamp + "\n");
-        }
+	public String getDescription(boolean detailed) {
+		StringBuffer buffer = new StringBuffer();
+		buffer.append("Name:" + name + "\n");
+		buffer.append("Created:" + createdTimestamp + "\n");
+		buffer.append("Web-Url:" + webInterfaceUrl + "\n");
+		buffer.append("State:" + state);
+		if (!state.equals(State.UNUSABLE) && stateChangeTimestamp != null) {
+			buffer.append(" (" + stateChangeTimestamp + ")" + "\n");
+		} else {
+			buffer.append("\n");
+		}
+		if (modifiedTimestamp != null) {
+			buffer.append("Last modified timestamp:" + modifiedTimestamp + "\n");
+		}
 
-        if (runtimeState.getSummary() != null && runtimeState.getSummary().length() > 0) {
-            buffer.append("\nWARNING!:" + runtimeState.getSummary() + "\n");
-        }
-        if (detailed) {
-            addDetailedInformation(buffer);
-        }
-        return buffer.toString();
-    }
+		if (runtimeState.getSummary() != null
+				&& runtimeState.getSummary().length() > 0) {
+			buffer.append("\nWARNING!:" + runtimeState.getSummary() + "\n");
+		}
+		if (detailed) {
+			addDetailedInformation(buffer);
+		}
+		return buffer.toString();
+	}
 
-    public List<BackupInfo> getBackupInfo() {
-        return backupInfo;
-    }
+	public List<BackupInfo> getBackupInfo() {
+		return backupInfo;
+	}
 
-    public String getWebInterfaceUrl() {
-        return webInterfaceUrl;
-    }
+	public String getWebInterfaceUrl() {
+		return webInterfaceUrl;
+	}
 
-    public AsterixRuntimeState getAsterixRuntimeState() {
-        return runtimeState;
-    }
+	public AsterixRuntimeState getAsterixRuntimeState() {
+		return runtimeState;
+	}
 
-    public void setAsterixRuntimeStates(AsterixRuntimeState runtimeState) {
-        this.runtimeState = runtimeState;
-    }
+	public void setAsterixRuntimeStates(AsterixRuntimeState runtimeState) {
+		this.runtimeState = runtimeState;
+	}
 
-    private void addDetailedInformation(StringBuffer buffer) {
-        buffer.append("Master node:" + cluster.getMasterNode().getId() + ":" + cluster.getMasterNode().getIp() + "\n");
-        for (Node node : cluster.getNode()) {
-            buffer.append(node.getId() + ":" + node.getIp() + "\n");
-        }
+	private void addDetailedInformation(StringBuffer buffer) {
+		buffer.append("Master node:" + cluster.getMasterNode().getId() + ":"
+				+ cluster.getMasterNode().getClusterIp() + "\n");
+		for (Node node : cluster.getNode()) {
+			buffer.append(node.getId() + ":" + node.getClusterIp() + "\n");
+		}
 
-        if (backupInfo != null && backupInfo.size() > 0) {
-            for (BackupInfo info : backupInfo) {
-                buffer.append(info + "\n");
-            }
-        }
-        buffer.append("\n");
-        buffer.append("Asterix version:" + asterixVersion + "\n");
-        buffer.append("Metadata Node:" + metadataNodeId + "\n");
-        buffer.append("Processes" + "\n");
-        for (ProcessInfo pInfo : runtimeState.getProcesses()) {
-            buffer.append(pInfo + "\n");
-        }
+		if (backupInfo != null && backupInfo.size() > 0) {
+			for (BackupInfo info : backupInfo) {
+				buffer.append(info + "\n");
+			}
+		}
+		buffer.append("\n");
+		buffer.append("Asterix version:" + asterixVersion + "\n");
+		buffer.append("Metadata Node:" + metadataNodeId + "\n");
+		buffer.append("Processes" + "\n");
+		for (ProcessInfo pInfo : runtimeState.getProcesses()) {
+			buffer.append(pInfo + "\n");
+		}
 
-    }
+	}
 
-    public State getPreviousState() {
-        return previousState;
-    }
+	public State getPreviousState() {
+		return previousState;
+	}
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
index 2880671..42ee112 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
@@ -21,11 +21,13 @@
     private static final long serialVersionUID = 304186774065853730L;
     private final String processName;
     private final String host;
+    private final String nodeId;
     private final int processId;
 
-    public ProcessInfo(String processName, String host, int processId) {
+    public ProcessInfo(String processName, String host, String nodeId, int processId) {
         this.processName = processName;
         this.host = host;
+        this.nodeId = nodeId;
         this.processId = processId;
     }
 
@@ -41,8 +43,12 @@
         return processId;
     }
 
+    public String getNodeId() {
+        return nodeId;
+    }
+
     public String toString() {
-        return processName + " at " + host + " [ " + processId + " ] ";
+        return processName + " at " + nodeId + " [ " + processId + " ] ";
     }
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java
index 1bd8772..b87d721 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/service/ZooKeeperService.java
@@ -98,6 +98,7 @@
         StringBuffer cmdBuffer = new StringBuffer();
         cmdBuffer.append(initScript + " ");
         cmdBuffer.append(conf.getZookeeper().getHomeDir() + " ");
+        cmdBuffer.append(conf.getZookeeper().getServers().getJavaHome() + " ");
         List<String> zkServers = conf.getZookeeper().getServers().getServer();
         for (String zkServer : zkServers) {
             cmdBuffer.append(zkServer + " ");
diff --git a/asterix-installer/src/main/resources/clusters/local/local.xml b/asterix-installer/src/main/resources/clusters/local/local.xml
index 7a8e78f..d397bcd 100644
--- a/asterix-installer/src/main/resources/clusters/local/local.xml
+++ b/asterix-installer/src/main/resources/clusters/local/local.xml
@@ -1,20 +1,21 @@
 <cluster xmlns="cluster">
   <name>local</name>
   <workingDir>
-     <dir>/tmp/asterix-installer</dir>
-     <NFS>true</NFS> 
+    <dir>/tmp/asterix-installer</dir>
+    <NFS>true</NFS>
   </workingDir>
   <logdir>/tmp/asterix/logs</logdir>
-  <store>/tmp/asterix/storage</store>
+  <iodevices>/tmp</iodevices>
+  <store>asterix/storage</store>
   <java_home></java_home>
-  <java_heap>1024m</java_heap>
+  <java_opts>-Xmx1024m</java_opts>
   <master-node>
-     <id>master</id>
-     <ip>127.0.0.1</ip>
-     <cluster-ip>127.0.0.1</cluster-ip>
+    <id>master</id>
+    <client-ip>127.0.0.1</client-ip>
+    <cluster-ip>127.0.0.1</cluster-ip>
   </master-node>
   <node>
-     <id>node1</id>
-     <ip>127.0.0.1</ip>
+    <id>node1</id>
+    <cluster-ip>127.0.0.1</cluster-ip>
   </node>
 </cluster>
diff --git a/asterix-installer/src/main/resources/conf/managix-conf.xml b/asterix-installer/src/main/resources/conf/managix-conf.xml
index ed3f8a2..10a62a9 100644
--- a/asterix-installer/src/main/resources/conf/managix-conf.xml
+++ b/asterix-installer/src/main/resources/conf/managix-conf.xml
@@ -10,7 +10,7 @@
     <homeDir>/tmp/zookeeper</homeDir>
     <clientPort>2900</clientPort>
     <servers>
-      <server>localhost</server>
+      <server>127.0.0.1</server>
     </servers>
   </zookeeper>
 </configuration>
diff --git a/asterix-installer/src/main/resources/schema/installer-conf.xsd b/asterix-installer/src/main/resources/schema/installer-conf.xsd
index 318e7ce..c21fc5b 100644
--- a/asterix-installer/src/main/resources/schema/installer-conf.xsd
+++ b/asterix-installer/src/main/resources/schema/installer-conf.xsd
@@ -11,6 +11,7 @@
 <xs:element name="version" type="xs:string"/>
 <xs:element name="url" type="xs:string"/>
 <xs:element name="backupDir" type="xs:string"/>
+<xs:element name="java_home" type="xs:string"/>
 
 <!-- definition of complex elements -->
 <xs:element name="hdfs">
@@ -44,6 +45,7 @@
 <xs:element name="servers">
   <xs:complexType>
     <xs:sequence>
+      <xs:element ref="mg:java_home"/>
       <xs:element ref="mg:server" maxOccurs="unbounded"/>
     </xs:sequence>
   </xs:complexType>
diff --git a/asterix-installer/src/main/resources/zookeeper/start_zk.sh b/asterix-installer/src/main/resources/zookeeper/start_zk.sh
index 033e5f8..68ced54 100755
--- a/asterix-installer/src/main/resources/zookeeper/start_zk.sh
+++ b/asterix-installer/src/main/resources/zookeeper/start_zk.sh
@@ -1,8 +1,9 @@
 ZK_HOME=$1
 ZK_ID=$2
+JAVA_HOME=$3
 mkdir $ZK_HOME/data
 echo $2 > $ZK_HOME/data/myid
 CLASSPATH=$ZK_HOME/lib/zookeeper-3.4.5.jar:$ZK_HOME/lib/log4j-1.2.15.jar:$ZK_HOME/lib/slf4j-api-1.6.1.jar:$ZK_HOME/conf:$ZK_HOME/conf/log4j.properties
 ZK_CONF=$ZK_HOME/zk.cfg
 export JAVA_OPTS="-Xdebug -Xrunjdwp:transport=dt_socket,address=8400,server=y,suspend=n"
-java $JAVA_OPTS -Dlog4j.configuration="file:$ZK_HOME/conf/log4j.properties" -cp $CLASSPATH org.apache.zookeeper.server.quorum.QuorumPeerMain $ZK_CONF
+$JAVA_HOME/bin/java $JAVA_OPTS -Dlog4j.configuration="file:$ZK_HOME/conf/log4j.properties" -cp $CLASSPATH org.apache.zookeeper.server.quorum.QuorumPeerMain $ZK_CONF
diff --git a/asterix-installer/src/main/resources/zookeeper/stop_zk b/asterix-installer/src/main/resources/zookeeper/stop_zk
index d8615d9..9bef86e 100755
--- a/asterix-installer/src/main/resources/zookeeper/stop_zk
+++ b/asterix-installer/src/main/resources/zookeeper/stop_zk
@@ -2,5 +2,5 @@
 shift 1
 for zk_host in  $@
 do
-  ssh $zk_host "kill -9 `jps | grep QuorumPeerMain | cut -d " "  -f1`" &
+  ssh $zk_host "kill -15 `jps | grep QuorumPeerMain | cut -d " "  -f1`" &
 done
diff --git a/asterix-installer/src/main/resources/zookeeper/zk.init b/asterix-installer/src/main/resources/zookeeper/zk.init
index 937d8b3..8ef6f5f 100755
--- a/asterix-installer/src/main/resources/zookeeper/zk.init
+++ b/asterix-installer/src/main/resources/zookeeper/zk.init
@@ -1,5 +1,6 @@
 ZK_HOME=$1
-shift 1
+SERVER_JAVA_HOME=$2
+shift 2
 cd $MANAGIX_HOME/.installer/zookeeper
 tar cf zk.pkg.tar *
 zk_server_id=1
@@ -8,6 +9,6 @@
   ssh $zk_host "mkdir -p $ZK_HOME"
   scp ./zk.pkg.tar $zk_host:$ZK_HOME/
   ssh $zk_host "cd $ZK_HOME && tar xf $ZK_HOME/zk.pkg.tar && chmod +x $ZK_HOME/bin/start_zk.sh"
-  ssh $zk_host "$ZK_HOME/bin/start_zk.sh $ZK_HOME $zk_server_id" &
+  ssh $zk_host "$ZK_HOME/bin/start_zk.sh $ZK_HOME $zk_server_id $SERVER_JAVA_HOME" &
   zk_server_id=`expr $zk_server_id + 1`	
 done
diff --git a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/java/edu/uci/ics/asterix/lexergenerator/rules/RulePartial.java b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/java/edu/uci/ics/asterix/lexergenerator/rules/RulePartial.java
index 89caf4f..045df58 100644
--- a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/java/edu/uci/ics/asterix/lexergenerator/rules/RulePartial.java
+++ b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/java/edu/uci/ics/asterix/lexergenerator/rules/RulePartial.java
@@ -61,8 +61,10 @@
     public String javaMatch(String action) {
         StringBuilder result = new StringBuilder();
         result.append("if (parse_" + partialName + "(currentChar)==TOKEN_" + partialName + "){");
+        result.append("currentChar = buffer[bufpos];");
         result.append(action);
         result.append("}");
+        result.append("else { currentChar = buffer[bufpos];}");
         return result.toString();
     }
 
diff --git a/asterix-metadata/pom.xml b/asterix-metadata/pom.xml
index ae90e4e..6f34026 100644
--- a/asterix-metadata/pom.xml
+++ b/asterix-metadata/pom.xml
@@ -43,20 +43,34 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-invertedindex</artifactId>
-		</dependency>
-		<dependency>
-			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-rtree</artifactId>
-		</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>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+                		hyracks-storage-am-lsm-invertedindex
+                	</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+                		hyracks-storage-am-lsm-btree
+                	</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>
+                		hyracks-storage-am-lsm-rtree
+                	</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
+		</dependency>
+		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-core</artifactId>
 			<version>0.20.2</version>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
index 66d46f9..0382315 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataCache.java
@@ -27,6 +27,7 @@
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
 
 /**
@@ -40,6 +41,8 @@
     protected final Map<String, Dataverse> dataverses = new HashMap<String, Dataverse>();
     // Key is dataverse name. Key of value map is dataset name.
     protected final Map<String, Map<String, Dataset>> datasets = new HashMap<String, Map<String, Dataset>>();
+    // Key is dataverse name. Key of value map is dataset name. Key of value map of value map is index name.
+    protected final Map<String, Map<String, Map<String, Index>>> indexes = new HashMap<String, Map<String, Map<String, Index>>>();
     // Key is dataverse name. Key of value map is datatype name.
     protected final Map<String, Map<String, Datatype>> datatypes = new HashMap<String, Map<String, Datatype>>();
     // Key is dataverse name.
@@ -79,15 +82,18 @@
         synchronized (dataverses) {
             synchronized (nodeGroups) {
                 synchronized (datasets) {
-                    synchronized (datatypes) {
-                        synchronized (functions) {
-                            synchronized (adapters) {
-                                dataverses.clear();
-                                nodeGroups.clear();
-                                datasets.clear();
-                                datatypes.clear();
-                                functions.clear();
-                                adapters.clear();
+                    synchronized (indexes) {
+                        synchronized (datatypes) {
+                            synchronized (functions) {
+                                synchronized (adapters) {
+                                    dataverses.clear();
+                                    nodeGroups.clear();
+                                    datasets.clear();
+                                    indexes.clear();
+                                    datatypes.clear();
+                                    functions.clear();
+                                    adapters.clear();
+                                }
                             }
                         }
                     }
@@ -126,6 +132,25 @@
         }
     }
 
+    public Object addIndexIfNotExists(Index index) {
+        synchronized (indexes) {
+            Map<String, Map<String, Index>> datasetMap = indexes.get(index.getDataverseName());
+            if (datasetMap == null) {
+                datasetMap = new HashMap<String, Map<String, Index>>();
+                indexes.put(index.getDataverseName(), datasetMap);
+            }
+            Map<String, Index> indexMap = datasetMap.get(index.getDatasetName());
+            if (indexMap == null) {
+                indexMap = new HashMap<String, Index>();
+                datasetMap.put(index.getDatasetName(), indexMap);
+            }
+            if (!indexMap.containsKey(index.getIndexName())) {
+                return indexMap.put(index.getIndexName(), index);
+            }
+            return null;
+        }
+    }
+
     public Object addDatatypeIfNotExists(Datatype datatype) {
         synchronized (datatypes) {
             Map<String, Datatype> m = datatypes.get(datatype.getDataverseName());
@@ -152,21 +177,24 @@
     public Object dropDataverse(Dataverse dataverse) {
         synchronized (dataverses) {
             synchronized (datasets) {
-                synchronized (datatypes) {
-                    synchronized (functions) {
-                        datasets.remove(dataverse.getDataverseName());
-                        datatypes.remove(dataverse.getDataverseName());
-                        adapters.remove(dataverse.getDataverseName());
-                        List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
-                        for (FunctionSignature signature : functions.keySet()) {
-                            if (signature.getNamespace().equals(dataverse.getDataverseName())) {
-                                markedFunctionsForRemoval.add(signature);
+                synchronized (indexes) {
+                    synchronized (datatypes) {
+                        synchronized (functions) {
+                            datasets.remove(dataverse.getDataverseName());
+                            indexes.remove(dataverse.getDataverseName());
+                            datatypes.remove(dataverse.getDataverseName());
+                            adapters.remove(dataverse.getDataverseName());
+                            List<FunctionSignature> markedFunctionsForRemoval = new ArrayList<FunctionSignature>();
+                            for (FunctionSignature signature : functions.keySet()) {
+                                if (signature.getNamespace().equals(dataverse.getDataverseName())) {
+                                    markedFunctionsForRemoval.add(signature);
+                                }
                             }
+                            for (FunctionSignature signature : markedFunctionsForRemoval) {
+                                functions.remove(signature);
+                            }
+                            return dataverses.remove(dataverse.getDataverseName());
                         }
-                        for (FunctionSignature signature : markedFunctionsForRemoval) {
-                            functions.remove(signature);
-                        }
-                        return dataverses.remove(dataverse.getDataverseName());
                     }
                 }
             }
@@ -175,11 +203,37 @@
 
     public Object dropDataset(Dataset dataset) {
         synchronized (datasets) {
-            Map<String, Dataset> m = datasets.get(dataset.getDataverseName());
-            if (m == null) {
+            synchronized (indexes) {
+
+                //remove the indexes of the dataset from indexes' cache
+                Map<String, Map<String, Index>> datasetMap = indexes.get(dataset.getDataverseName());
+                if (datasetMap != null) {
+                    datasetMap.remove(dataset.getDatasetName());
+                }
+
+                //remove the dataset from datasets' cache
+                Map<String, Dataset> m = datasets.get(dataset.getDataverseName());
+                if (m == null) {
+                    return null;
+                }
+                return m.remove(dataset.getDatasetName());
+            }
+        }
+    }
+    
+    public Object dropIndex(Index index) {
+        synchronized (indexes) {
+            Map<String, Map<String, Index>> datasetMap = indexes.get(index.getDataverseName());
+            if (datasetMap == null) {
                 return null;
             }
-            return m.remove(dataset.getDatasetName());
+            
+            Map<String, Index> indexMap = datasetMap.get(index.getDatasetName());
+            if (indexMap == null) {
+                return null;
+            }
+            
+            return indexMap.remove(index.getIndexName());
         }
     }
 
@@ -214,6 +268,20 @@
             return m.get(datasetName);
         }
     }
+    
+    public Index getIndex(String dataverseName, String datasetName, String indexName) {
+        synchronized (indexes) {
+            Map<String, Map<String, Index>> datasetMap = indexes.get(dataverseName);
+            if (datasetMap == null) {
+                return null;
+            }
+            Map<String, Index> indexMap = datasetMap.get(datasetName);
+            if (indexMap == null) {
+                return null;
+            }
+            return indexMap.get(indexName);
+        }
+    }
 
     public Datatype getDatatype(String dataverseName, String datatypeName) {
         synchronized (datatypes) {
@@ -311,7 +379,8 @@
             DatasourceAdapter adapterObject = adapters.get(adapter.getAdapterIdentifier().getNamespace()).get(
                     adapter.getAdapterIdentifier().getAdapterName());
             if (adapterObject != null) {
-                Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier().getNamespace());
+                Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier()
+                        .getNamespace());
                 if (adaptersInDataverse == null) {
                     adaptersInDataverse = new HashMap<String, DatasourceAdapter>();
                     adapters.put(adapter.getAdapterIdentifier().getNamespace(), adaptersInDataverse);
@@ -324,7 +393,8 @@
 
     public Object dropAdapter(DatasourceAdapter adapter) {
         synchronized (adapters) {
-            Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier().getNamespace());
+            Map<String, DatasourceAdapter> adaptersInDataverse = adapters.get(adapter.getAdapterIdentifier()
+                    .getNamespace());
             if (adaptersInDataverse != null) {
                 return adaptersInDataverse.remove(adapter.getAdapterIdentifier().getAdapterName());
             }
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 9f1b9b5..17cfc9a 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
@@ -17,14 +17,16 @@
 
 import java.rmi.RemoteException;
 import java.util.List;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.api.IMetadataManager;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
@@ -32,7 +34,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.
@@ -78,10 +81,10 @@
 public class MetadataManager implements IMetadataManager {
     // Set in init().
     public static MetadataManager INSTANCE;
-
     private final MetadataCache cache = new MetadataCache();
     private IAsterixStateProxy proxy;
     private IMetadataNode metadataNode;
+    private final ReadWriteLock metadataLatch;
 
     public MetadataManager(IAsterixStateProxy proxy) {
         if (proxy == null) {
@@ -89,6 +92,7 @@
         }
         this.proxy = proxy;
         this.metadataNode = null;
+        this.metadataLatch = new ReentrantReadWriteLock(true);
     }
 
     @Override
@@ -108,36 +112,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);
         }
@@ -147,7 +151,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);
         }
@@ -155,6 +159,15 @@
     }
 
     @Override
+    public List<Dataverse> getDataverses(MetadataTransactionContext ctx) throws MetadataException {
+        try {
+            return metadataNode.getDataverses(ctx.getJobId());
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
     public Dataverse getDataverse(MetadataTransactionContext ctx, String dataverseName) throws MetadataException {
         // First look in the context to see if this transaction created the
         // requested dataverse itself (but the dataverse is still uncommitted).
@@ -175,7 +188,7 @@
             return dataverse;
         }
         try {
-            dataverse = metadataNode.getDataverse(ctx.getTxnId(), dataverseName);
+            dataverse = metadataNode.getDataverse(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -194,7 +207,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);
         }
@@ -205,11 +218,14 @@
 
     @Override
     public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException {
+        // add dataset into metadataNode 
         try {
-            metadataNode.addDataset(ctx.getTxnId(), dataset);
+            metadataNode.addDataset(ctx.getJobId(), dataset);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
+
+        // reflect the dataset into the cache
         ctx.addDataset(dataset);
     }
 
@@ -217,7 +233,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);
         }
@@ -227,6 +243,7 @@
     @Override
     public Dataset getDataset(MetadataTransactionContext ctx, String dataverseName, String datasetName)
             throws MetadataException {
+
         // First look in the context to see if this transaction created the
         // requested dataset itself (but the dataset is still uncommitted).
         Dataset dataset = ctx.getDataset(dataverseName, datasetName);
@@ -253,7 +270,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);
         }
@@ -270,7 +287,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);
         }
@@ -280,7 +297,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);
         }
@@ -291,7 +308,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);
         }
@@ -327,7 +344,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);
         }
@@ -342,16 +359,17 @@
     @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);
         }
+        ctx.addIndex(index);
     }
 
     @Override
     public void addAdapter(MetadataTransactionContext mdTxnCtx, DatasourceAdapter adapter) throws MetadataException {
         try {
-            metadataNode.addAdapter(mdTxnCtx.getTxnId(), adapter);
+            metadataNode.addAdapter(mdTxnCtx.getJobId(), adapter);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -363,26 +381,62 @@
     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);
         }
+        ctx.dropIndex(dataverseName, datasetName, indexName);
     }
 
     @Override
     public Index getIndex(MetadataTransactionContext ctx, String dataverseName, String datasetName, String indexName)
             throws MetadataException {
+
+        // First look in the context to see if this transaction created the
+        // requested index itself (but the index is still uncommitted).
+        Index index = ctx.getIndex(dataverseName, datasetName, indexName);
+        if (index != null) {
+            // Don't add this index to the cache, since it is still
+            // uncommitted.
+            return index;
+        }
+
+        if (ctx.indexIsDropped(dataverseName, datasetName, indexName)) {
+            // Index has been dropped by this transaction but could still be
+            // in the cache.
+            return null;
+        }
+
+        //TODO 
+        //check what this is for?
+        if (!MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverseName) && ctx.getDataverse(dataverseName) != null) {
+            // This transaction has dropped and subsequently created the same
+            // dataverse.
+            return null;
+        }
+
+        index = cache.getIndex(dataverseName, datasetName, indexName);
+        if (index != null) {
+            // Index is already in the cache, don't add it again.
+            return index;
+        }
         try {
-            return metadataNode.getIndex(ctx.getTxnId(), dataverseName, datasetName, indexName);
+            index = metadataNode.getIndex(ctx.getJobId(), dataverseName, datasetName, indexName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
+        // We fetched the index from the MetadataNode. Add it to the cache
+        // when this transaction commits.
+        if (index != null) {
+            ctx.addIndex(index);
+        }
+        return index;
     }
 
     @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);
         }
@@ -391,7 +445,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);
         }
@@ -401,7 +455,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);
         }
@@ -429,7 +483,7 @@
             return nodeGroup;
         }
         try {
-            nodeGroup = metadataNode.getNodeGroup(ctx.getTxnId(), nodeGroupName);
+            nodeGroup = metadataNode.getNodeGroup(ctx.getJobId(), nodeGroupName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -444,7 +498,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);
         }
@@ -455,7 +509,7 @@
     public void dropFunction(MetadataTransactionContext ctx, FunctionSignature functionSignature)
             throws MetadataException {
         try {
-            metadataNode.dropFunction(ctx.getTxnId(), functionSignature);
+            metadataNode.dropFunction(ctx.getJobId(), functionSignature);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -489,7 +543,7 @@
             return function;
         }
         try {
-            function = metadataNode.getFunction(ctx.getTxnId(), functionSignature);
+            function = metadataNode.getFunction(ctx.getJobId(), functionSignature);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -503,13 +557,22 @@
     }
 
     @Override
+    public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException {
+        try {
+            metadataNode.initializeDatasetIdFactory(ctx.getJobId());
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+    }
+
+    @Override
     public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
             throws MetadataException {
         List<Function> dataverseFunctions;
         try {
             // Assuming that the transaction can read its own writes on the
             // metadata node.
-            dataverseFunctions = metadataNode.getDataverseFunctions(ctx.getTxnId(), dataverseName);
+            dataverseFunctions = metadataNode.getDataverseFunctions(ctx.getJobId(), dataverseName);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -521,7 +584,7 @@
     @Override
     public void dropAdapter(MetadataTransactionContext ctx, String dataverseName, String name) throws MetadataException {
         try {
-            metadataNode.dropAdapter(ctx.getTxnId(), dataverseName, name);
+            metadataNode.dropAdapter(ctx.getJobId(), dataverseName, name);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
@@ -532,11 +595,31 @@
             throws MetadataException {
         DatasourceAdapter adapter = null;
         try {
-            adapter = metadataNode.getAdapter(ctx.getTxnId(), dataverseName, name);
+            adapter = metadataNode.getAdapter(ctx.getJobId(), dataverseName, name);
         } catch (RemoteException e) {
             throw new MetadataException(e);
         }
         return adapter;
     }
 
+    @Override
+    public void acquireWriteLatch() {
+        metadataLatch.writeLock().lock();
+    }
+
+    @Override
+    public void releaseWriteLatch() {
+        metadataLatch.writeLock().unlock();
+    }
+
+    @Override
+    public void acquireReadLatch() {
+        metadataLatch.readLock().lock();
+    }
+
+    @Override
+    public void releaseReadLatch() {
+        metadataLatch.readLock().unlock();
+    }
+
 }
\ 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 120f1a5..d9c6ed4 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
@@ -30,8 +30,8 @@
 import edu.uci.ics.asterix.metadata.api.IValueExtractor;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataSecondaryIndexes;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
@@ -39,8 +39,8 @@
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Node;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasetTupleTranslator;
+import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
@@ -56,9 +56,15 @@
 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.opcallbacks.PrimaryIndexModificationOperationCallback;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallback;
+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.IResourceManager.ResourceType;
+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;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext.TransactionType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
@@ -66,28 +72,28 @@
 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.dataflow.common.util.TupleUtils;
 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.IModificationOperationCallback;
 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.common.api.ILSMIndex;
 
 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 TransactionProvider transactionProvider;
+    private IIndexLifecycleManager indexLifecycleManager;
+    private TransactionSubsystem transactionSubsystem;
 
     public static final MetadataNode INSTANCE = new MetadataNode();
 
@@ -96,26 +102,26 @@
     }
 
     public void initialize(AsterixAppRuntimeContext runtimeContext) {
-        this.transactionProvider = runtimeContext.getTransactionProvider();
-        this.indexRegistry = runtimeContext.getIndexRegistry();
+        this.transactionSubsystem = runtimeContext.getTransactionSubsystem();
+        this.indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
     }
 
     @Override
-    public void beginTransaction(long transactionId) throws ACIDException, RemoteException {
-        transactionProvider.getTransactionManager().beginTransaction(transactionId);
+    public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
+        transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
     }
 
     @Override
-    public void commitTransaction(long txnId) throws RemoteException, ACIDException {
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
-        transactionProvider.getTransactionManager().commitTransaction(txnCtx);
+    public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
+        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
     }
 
     @Override
-    public void abortTransaction(long txnId) throws RemoteException, ACIDException {
+    public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
         try {
-            TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
-            transactionProvider.getTransactionManager().abortTransaction(txnCtx);
+            TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+            transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
         } catch (ACIDException e) {
             e.printStackTrace();
             throw e;
@@ -123,23 +129,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 = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        transactionSubsystem.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 = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        transactionSubsystem.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);
@@ -149,26 +155,27 @@
     }
 
     @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);
+                        dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
+
+                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);
@@ -178,11 +185,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) {
@@ -191,11 +198,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) {
@@ -204,11 +211,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);
@@ -218,11 +225,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) {
@@ -231,12 +238,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 function with this name " + function.getName() + " and arity "
@@ -246,65 +253,88 @@
         }
     }
 
-    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 metadataIndex, ITupleReference tuple)
+            throws Exception {
+        long resourceID = metadataIndex.getResourceID();
+        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+        indexLifecycleManager.open(resourceID);
+
+        //prepare a Callback for logging
+        IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
+                lsmIndex, IndexOperation.INSERT);
+
+        IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+
+        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        txnCtx.setTransactionType(TransactionType.READ_WRITE);
+
         // TODO: fix exceptions once new BTree exception model is in hyracks.
         indexAccessor.insert(tuple);
-        index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOp.INSERT, tuple);
+
+        indexLifecycleManager.close(resourceID);
+    }
+
+    private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
+            IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
+        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+
+        if (metadataIndex.isPrimaryIndex()) {
+            return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+                    metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+                    transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+        } else {
+            return new SecondaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
+                    metadataIndex.getPrimaryKeyIndexes(), txnCtx, transactionSubsystem.getLockManager(),
+                    transactionSubsystem, resourceId, ResourceType.LSM_BTREE, indexOp);
+        }
     }
 
     @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());
                 }
             }
 
             // As a side effect, acquires an S lock on the 'Function' dataset
             // on behalf of txnId.
-            List<Function> dataverseFunctions = getDataverseFunctions(txnId, dataverseName);
+            List<Function> dataverseFunctions = getDataverseFunctions(jobId, dataverseName);
             if (dataverseFunctions != null && dataverseFunctions.size() > 0) {
                 // Drop all functions in this dataverse.
                 for (Function function : dataverseFunctions) {
-                    dropFunction(txnId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
+                    dropFunction(jobId, new FunctionSignature(dataverseName, function.getName(), function.getArity()));
                 }
             }
 
             // As a side effect, acquires an S lock on the 'Adapter' dataset
             // on behalf of txnId.
-            List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(txnId, dataverseName);
+            List<DatasourceAdapter> dataverseAdapters = getDataverseAdapters(jobId, dataverseName);
             if (dataverseAdapters != null && dataverseAdapters.size() > 0) {
                 // Drop all functions in this dataverse.
                 for (DatasourceAdapter adapter : dataverseAdapters) {
-                    dropAdapter(txnId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
+                    dropAdapter(jobId, dataverseName, adapter.getAdapterIdentifier().getAdapterName());
                 }
             }
 
@@ -312,8 +342,9 @@
             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) {
@@ -324,11 +355,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);
         }
@@ -340,50 +371,63 @@
             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);
+            try {
+                ITupleReference datasetTuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATASET_DATASET,
+                        searchKey);
+                deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
+            } catch (TreeIndexException tie) {
+                //ignore this exception and continue deleting all relevant artifacts. 
+            }
+
             // 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,
-                        MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
-                deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
+                try {
+                    ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
+                            MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
+                    deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
+                } catch (TreeIndexException tie) {
+                    //ignore this exception and continue deleting all relevant artifacts.
+                }
             }
             // 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,
-                    MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
-            deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+            try {
+                ITupleReference dataTypeTuple = getTupleToBeDeleted(jobId,
+                        MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
+                deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+            } catch (TreeIndexException tie) {
+                //ignore this exception and continue deleting all relevant artifacts.
+            }
+
             // Delete entry(s) from the 'indexes' dataset.
             if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
-                List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
-                for (Index index : datasetIndexes) {
-                    dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
+                List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
+                if (datasetIndexes != null) {
+                    for (Index index : datasetIndexes) {
+                        dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
+                    }
                 }
             }
-            // TODO: Change this to be a BTree specific exception, e.g.,
-            // BTreeKeyDoesNotExistException.
-        } catch (TreeIndexException e) {
-            throw new MetadataException("Cannot drop dataset '" + datasetName + "' because it doesn't exist.", e);
         } catch (Exception e) {
             throw new MetadataException(e);
         }
     }
 
     @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) {
@@ -395,10 +439,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);
         }
@@ -414,8 +458,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) {
@@ -426,13 +470,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);
         }
@@ -457,15 +501,15 @@
             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);
+            ITupleReference tuple = getTupleToBeDeleted(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, searchKey);
             // This call uses the secondary index on datatype. Get nested types before deleting entry from secondary index.
-            List<String> nestedTypes = getNestedDatatypeNames(txnId, dataverseName, datatypeName);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
-            deleteFromDatatypeSecondaryIndex(txnId, dataverseName, datatypeName);
+            List<String> nestedTypes = getNestedDatatypeNames(jobId, dataverseName, datatypeName);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATATYPE_DATASET, tuple);
+            deleteFromDatatypeSecondaryIndex(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.,
@@ -477,14 +521,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) {
@@ -496,17 +540,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.
@@ -519,31 +563,49 @@
         }
     }
 
-    private void deleteTupleFromIndex(long txnId, IMetadataIndex index, ITupleReference tuple) throws Exception {
-        int fileId = index.getFileId();
-        BTree btree = (BTree) indexRegistry.get(fileId);
-        btree.open(fileId);
+    private void deleteTupleFromIndex(JobId jobId, IMetadataIndex metadataIndex, ITupleReference tuple)
+            throws Exception {
+        long resourceID = metadataIndex.getResourceID();
+        ILSMIndex lsmIndex = (ILSMIndex) indexLifecycleManager.getIndex(resourceID);
+        indexLifecycleManager.open(resourceID);
+        //prepare a Callback for logging
+        IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
+                lsmIndex, IndexOperation.DELETE);
+        IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-        ITreeIndexAccessor indexAccessor = btree.createAccessor();
-        TransactionContext txnCtx = transactionProvider.getTransactionManager().getTransactionContext(txnId);
-        // 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);
+        TransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        txnCtx.setTransactionType(TransactionType.READ_WRITE);
+
         indexAccessor.delete(tuple);
-        index.getTreeLogger().generateLogRecord(transactionProvider, txnCtx, IndexOp.DELETE, tuple);
+        indexLifecycleManager.close(resourceID);
     }
 
     @Override
-    public Dataverse getDataverse(long txnId, String dataverseName) throws MetadataException, RemoteException {
+    public List<Dataverse> getDataverses(JobId jobId) throws MetadataException, RemoteException {
+        try {
+            DataverseTupleTranslator tupleReaderWriter = new DataverseTupleTranslator(false);
+            IValueExtractor<Dataverse> valueExtractor = new MetadataEntityValueExtractor<Dataverse>(tupleReaderWriter);
+            List<Dataverse> results = new ArrayList<Dataverse>();
+            searchIndex(jobId, MetadataPrimaryIndexes.DATAVERSE_DATASET, null, valueExtractor, results);
+            if (results.isEmpty()) {
+                return null;
+            }
+            return results;
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+
+    }
+
+    @Override
+    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;
             }
@@ -555,28 +617,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);
@@ -584,14 +646,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;
             }
@@ -601,13 +663,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, datatypeName);
             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) {
@@ -615,13 +677,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) {
@@ -629,13 +691,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) {
@@ -643,13 +705,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);
@@ -657,14 +719,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;
             }
@@ -675,14 +737,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);
@@ -690,14 +752,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;
             }
@@ -708,13 +770,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;
             }
@@ -725,7 +787,7 @@
     }
 
     @Override
-    public Function getFunction(long txnId, FunctionSignature functionSignature) throws MetadataException,
+    public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
             RemoteException {
         try {
             ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName(), ""
@@ -733,7 +795,7 @@
             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;
             }
@@ -745,15 +807,27 @@
     }
 
     @Override
-    public void dropFunction(long txnId, FunctionSignature functionSignature) throws MetadataException, RemoteException {
+    public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
+            RemoteException {
+
+        Function function;
+        try {
+            function = getFunction(jobId, functionSignature);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+        if (function == null) {
+            throw new MetadataException("Cannot drop function '" + functionSignature.toString()
+                    + "' because it doesn't exist.");
+        }
         try {
             // Delete entry from the 'function' dataset.
             ITupleReference searchKey = createTuple(functionSignature.getNamespace(), functionSignature.getName());
             // 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.
@@ -765,11 +839,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.
@@ -779,29 +853,113 @@
         return results.get(0);
     }
 
-    private <ResultType> void searchIndex(long txnId, 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);
-        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);
-        IBinaryComparator[] searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
-        for (int i = 0; i < searchKey.getFieldCount(); i++) {
-            searchCmps[i] = comparatorFactories[i].createBinaryComparator();
+    //Debugging Method
+    public String printMetadata() {
+
+        StringBuilder sb = new StringBuilder();
+        try {
+            IMetadataIndex index = MetadataPrimaryIndexes.DATAVERSE_DATASET;
+            long resourceID = index.getResourceID();
+            IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+            indexLifecycleManager.open(resourceID);
+            IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+
+            RangePredicate rangePred = null;
+            rangePred = new RangePredicate(null, null, true, true, null, null);
+            indexAccessor.search(rangeCursor, rangePred);
+            try {
+                while (rangeCursor.hasNext()) {
+                    rangeCursor.next();
+                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(),
+                            new ISerializerDeserializer[] { AqlSerializerDeserializerProvider.INSTANCE
+                                    .getSerializerDeserializer(BuiltinType.ASTRING) }));
+                }
+            } finally {
+                rangeCursor.close();
+            }
+            indexLifecycleManager.close(resourceID);
+
+            index = MetadataPrimaryIndexes.DATASET_DATASET;
+            resourceID = index.getResourceID();
+            indexInstance = indexLifecycleManager.getIndex(resourceID);
+            indexLifecycleManager.open(resourceID);
+            indexAccessor = indexInstance
+                    .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+
+            rangePred = null;
+            rangePred = new RangePredicate(null, null, true, true, null, null);
+            indexAccessor.search(rangeCursor, rangePred);
+            try {
+                while (rangeCursor.hasNext()) {
+                    rangeCursor.next();
+                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+                            AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+                            AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+                }
+            } finally {
+                rangeCursor.close();
+            }
+            indexLifecycleManager.close(resourceID);
+
+            index = MetadataPrimaryIndexes.INDEX_DATASET;
+            resourceID = index.getResourceID();
+            indexInstance = indexLifecycleManager.getIndex(resourceID);
+            indexLifecycleManager.open(resourceID);
+            indexAccessor = indexInstance
+                    .createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+            rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+
+            rangePred = null;
+            rangePred = new RangePredicate(null, null, true, true, null, null);
+            indexAccessor.search(rangeCursor, rangePred);
+            try {
+                while (rangeCursor.hasNext()) {
+                    rangeCursor.next();
+                    sb.append(TupleUtils.printTuple(rangeCursor.getTuple(), new ISerializerDeserializer[] {
+                            AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+                            AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING),
+                            AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING) }));
+                }
+            } finally {
+                rangeCursor.close();
+            }
+            indexLifecycleManager.close(resourceID);
+        } catch (Exception e) {
+            e.printStackTrace();
         }
-        MultiComparator searchCmp = new MultiComparator(searchCmps);
-        RangePredicate rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
+        return sb.toString();
+    }
+
+    private <ResultType> void searchIndex(JobId jobId, IMetadataIndex index, ITupleReference searchKey,
+            IValueExtractor<ResultType> valueExtractor, List<ResultType> results) throws Exception {
+        IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
+        long resourceID = index.getResourceID();
+        IIndex indexInstance = indexLifecycleManager.getIndex(resourceID);
+        indexLifecycleManager.open(resourceID);
+        IIndexAccessor indexAccessor = indexInstance.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+        ITreeIndexCursor rangeCursor = (ITreeIndexCursor) indexAccessor.createSearchCursor();
+
+        IBinaryComparator[] searchCmps = null;
+        MultiComparator searchCmp = null;
+        RangePredicate rangePred = null;
+        if (searchKey != null) {
+            searchCmps = new IBinaryComparator[searchKey.getFieldCount()];
+            for (int i = 0; i < searchKey.getFieldCount(); i++) {
+                searchCmps[i] = comparatorFactories[i].createBinaryComparator();
+            }
+            searchCmp = new MultiComparator(searchCmps);
+        }
+        rangePred = new RangePredicate(searchKey, searchKey, true, true, searchCmp, searchCmp);
         indexAccessor.search(rangeCursor, rangePred);
 
         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);
                 }
@@ -809,6 +967,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
@@ -829,14 +1025,14 @@
     }
 
     @Override
-    public List<Function> getDataverseFunctions(long txnId, String dataverseName) throws MetadataException,
+    public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             FunctionTupleTranslator tupleReaderWriter = new FunctionTupleTranslator(false);
             IValueExtractor<Function> valueExtractor = new MetadataEntityValueExtractor<Function>(tupleReaderWriter);
             List<Function> results = new ArrayList<Function>();
-            searchIndex(txnId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.FUNCTION_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (Exception e) {
             throw new MetadataException(e);
@@ -844,12 +1040,12 @@
     }
 
     @Override
-    public void addAdapter(long txnId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
+    public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException {
         try {
             // Insert into the 'Adapter' dataset.
             DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(true);
             ITupleReference adapterTuple = tupleReaderWriter.getTupleFromMetadataEntity(adapter);
-            insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, adapterTuple);
 
         } catch (BTreeDuplicateKeyException e) {
             throw new MetadataException("A adapter with this name " + adapter.getAdapterIdentifier().getAdapterName()
@@ -861,11 +1057,11 @@
     }
 
     @Override
-    public void dropAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
+    public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
             RemoteException {
         DatasourceAdapter adapter;
         try {
-            adapter = getAdapter(txnId, dataverseName, adapterName);
+            adapter = getAdapter(jobId, dataverseName, adapterName);
         } catch (Exception e) {
             throw new MetadataException(e);
         }
@@ -877,8 +1073,9 @@
             ITupleReference searchKey = createTuple(dataverseName, adapterName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the 'Adapter' dataset.
-            ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
-            deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
+            ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
+                    MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey);
+            deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, datasetTuple);
 
             // TODO: Change this to be a BTree specific exception, e.g.,
             // BTreeKeyDoesNotExistException.
@@ -891,14 +1088,15 @@
     }
 
     @Override
-    public DatasourceAdapter getAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
-            RemoteException {
+    public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName)
+            throws MetadataException, RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName, adapterName);
             DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
             List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
-            IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(tupleReaderWriter);
-            searchIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+            IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+                    tupleReaderWriter);
+            searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
             if (results.isEmpty()) {
                 return null;
             }
@@ -909,18 +1107,18 @@
     }
 
     @Override
-    public List<DatasourceAdapter> getDataverseAdapters(long txnId, String dataverseName) throws MetadataException,
+    public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException {
         try {
             ITupleReference searchKey = createTuple(dataverseName);
             DatasourceAdapterTupleTranslator tupleReaderWriter = new DatasourceAdapterTupleTranslator(false);
-            IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(tupleReaderWriter);
+            IValueExtractor<DatasourceAdapter> valueExtractor = new MetadataEntityValueExtractor<DatasourceAdapter>(
+                    tupleReaderWriter);
             List<DatasourceAdapter> results = new ArrayList<DatasourceAdapter>();
-            searchIndex(txnId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
+            searchIndex(jobId, MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, searchKey, valueExtractor, results);
             return results;
         } catch (Exception e) {
             throw new MetadataException(e);
         }
     }
-
 }
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 66c02ee..cbd37d6 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
@@ -19,13 +19,15 @@
 
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
-import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.Function;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.om.functions.AsterixFunction;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
 
 /**
  * Used to implement serializable transactions against the MetadataCache.
@@ -57,14 +59,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) {
@@ -77,6 +79,11 @@
         logAndApply(new MetadataLogicalOperation(dataset, true));
     }
 
+    public void addIndex(Index index) {
+        droppedCache.dropIndex(index);
+        logAndApply(new MetadataLogicalOperation(index, true));
+    }
+
     public void addDatatype(Datatype datatype) {
         droppedCache.dropDatatype(datatype);
         logAndApply(new MetadataLogicalOperation(datatype, true));
@@ -97,18 +104,25 @@
         logAndApply(new MetadataLogicalOperation(adapter, true));
     }
 
-    public void dropDataverse(String dataverseName) {
-        Dataverse dataverse = new Dataverse(dataverseName, null);
-        droppedCache.addDataverseIfNotExists(dataverse);
-        logAndApply(new MetadataLogicalOperation(dataverse, false));
-    }
-
     public void dropDataset(String dataverseName, String datasetName) {
-        Dataset dataset = new Dataset(dataverseName, datasetName, null, null, null, null);
+        Dataset dataset = new Dataset(dataverseName, datasetName, null, null, null, null, -1,
+                IMetadataEntity.PENDING_NO_OP);
         droppedCache.addDatasetIfNotExists(dataset);
         logAndApply(new MetadataLogicalOperation(dataset, false));
     }
 
+    public void dropIndex(String dataverseName, String datasetName, String indexName) {
+        Index index = new Index(dataverseName, datasetName, indexName, null, null, false, IMetadataEntity.PENDING_NO_OP);
+        droppedCache.addIndexIfNotExists(index);
+        logAndApply(new MetadataLogicalOperation(index, false));
+    }
+
+    public void dropDataverse(String dataverseName) {
+        Dataverse dataverse = new Dataverse(dataverseName, null, IMetadataEntity.PENDING_NO_OP);
+        droppedCache.addDataverseIfNotExists(dataverse);
+        logAndApply(new MetadataLogicalOperation(dataverse, false));
+    }
+
     public void dropDataDatatype(String dataverseName, String datatypeName) {
         Datatype datatype = new Datatype(dataverseName, datatypeName, null, false);
         droppedCache.addDatatypeIfNotExists(datatype);
@@ -151,6 +165,16 @@
         return droppedCache.getDataset(dataverseName, datasetName) != null;
     }
 
+    public boolean indexIsDropped(String dataverseName, String datasetName, String indexName) {
+        if (droppedCache.getDataverse(dataverseName) != null) {
+            return true;
+        }
+        if (droppedCache.getDataset(dataverseName, datasetName) != null) {
+            return true;
+        }
+        return droppedCache.getIndex(dataverseName, datasetName, indexName) != null;
+    }
+
     public boolean datatypeIsDropped(String dataverseName, String datatypeName) {
         if (droppedCache.getDataverse(dataverseName) != null) {
             return true;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataEntity.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataEntity.java
index 847ec67..43f8dc4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataEntity.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataEntity.java
@@ -20,6 +20,11 @@
 import edu.uci.ics.asterix.metadata.MetadataCache;
 
 public interface IMetadataEntity extends Serializable {
+    
+    public static final int PENDING_NO_OP = 0;
+    public static final int PENDING_ADD_OP = 1;
+    public static final int PENDING_DROP_OP = 2;
+    
     Object addToCache(MetadataCache cache);
 
     Object dropFromCache(MetadataCache cache);
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..ca6ab08 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
@@ -19,12 +19,14 @@
 
 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.logging.IndexLogger;
+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.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
 /**
  * Descriptor interface for a primary or secondary index on metadata datasets.
@@ -46,6 +48,8 @@
 
     public ITypeTraits[] getTypeTraits();
 
+    public int[] getBloomFilterKeyFields();
+
     public RecordDescriptor getRecordDescriptor();
 
     public IBinaryComparatorFactory[] getKeyBinaryComparatorFactory();
@@ -58,13 +62,22 @@
 
     public ARecordType getPayloadRecordType();
 
-    public void setFileId(int fileId);
+    public void setFile(FileReference file);
 
-    public void initTreeLogger(ITreeIndex treeIndex) throws ACIDException;
+    public FileReference getFile();
+
+    public void setFileId(int fileId);
 
     public int getFileId();
 
-    public byte[] getResourceId();
+    public void setResourceID(long resourceID);
 
-    public TreeLogger getTreeLogger();
+    public long getResourceID();
+
+    public DatasetId getDatasetId();
+
+    boolean isPrimaryIndex();
+
+    int[] getPrimaryKeyIndexes();
+
 }
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 01e9649..fdf2d60 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
@@ -30,6 +30,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,
@@ -94,7 +95,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.
@@ -119,6 +120,16 @@
     public void addDataverse(MetadataTransactionContext ctx, Dataverse dataverse) throws MetadataException;
 
     /**
+     * Retrieves all dataverses
+     * 
+     * @param ctx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @return A list of dataverse instances.
+     * @throws MetadataException
+     */
+    List<Dataverse> getDataverses(MetadataTransactionContext ctx) throws MetadataException;
+    
+    /**
      * Retrieves a dataverse with given name.
      * 
      * @param ctx
@@ -168,7 +179,7 @@
      *             For example, if the dataset already exists.
      */
     public void addDataset(MetadataTransactionContext ctx, Dataset dataset) throws MetadataException;
-
+    
     /**
      * Retrieves a dataset within a given dataverse.
      * 
@@ -430,4 +441,16 @@
     public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
             throws MetadataException;
 
+	
+    public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
+    
+    public void acquireWriteLatch();
+
+    public void releaseWriteLatch();
+
+    public void acquireReadLatch();
+
+    public void releaseReadLatch();
+
+
 }
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 76c5746..836d42f 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
@@ -31,6 +31,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
@@ -49,7 +50,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.
@@ -57,7 +58,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.
@@ -65,7 +66,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
@@ -74,7 +75,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.
@@ -82,13 +83,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.
@@ -96,13 +97,26 @@
      *             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 all dataverses, acquiring local locks on behalf of
+     * the given transaction id.
+     * 
+     * @param jobId
+     *            A globally unique id for an active metadata transaction.
+     * @return A list of dataverse instances.
+     * @throws MetadataException
+     *             For example, if the dataverse does not exist.
+     * @throws RemoteException
+     */
+    public List<Dataverse> getDataverses(JobId jobId) 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.
@@ -111,13 +125,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.
@@ -125,7 +139,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;
 
     /**
@@ -133,20 +147,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.
@@ -154,13 +168,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.
@@ -171,14 +185,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.
@@ -189,14 +203,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.
@@ -206,7 +220,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;
 
     /**
@@ -214,7 +228,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.
@@ -222,13 +236,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.
@@ -240,14 +254,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.
@@ -258,14 +272,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.
@@ -273,13 +287,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.
@@ -290,14 +304,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.
@@ -308,14 +322,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.
@@ -323,13 +337,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.
@@ -337,13 +351,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.
@@ -352,13 +366,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.
@@ -366,10 +380,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 functionSignature
      *            An instance of functionSignature representing the function
@@ -377,14 +392,14 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public Function getFunction(long txnId, FunctionSignature functionSignature) throws MetadataException,
+    public Function getFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException,
             RemoteException;
 
     /**
      * 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 functionSignature
      *            An instance of functionSignature representing the function
@@ -393,10 +408,10 @@
      *             group to be deleted.
      * @throws RemoteException
      */
-    public void dropFunction(long txnId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
+    public void dropFunction(JobId jobId, FunctionSignature functionSignature) throws MetadataException, RemoteException;
 
     /**
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param function
      *            Function to be inserted
@@ -405,7 +420,7 @@
      *             unknown function
      * @throws RemoteException
      */
-    public void addFunction(long txnId, Function function) throws MetadataException, RemoteException;
+    public void addFunction(JobId jobId, Function function) throws MetadataException, RemoteException;
 
     /**
      * @param ctx
@@ -414,7 +429,7 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public List<Function> getDataverseFunctions(long txnId, String dataverseName) throws MetadataException,
+    public List<Function> getDataverseFunctions(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException;
 
     /**
@@ -424,17 +439,17 @@
      * @throws MetadataException
      * @throws RemoteException
      */
-    public List<DatasourceAdapter> getDataverseAdapters(long txnId, String dataverseName) throws MetadataException,
+    public List<DatasourceAdapter> getDataverseAdapters(JobId jobId, String dataverseName) throws MetadataException,
             RemoteException;
 
-    public DatasourceAdapter getAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
+    public DatasourceAdapter getAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
             RemoteException;
 
     /**
      * Deletes a adapter , 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 adapter that is to be deleted.
@@ -443,11 +458,11 @@
      *            if the adapter does not exists.
      * @throws RemoteException
      */
-    public void dropAdapter(long txnId, String dataverseName, String adapterName) throws MetadataException,
+    public void dropAdapter(JobId jobId, String dataverseName, String adapterName) throws MetadataException,
             RemoteException;
 
     /**
-     * @param txnId
+     * @param jobId
      *            A globally unique id for an active metadata transaction.
      * @param adapter
      *            Adapter to be inserted
@@ -455,6 +470,8 @@
      *             for example, if the adapter already exists.
      * @throws RemoteException
      */
-    public void addAdapter(long txnId, DatasourceAdapter adapter) throws MetadataException, RemoteException;
+    public void addAdapter(JobId jobId, DatasourceAdapter adapter) 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/AsterixProperties.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixProperties.java
index d0d04ee..71a7de3 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixProperties.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixProperties.java
@@ -15,6 +15,7 @@
 
 package edu.uci.ics.asterix.metadata.bootstrap;
 
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
@@ -96,6 +97,19 @@
                 String ncName = pn.substring(0, pn.indexOf('.'));
                 val = p.getProperty(pn);
                 String[] folderNames = val.split("\\s*,\\s*");
+                int i = 0;
+                for (String store : folderNames) {
+                    boolean needsStartSep = !store.startsWith(File.separator);
+                    boolean needsEndSep = !store.endsWith(File.separator);
+                    if (needsStartSep && needsEndSep) {
+                        folderNames[i] = File.separator + store + File.separator;
+                    } else if (needsStartSep) {
+                        folderNames[i] = File.separator + store;
+                    } else if (needsEndSep) {
+                        folderNames[i] = store + File.separator;
+                    }
+                    i++;
+                }
                 stores.put(ncName, folderNames);
                 nodeNames = new HashSet<String>();
                 nodeNames.addAll(stores.keySet());
@@ -127,7 +141,7 @@
         return nodeNames;
     }
 
-    public  String getOutputDir() {
+    public String getOutputDir() {
         return outputDir;
     }
 }
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 d7c23e0..805a550 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
@@ -16,6 +16,7 @@
 package edu.uci.ics.asterix.metadata.bootstrap;
 
 import java.io.File;
+import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -26,12 +27,16 @@
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.context.AsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 import edu.uci.ics.asterix.metadata.IDatasetDetails;
+import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.metadata.api.IMetadataIndex;
 import edu.uci.ics.asterix.metadata.entities.AsterixBuiltinTypeMap;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
@@ -48,30 +53,36 @@
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
-import edu.uci.ics.asterix.transaction.management.service.logging.DataUtil;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeResourceManager;
+import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
+import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
+import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceManagerRepository;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 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.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexFrameFactory;
+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.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.LocalResource;
+import edu.uci.ics.hyracks.storage.common.file.TransientFileMapManager;
 
 /**
  * Initializes the remote metadata storage facilities ("universe") using a
@@ -83,17 +94,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;
 
@@ -108,10 +125,9 @@
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
     }
 
-    public static void startUniverse(AsterixProperties asterixProperties, INCApplicationContext ncApplicationContext)
+    public static void startUniverse(AsterixProperties asterixProperties, INCApplicationContext ncApplicationContext, boolean isNewUniverse)
             throws Exception {
-        AsterixAppRuntimeContext runtimeContext = (AsterixAppRuntimeContext) ncApplicationContext
-                .getApplicationObject();
+        runtimeContext = (AsterixAppRuntimeContext) ncApplicationContext.getApplicationObject();
 
         // Initialize static metadata objects, such as record types and metadata
         // index descriptors.
@@ -122,14 +138,16 @@
         MetadataSecondaryIndexes.init();
         initLocalIndexArrays();
 
-        boolean isNewUniverse = true;
-        TransactionalResourceRepository resourceRepository = runtimeContext.getTransactionProvider()
+        TransactionalResourceManagerRepository resourceRepository = runtimeContext.getTransactionSubsystem()
                 .getTransactionalResourceRepository();
-        resourceRepository.registerTransactionalResourceManager(TreeResourceManager.ID, new TreeResourceManager(
-                runtimeContext.getTransactionProvider()));
+        resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_BTREE, new IndexResourceManager(
+                ResourceType.LSM_BTREE, runtimeContext.getTransactionSubsystem()));
+        resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_RTREE, new IndexResourceManager(
+                ResourceType.LSM_RTREE, runtimeContext.getTransactionSubsystem()));
+        resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_INVERTED_INDEX,
+                new IndexResourceManager(ResourceType.LSM_INVERTED_INDEX, runtimeContext.getTransactionSubsystem()));
 
         metadataNodeName = asterixProperties.getMetadataNodeName();
-        isNewUniverse = asterixProperties.isNewUniverse();
         metadataStore = asterixProperties.getMetadataStore();
         nodeNames = asterixProperties.getNodeNames();
         // nodeStores = asterixProperity.getStores();
@@ -139,32 +157,24 @@
             (new File(outputDir)).mkdirs();
         }
 
-        btreeRegistry = runtimeContext.getIndexRegistry();
+        indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
+        localResourceRepository = runtimeContext.getLocalResourceRepository();
         bufferCache = runtimeContext.getBufferCache();
         fileMapProvider = runtimeContext.getFileMapManager();
+        ioManager = ncApplicationContext.getRootContext().getIOManager();
 
-        // 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]);
-        }
+        if (isNewUniverse) {
+            MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            try {
+                // Begin a transaction against the metadata.
+                // Lock the metadata in X mode.
+                MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
 
-        // Begin a transaction against the metadata.
-        // Lock the metadata in X mode.
-        MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
-        MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.EXCLUSIVE);
-
-        try {
-            if (isNewUniverse) {
                 for (int i = 0; i < primaryIndexes.length; i++) {
-                    createIndex(primaryIndexes[i]);
-                    registerTransactionalResource(primaryIndexes[i], resourceRepository);
+                    enlistMetadataDataset(primaryIndexes[i], true);
                 }
                 for (int i = 0; i < secondaryIndexes.length; i++) {
-                    createIndex(secondaryIndexes[i]);
-                    registerTransactionalResource(secondaryIndexes[i], resourceRepository);
+                    enlistMetadataDataset(secondaryIndexes[i], true);
                 }
                 insertInitialDataverses(mdTxnCtx);
                 insertInitialDatasets(mdTxnCtx);
@@ -173,72 +183,47 @@
                 insertNodes(mdTxnCtx);
                 insertInitialGroups(mdTxnCtx);
                 insertInitialAdapters(mdTxnCtx);
-                LOGGER.info("FINISHED CREATING METADATA B-TREES.");
-            } else {
-                for (int i = 0; i < primaryIndexes.length; i++) {
-                    enlistMetadataDataset(primaryIndexes[i]);
-                    registerTransactionalResource(primaryIndexes[i], resourceRepository);
-                }
-                for (int i = 0; i < secondaryIndexes.length; i++) {
-                    enlistMetadataDataset(secondaryIndexes[i]);
-                    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);
+                throw e;
             }
-            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
-        } catch (Exception e) {
-            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
-            throw e;
+            LOGGER.info("FINISHED CREATING METADATA B-TREES.");
+        } else {
+            for (int i = 0; i < primaryIndexes.length; i++) {
+                enlistMetadataDataset(primaryIndexes[i], false);
+            }
+            for (int i = 0; i < secondaryIndexes.length; i++) {
+                enlistMetadataDataset(secondaryIndexes[i], false);
+            }
+            LOGGER.info("FINISHED ENLISTMENT OF METADATA B-TREES.");
         }
+
     }
 
     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);
-    }
-
-    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);
-        resourceRepository.registerTransactionalResource(resourceId, treeIndex);
-        index.initTreeLogger(treeIndex);
+        for (int i = 0; i < secondaryIndexes.length; i++) {
+            long resourceID = localResourceRepository.getResourceByName(
+                    secondaryIndexes[i].getFile().getFile().getPath()).getResourceId();
+            indexLifecycleManager.close(resourceID);
+            indexLifecycleManager.unregister(resourceID);
+        }
     }
 
     public static void insertInitialDataverses(MetadataTransactionContext mdTxnCtx) throws Exception {
         String dataverseName = MetadataPrimaryIndexes.DATAVERSE_DATASET.getDataverseName();
         String dataFormat = NonTaggedDataFormat.NON_TAGGED_DATA_FORMAT;
-        MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(dataverseName, dataFormat));
+        MetadataManager.INSTANCE.addDataverse(mdTxnCtx, new Dataverse(dataverseName, dataFormat,
+                IMetadataEntity.PENDING_NO_OP));
     }
 
     public static void insertInitialDatasets(MetadataTransactionContext mdTxnCtx) throws Exception {
@@ -248,7 +233,7 @@
                     primaryIndexes[i].getNodeGroupName());
             MetadataManager.INSTANCE.addDataset(mdTxnCtx, new Dataset(primaryIndexes[i].getDataverseName(),
                     primaryIndexes[i].getIndexedDatasetName(), primaryIndexes[i].getPayloadRecordType().getTypeName(),
-                    id, new HashMap<String,String>(), DatasetType.INTERNAL));
+                    id, new HashMap<String,String>(), DatasetType.INTERNAL, primaryIndexes[i].getDatasetId().getId(), IMetadataEntity.PENDING_NO_OP));
         }
     }
 
@@ -279,7 +264,7 @@
         for (int i = 0; i < secondaryIndexes.length; i++) {
             MetadataManager.INSTANCE.addIndex(mdTxnCtx, new Index(secondaryIndexes[i].getDataverseName(),
                     secondaryIndexes[i].getIndexedDatasetName(), secondaryIndexes[i].getIndexName(), IndexType.BTREE,
-                    secondaryIndexes[i].getPartitioningExpr(), false));
+                    secondaryIndexes[i].getPartitioningExpr(), false, IMetadataEntity.PENDING_NO_OP));
         }
     }
 
@@ -332,34 +317,54 @@
                 adapterFactoryClassName, DatasourceAdapter.AdapterType.INTERNAL);
     }
 
-    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();
+        int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
         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);
-    }
+        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(DEFAULT_MEM_NUM_PAGES,
+                metaDataFrameFactory);
+        LSMBTree lsmBtree = null;
+        long resourceID = -1;
+        if (create) {
+            lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
+                    bufferCache, fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
+                    runtimeContext.getLSMMergePolicy(), runtimeContext.getLSMBTreeOperationTrackerFactory(),
+                    runtimeContext.getLSMIOScheduler(), AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
+            lsmBtree.create();
+            resourceID = runtimeContext.getResourceIdFactory().createId();
+            indexLifecycleManager.register(resourceID, lsmBtree);
 
-    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);
+            ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
+                    comparatorFactories, bloomFilterKeyFields, index.isPrimaryIndex(),
+                    GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+            ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+                    localResourceMetadata, LocalResource.LSMBTreeResource);
+            ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
+            localResourceRepository.insert(localResourceFactory.createLocalResource(resourceID, file.getFile()
+                    .getPath(), 0));
+        } else {
+            resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
+            lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
+            if (lsmBtree == null) {
+                lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
+                        bufferCache, fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
+                        runtimeContext.getLSMMergePolicy(), runtimeContext.getLSMBTreeOperationTrackerFactory(),
+                        runtimeContext.getLSMIOScheduler(), AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
+                indexLifecycleManager.register(resourceID, lsmBtree);
+            }
+        }
+        
+        index.setResourceID(resourceID);
+        index.setFile(file);
+        indexLifecycleManager.open(resourceID);
     }
+    
+    
 
     public static String getOutputDir() {
         return outputDir;
@@ -369,4 +374,51 @@
         return metadataNodeName;
     }
 
+    public static void startDDLRecovery() throws RemoteException, ACIDException, MetadataException {
+        //#. clean up any record which has pendingAdd/DelOp flag 
+        //   as traversing all records from DATAVERSE_DATASET to DATASET_DATASET, and then to INDEX_DATASET.
+        String dataverseName = null;
+        String datasetName = null;
+        String indexName = null;
+        MetadataTransactionContext mdTxnCtx = null;
+        
+        MetadataManager.INSTANCE.acquireWriteLatch();
+        
+        try {
+            mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+
+            List<Dataverse> dataverses = MetadataManager.INSTANCE.getDataverses(mdTxnCtx);
+            for (Dataverse dataverse : dataverses) {
+                dataverseName = dataverse.getDataverseName();
+                if (dataverse.getPendingOp() != IMetadataEntity.PENDING_NO_OP) {
+                    //drop pending dataverse
+                    MetadataManager.INSTANCE.dropDataverse(mdTxnCtx, dataverseName);
+                } else {
+                    List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dataverseName);
+                    for (Dataset dataset : datasets) {
+                        datasetName = dataset.getDatasetName();
+                        if (dataset.getPendingOp() != IMetadataEntity.PENDING_NO_OP) {
+                            //drop pending dataset
+                            MetadataManager.INSTANCE.dropDataset(mdTxnCtx, dataverseName, datasetName);
+                        } else {
+                            List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
+                                    datasetName);
+                            for (Index index : indexes) {
+                                indexName = index.getIndexName();
+                                if (index.getPendingOp() != IMetadataEntity.PENDING_NO_OP) {
+                                    //drop pending index
+                                    MetadataManager.INSTANCE.dropIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        } catch (Exception e) {
+            MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
+            throw new MetadataException(e);
+        } finally {
+            MetadataManager.INSTANCE.releaseWriteLatch();
+        }
+    }
 }
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 5fcac2e..c0ce030 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
@@ -28,14 +28,16 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 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.asterix.transaction.management.service.logging.IndexLogger;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
 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.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 
 /**
  * Descriptor for a primary or secondary index on metadata datasets.
@@ -51,6 +53,8 @@
     protected final String[] keyNames;
     // Field permutation for BTree insert. Auto-created based on numFields.
     protected final int[] fieldPermutation;
+    // Key Fields that will be used for the bloom filters in the LSM-btree.
+    protected final int[] bloomFilterKeyFields;
     // Type of payload record for primary indexes. null for secondary indexes.
     protected final ARecordType payloadType;
     // Record descriptor of btree tuple. Created in c'tor.
@@ -61,15 +65,22 @@
     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;
     // Resource id of this index for use in transactions.
-    protected byte[] indexResourceId;
-    // Logger for tree indexes.
-    private TreeLogger treeLogger;
+    protected long resourceId;
+    // datasetId
+    private final DatasetId datasetId;
+    // Flag of primary index
+    protected final boolean isPrimaryIndex;
+    // PrimaryKeyField indexes used for secondary index operations
+    protected final int[] primaryKeyIndexes;
 
     public MetadataIndex(String datasetName, String indexName, int numFields, IAType[] keyTypes, String[] keyNames,
-            ARecordType payloadType) throws AsterixRuntimeException {
+            int numSecondaryIndexKeys, ARecordType payloadType, int datasetId, boolean isPrimaryIndex,
+            int[] primaryKeyIndexes) throws AsterixRuntimeException {
         // Sanity checks.
         if (keyTypes.length != keyNames.length) {
             throw new AsterixRuntimeException("Unequal number of key types and names given.");
@@ -115,14 +126,31 @@
         // 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]);
         }
+
+        if (isPrimaryIndex) {
+            bloomFilterKeyFields = new int[primaryKeyIndexes.length];
+            for (int i = 0; i < primaryKeyIndexes.length; ++i) {
+                bloomFilterKeyFields[i] = primaryKeyIndexes[i];
+            }
+        } else {
+            bloomFilterKeyFields = new int[numSecondaryIndexKeys];
+            for (int i = 0; i < numSecondaryIndexKeys; ++i) {
+                bloomFilterKeyFields[i] = i;
+            }
+        }
+
+        this.datasetId = new DatasetId(datasetId);
+        this.isPrimaryIndex = isPrimaryIndex;
+
+        //PrimaryKeyFieldIndexes
+        this.primaryKeyIndexes = primaryKeyIndexes;
     }
 
     @Override
@@ -136,6 +164,11 @@
     }
 
     @Override
+    public int[] getBloomFilterKeyFields() {
+        return bloomFilterKeyFields;
+    }
+
+    @Override
     public int getKeyFieldCount() {
         return keyTypes.length;
     }
@@ -197,12 +230,6 @@
     @Override
     public void setFileId(int fileId) {
         this.fileId = fileId;
-        this.indexResourceId = DataUtil.intToByteArray(fileId);
-    }
-
-    @Override
-    public void initTreeLogger(ITreeIndex treeIndex) throws ACIDException {
-        this.treeLogger = new TreeLogger(indexResourceId, treeIndex);
     }
 
     @Override
@@ -216,11 +243,37 @@
     }
 
     @Override
-    public byte[] getResourceId() {
-        return indexResourceId;
+    public void setFile(FileReference file) {
+        this.file = file;
     }
 
-    public TreeLogger getTreeLogger() {
-        return treeLogger;
+    @Override
+    public FileReference getFile() {
+        return this.file;
+    }
+
+    @Override
+    public void setResourceID(long resourceID) {
+        this.resourceId = resourceID;
+    }
+
+    @Override
+    public long getResourceID() {
+        return resourceId;
+    }
+
+    @Override
+    public DatasetId getDatasetId() {
+        return datasetId;
+    }
+
+    @Override
+    public boolean isPrimaryIndex() {
+        return isPrimaryIndex;
+    }
+
+    @Override
+    public int[] getPrimaryKeyIndexes() {
+        return primaryKeyIndexes;
     }
 }
\ 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 b1f6796..2fc3e59 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,18 @@
     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 DATASOURCE_ADAPTER_DATASET_ID = 8;
+    public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
+
     public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
 
     /**
@@ -49,33 +61,37 @@
         }
 
         DATAVERSE_DATASET = new MetadataIndex("Dataverse", null, 2, new IAType[] { BuiltinType.ASTRING },
-                new String[] { "DataverseName" }, MetadataRecordTypes.DATAVERSE_RECORDTYPE);
+                new String[] { "DataverseName" }, 0, MetadataRecordTypes.DATAVERSE_RECORDTYPE, DATAVERSE_DATASET_ID,
+                true, new int[] { 0 });
 
         DATASET_DATASET = new MetadataIndex("Dataset", null, 3,
                 new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName",
-                        "DatasetName" }, MetadataRecordTypes.DATASET_RECORDTYPE);
+                        "DatasetName" }, 0, MetadataRecordTypes.DATASET_RECORDTYPE, DATASET_DATASET_ID, true,
+                new int[] { 0, 1 });
 
         DATATYPE_DATASET = new MetadataIndex("Datatype", null, 3, new IAType[] { BuiltinType.ASTRING,
-                BuiltinType.ASTRING }, new String[] { "DataverseName", "DatatypeName" },
-                MetadataRecordTypes.DATATYPE_RECORDTYPE);
+                BuiltinType.ASTRING }, new String[] { "DataverseName", "DatatypeName" }, 0,
+                MetadataRecordTypes.DATATYPE_RECORDTYPE, DATATYPE_DATASET_ID, true, new int[] { 0, 1 });
 
         INDEX_DATASET = new MetadataIndex("Index", null, 4, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING,
-                BuiltinType.ASTRING }, new String[] { "DataverseName", "DatasetName", "IndexName" },
-                MetadataRecordTypes.INDEX_RECORDTYPE);
+                BuiltinType.ASTRING }, new String[] { "DataverseName", "DatasetName", "IndexName" }, 0,
+                MetadataRecordTypes.INDEX_RECORDTYPE, INDEX_DATASET_ID, true, new int[] { 0, 1, 2 });
 
         NODE_DATASET = new MetadataIndex("Node", null, 2, new IAType[] { BuiltinType.ASTRING },
-                new String[] { "NodeName" }, MetadataRecordTypes.NODE_RECORDTYPE);
+                new String[] { "NodeName" }, 0, MetadataRecordTypes.NODE_RECORDTYPE, NODE_DATASET_ID, true,
+                new int[] { 0 });
 
         NODEGROUP_DATASET = new MetadataIndex("Nodegroup", null, 2, new IAType[] { BuiltinType.ASTRING },
-                new String[] { "GroupName" }, MetadataRecordTypes.NODEGROUP_RECORDTYPE);
+                new String[] { "GroupName" }, 0, MetadataRecordTypes.NODEGROUP_RECORDTYPE, NODEGROUP_DATASET_ID, true,
+                new int[] { 0 });
 
         FUNCTION_DATASET = new MetadataIndex("Function", null, 4, new IAType[] { BuiltinType.ASTRING,
-                BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name", "Arity" },
-                MetadataRecordTypes.FUNCTION_RECORDTYPE);
+                BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name", "Arity" }, 0,
+                MetadataRecordTypes.FUNCTION_RECORDTYPE, FUNCTION_DATASET_ID, true, new int[] { 0, 1, 2 });
 
-        DATASOURCE_ADAPTER_DATASET = new MetadataIndex("DatasourceAdapter", null, 3,
-                new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" },
-                MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE);
-
+        DATASOURCE_ADAPTER_DATASET = new MetadataIndex("DatasourceAdapter", null, 3, new IAType[] {
+                BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" }, 0,
+                MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, DATASOURCE_ADAPTER_DATASET_ID, true, new int[] { 0,
+                        1 });
     }
 }
\ 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 679c2db..50681ee 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
@@ -86,10 +86,12 @@
     public static final int DATAVERSE_ARECORD_NAME_FIELD_INDEX = 0;
     public static final int DATAVERSE_ARECORD_FORMAT_FIELD_INDEX = 1;
     public static final int DATAVERSE_ARECORD_TIMESTAMP_FIELD_INDEX = 2;
+    public static final int DATAVERSE_ARECORD_PENDINGOP_FIELD_INDEX = 3;
 
     private static final ARecordType createDataverseRecordType() throws AsterixException {
-        return new ARecordType("DataverseRecordType", new String[] { "DataverseName", "DataFormat", "Timestamp" },
-                new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING }, true);
+        return new ARecordType("DataverseRecordType", new String[] { "DataverseName", "DataFormat", "Timestamp",
+                "PendingOp" }, new IAType[] { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
+                BuiltinType.AINT32 }, true);
     }
 
     // Helper constants for accessing fields in an ARecord of anonymous type
@@ -161,8 +163,6 @@
         return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
-    // 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;
@@ -172,10 +172,12 @@
     public static final int DATASET_ARECORD_FEEDDETAILS_FIELD_INDEX = 6;
     public static final int DATASET_ARECORD_HINTS_FIELD_INDEX = 7;
     public static final int DATASET_ARECORD_TIMESTAMP_FIELD_INDEX = 8;
+    public static final int DATASET_ARECORD_DATASETID_FIELD_INDEX = 9;
+    public static final int DATASET_ARECORD_PENDINGOP_FIELD_INDEX = 10;
 
     private static final ARecordType createDatasetRecordType() throws AsterixException {
         String[] fieldNames = { "DataverseName", "DatasetName", "DataTypeName", "DatasetType", "InternalDetails",
-                "ExternalDetails", "FeedDetails", "Hints", "Timestamp" };
+                "ExternalDetails", "FeedDetails", "Hints", "Timestamp", "DatasetId", "PendingOp" };
 
         List<IAType> internalRecordUnionList = new ArrayList<IAType>();
         internalRecordUnionList.add(BuiltinType.ANULL);
@@ -196,7 +198,7 @@
 
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
                 internalRecordUnion, externalRecordUnion, feedRecordUnion, unorderedListOfHintsType,
-                BuiltinType.ASTRING };
+                BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.AINT32 };
         return new ARecordType("DatasetRecordType", fieldNames, fieldTypes, true);
     }
 
@@ -281,13 +283,14 @@
     public static final int INDEX_ARECORD_SEARCHKEY_FIELD_INDEX = 4;
     public static final int INDEX_ARECORD_ISPRIMARY_FIELD_INDEX = 5;
     public static final int INDEX_ARECORD_TIMESTAMP_FIELD_INDEX = 6;
+    public static final int INDEX_ARECORD_PENDINGOP_FIELD_INDEX = 7;
 
     private static final ARecordType createIndexRecordType() throws AsterixException {
         AOrderedListType olType = new AOrderedListType(BuiltinType.ASTRING, null);
         String[] fieldNames = { "DataverseName", "DatasetName", "IndexName", "IndexStructure", "SearchKey",
-                "IsPrimary", "Timestamp" };
+                "IsPrimary", "Timestamp", "PendingOp" };
         IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.ASTRING,
-                olType, BuiltinType.ABOOLEAN, BuiltinType.ASTRING };
+                olType, BuiltinType.ABOOLEAN, BuiltinType.ASTRING, BuiltinType.AINT32 };
         return new ARecordType("IndexRecordType", 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..a0da856 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,17 @@
 
         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" }, 1, null,
+                MetadataPrimaryIndexes.DATASET_DATASET_ID, false, new int[] { 1, 2 });
 
         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" }, 2, null, MetadataPrimaryIndexes.DATASET_DATASET_ID, false, new int[] {
+                0, 2 });
 
         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" }, 2, null, MetadataPrimaryIndexes.DATATYPE_DATASET_ID, false,
+                new int[] { 0, 2 });
     }
 }
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DatasetHints.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
similarity index 97%
rename from asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DatasetHints.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
index 0b66c30..149d447 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DatasetHints.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/DatasetHints.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.translator;
+package edu.uci.ics.asterix.metadata.dataset.hints;
 
 import java.util.HashSet;
 import java.util.Set;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/IHint.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
similarity index 95%
rename from asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/IHint.java
rename to asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
index 1c9fd7b..8957ca7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/IHint.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/dataset/hints/IHint.java
@@ -12,7 +12,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package edu.uci.ics.asterix.translator;
+package edu.uci.ics.asterix.metadata.dataset.hints;
 
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 
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
new file mode 100644
index 0000000..35a9b83
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -0,0 +1,272 @@
+/*
+ * Copyright 2009-2010 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.metadata.declared;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.annotations.TypeDataGen;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.base.IDataFormat;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.api.IMetadataManager;
+import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Datatype;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.NodeGroup;
+import edu.uci.ics.asterix.om.types.IAType;
+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;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
+import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+
+public class AqlCompiledMetadataDeclarations {
+    private static Logger LOGGER = Logger.getLogger(AqlCompiledMetadataDeclarations.class.getName());
+
+    // We are assuming that there is a one AqlCompiledMetadataDeclarations per
+    // transaction.
+    private final MetadataTransactionContext mdTxnCtx;
+    private String dataverseName = null;
+    private FileSplit outputFile;
+    private Map<String, String[]> stores;
+    private IDataFormat format;
+    private Map<String, String> config;
+
+    private final Map<String, IAType> types;
+    private final Map<String, TypeDataGen> typeDataGenMap;
+    private final IAWriterFactory writerFactory;
+
+    private IMetadataManager metadataManager = MetadataManager.INSTANCE;
+    private boolean isConnected = false;
+
+    public AqlCompiledMetadataDeclarations(MetadataTransactionContext mdTxnCtx, String dataverseName,
+            FileSplit outputFile, Map<String, String> config, Map<String, String[]> stores, Map<String, IAType> types,
+            Map<String, TypeDataGen> typeDataGenMap, IAWriterFactory writerFactory, boolean online) {
+        this.mdTxnCtx = mdTxnCtx;
+        this.dataverseName = dataverseName;
+        this.outputFile = outputFile;
+        this.config = config;
+        if (stores == null && online) {
+            this.stores = AsterixProperties.INSTANCE.getStores();
+        } else {
+            this.stores = stores;
+        }
+        this.types = types;
+        this.typeDataGenMap = typeDataGenMap;
+        this.writerFactory = writerFactory;
+    }
+
+    public void connectToDataverse(String dvName) throws AlgebricksException, AsterixException {
+        if (isConnected) {
+            throw new AlgebricksException("You are already connected to " + dataverseName + " dataverse");
+        }
+        Dataverse dv;
+        try {
+            dv = metadataManager.getDataverse(mdTxnCtx, dvName);
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+        if (dv == null) {
+            throw new AlgebricksException("There is no dataverse with this name " + dvName + " to connect to.");
+        }
+        dataverseName = dvName;
+        isConnected = true;
+        try {
+            format = (IDataFormat) Class.forName(dv.getDataFormat()).newInstance();
+        } catch (Exception e) {
+            throw new AsterixException(e);
+        }
+    }
+
+    public void disconnectFromDataverse() throws AlgebricksException {
+        if (!isConnected) {
+            throw new AlgebricksException("You are not connected to any dataverse");
+        }
+        dataverseName = null;
+        format = null;
+        isConnected = false;
+    }
+
+    public boolean isConnectedToDataverse() {
+        return isConnected;
+    }
+
+    public String getDataverseName() {
+        return dataverseName;
+    }
+
+    public FileSplit getOutputFile() {
+        return outputFile;
+    }
+
+    public IDataFormat getFormat() throws AlgebricksException {
+        if (!isConnected) {
+            throw new AlgebricksException("You need first to connect to a dataverse.");
+        }
+        return format;
+    }
+
+    public String getPropertyValue(String propertyName) {
+        return config.get(propertyName);
+    }
+
+    public IAType findType(String typeName) {
+        Datatype type;
+        try {
+            type = metadataManager.getDatatype(mdTxnCtx, dataverseName, typeName);
+        } catch (Exception e) {
+            throw new IllegalStateException();
+        }
+        if (type == null) {
+            throw new IllegalStateException();
+        }
+        return type.getDatatype();
+    }
+
+    public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException {
+        NodeGroup ng;
+        try {
+            ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+        if (ng == null) {
+            throw new AlgebricksException("No node group with this name " + nodeGroupName);
+        }
+        return ng.getNodeNames();
+    }
+
+    public Map<String, String[]> getAllStores() {
+        return stores;
+    }
+
+    public Dataset findDataset(String datasetName) throws AlgebricksException {
+        try {
+            return metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
+        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);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    public Index getIndex(String dataverseName, String datasetName, String indexName) throws AlgebricksException {
+        try {
+            return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    public void setOutputFile(FileSplit outputFile) {
+        this.outputFile = outputFile;
+    }
+
+    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+            String datasetName, String targetIdxName) throws AlgebricksException {
+        FileSplit[] splits = splitsForInternalOrFeedDataset(datasetName, targetIdxName);
+        IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
+        String[] loc = new String[splits.length];
+        for (int p = 0; p < splits.length; p++) {
+            loc[p] = splits[p].getNodeName();
+        }
+        AlgebricksPartitionConstraint pc = new AlgebricksAbsolutePartitionConstraint(loc);
+        return new Pair<IFileSplitProvider, AlgebricksPartitionConstraint>(splitProvider, pc);
+    }
+
+    private FileSplit[] splitsForInternalOrFeedDataset(String datasetName, String targetIdxName)
+            throws AlgebricksException {
+
+        File relPathFile = new File(getRelativePath(datasetName + "_idx_" + targetIdxName));
+        Dataset dataset = findDataset(datasetName);
+        if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
+            throw new AlgebricksException("Not an internal or feed dataset");
+        }
+        InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+        List<String> nodeGroup = findNodeGroupNodeNames(datasetDetails.getNodeGroupName());
+        if (nodeGroup == null) {
+            throw new AlgebricksException("Couldn't find node group " + datasetDetails.getNodeGroupName());
+        }
+
+        List<FileSplit> splitArray = new ArrayList<FileSplit>();
+        for (String nd : nodeGroup) {
+            String[] nodeStores = stores.get(nd);
+            if (nodeStores == null) {
+                LOGGER.warning("Node " + nd + " has no stores.");
+                throw new AlgebricksException("Node " + nd + " has no stores.");
+            } else {
+                for (int j = 0; j < nodeStores.length; j++) {
+                    File f = new File(nodeStores[j] + File.separator + relPathFile);
+                    splitArray.add(new FileSplit(nd, new FileReference(f)));
+                }
+            }
+        }
+        FileSplit[] splits = new FileSplit[splitArray.size()];
+        int i = 0;
+        for (FileSplit fs : splitArray) {
+            splits[i++] = fs;
+        }
+        return splits;
+    }
+
+    public String getRelativePath(String fileName) {
+        return dataverseName + File.separator + fileName;
+    }
+
+    public Map<String, TypeDataGen> getTypeDataGenMap() {
+        return typeDataGenMap;
+    }
+
+    public Map<String, IAType> getTypeDeclarations() {
+        return types;
+    }
+
+    public IAWriterFactory getWriterFactory() {
+        return writerFactory;
+    }
+
+    public MetadataTransactionContext getMetadataTransactionContext() {
+        return mdTxnCtx;
+    }
+}
\ No newline at end of file
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataImplConfig.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataImplConfig.java
new file mode 100644
index 0000000..4ea752b
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataImplConfig.java
@@ -0,0 +1,28 @@
+/*
+ * Copyright 2009-2013 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.metadata.declared;
+
+public class AqlMetadataImplConfig {
+    private final boolean useInstantLock;
+
+    public AqlMetadataImplConfig(boolean useInstantLock) {
+        this.useInstantLock = useInstantLock;
+    }
+
+    public boolean isInstantLock() {
+        return useInstantLock;
+    }
+}
\ No newline at end of file
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 ee043b3..e1f707c 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
@@ -24,7 +24,10 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.context.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
 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;
@@ -46,6 +49,7 @@
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+import edu.uci.ics.asterix.metadata.dataset.hints.DatasetHints.DatasetCardinalityHint;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
@@ -63,9 +67,15 @@
 import edu.uci.ics.asterix.runtime.base.AsterixTupleFilterFactory;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
-import edu.uci.ics.asterix.runtime.transaction.TreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexSearchOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexModificationOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
+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.AlgebricksCountPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
@@ -90,6 +100,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.IResultSerializerFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
@@ -97,29 +108,38 @@
 import edu.uci.ics.hyracks.api.dataset.ResultSetId;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
+import edu.uci.ics.hyracks.data.std.primitive.ShortPointable;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.FileScanOperatorDescriptor;
 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.dataflow.std.result.ResultWriterOperatorDescriptor;
-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.ISearchOperationCallbackFactory;
 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.impls.NoOpOperationCallbackProvider;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
+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.common.dataflow.LSMTreeIndexInsertUpdateDeleteOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexDataflowHelperFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.dataflow.LSMInvertedIndexInsertUpdateDeleteOperator;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+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 static Logger LOGGER = Logger.getLogger(AqlMetadataProvider.class.getName());
-
-    private final MetadataTransactionContext mdTxnCtx;
+    private MetadataTransactionContext mdTxnCtx;
     private boolean isWriteTransaction;
     private Map<String, String[]> stores;
     private Map<String, String> config;
@@ -127,9 +147,9 @@
     private FileSplit outputFile;
     private ResultSetId resultSetId;
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
-    private long jobTxnId;
 
     private final Dataverse defaultDataverse;
+    private JobId jobId;
 
     private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
 
@@ -149,14 +169,13 @@
         return config;
     }
 
-    public AqlMetadataProvider(MetadataTransactionContext mdTxnCtx, Dataverse defaultDataverse) {
-        this.mdTxnCtx = mdTxnCtx;
+    public AqlMetadataProvider(Dataverse defaultDataverse) {
         this.defaultDataverse = defaultDataverse;
         this.stores = AsterixProperties.INSTANCE.getStores();
     }
 
-    public void setJobTxnId(long txnId) {
-        this.jobTxnId = txnId;
+    public void setJobId(JobId jobId) {
+        this.jobId = jobId;
     }
 
     public Dataverse getDefaultDataverse() {
@@ -175,6 +194,10 @@
         this.writerFactory = writerFactory;
     }
 
+    public void setMetadataTxnContext(MetadataTransactionContext mdTxnCtx) {
+        this.mdTxnCtx = mdTxnCtx;
+    }
+
     public MetadataTransactionContext getMetadataTxnContext() {
         return mdTxnCtx;
     }
@@ -225,7 +248,7 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getScannerRuntime(
             IDataSource<AqlSourceId> dataSource, List<LogicalVariable> scanVariables,
             List<LogicalVariable> projectVariables, boolean projectPushed, IOperatorSchema opSchema,
-            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec)
+            IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec, Object implConfig)
             throws AlgebricksException {
         Dataset dataset;
         try {
@@ -242,12 +265,12 @@
                         return buildExternalDatasetScan(jobSpec, dataset, dataSource);
                     } else {
                         return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
-                                context);
+                                context, implConfig);
 
                     }
                 case INTERNAL: {
                     return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
-                            context);
+                            context, implConfig);
                 }
                 case EXTERNAL: {
                     return buildExternalDatasetScan(jobSpec, dataset, dataSource);
@@ -263,14 +286,14 @@
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            Dataset dataset, IDataSource<AqlSourceId> dataSource, JobGenContext context) throws AlgebricksException,
-            MetadataException {
+            Dataset dataset, IDataSource<AqlSourceId> dataSource, JobGenContext context, Object implConfig)
+            throws AlgebricksException, MetadataException {
         AqlSourceId asid = dataSource.getId();
         String dataverseName = asid.getDataverseName();
         String datasetName = asid.getDatasetName();
         Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
         return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, context, false, dataset,
-                primaryIndex.getIndexName(), null, null, true, true);
+                primaryIndex.getIndexName(), null, null, true, true, implConfig);
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
@@ -445,7 +468,8 @@
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] lowKeyFields,
-            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive) throws AlgebricksException {
+            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive, Object implConfig)
+            throws AlgebricksException {
         boolean isSecondary = true;
         try {
             Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
@@ -457,23 +481,29 @@
             RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
             int numKeys = numPrimaryKeys;
             int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
+            ITypeTraits[] typeTraits = null;
+            int[] bloomFilterKeyFields;
             if (isSecondary) {
                 Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
                         dataset.getDatasetName(), indexName);
                 int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
                 numKeys += numSecondaryKeys;
                 keysStartIndex = outputVars.size() - numKeys;
-            }
-            IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
-                    outputVars, keysStartIndex, numKeys, typeEnv, context);
-            ITypeTraits[] typeTraits = null;
-
-            if (isSecondary) {
                 typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
+                bloomFilterKeyFields = new int[numSecondaryKeys];
+                for (int i = 0; i < numSecondaryKeys; i++) {
+                    bloomFilterKeyFields[i] = i;
+                }
             } else {
                 typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
                         context);
+                bloomFilterKeyFields = new int[numPrimaryKeys];
+                for (int i = 0; i < numPrimaryKeys; i++) {
+                    bloomFilterKeyFields[i] = i;
+                }
             }
+            IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+                    outputVars, keysStartIndex, numKeys, typeEnv, context);
 
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
@@ -483,11 +513,40 @@
             } catch (Exception e) {
                 throw new AlgebricksException(e);
             }
+
+            ISearchOperationCallbackFactory searchCallbackFactory = null;
+            if (isSecondary) {
+                searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
+            } else {
+                JobId jobId = ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId();
+                int datasetId = dataset.getDatasetId();
+                int[] primaryKeyFields = new int[numPrimaryKeys];
+                for (int i = 0; i < numPrimaryKeys; i++) {
+                    primaryKeyFields[i] = i;
+                }
+
+                AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
+                TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+                if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
+                    searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
+                            primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+                } else {
+                    searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
+                            primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+                }
+            }
             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, bloomFilterKeyFields, lowKeyFields, highKeyFields,
+                    lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
+                    retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+
         } catch (MetadataException me) {
             throw new AlgebricksException(me);
         }
@@ -539,11 +598,24 @@
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+
+            IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+                    dataset, recType, context.getBinaryComparatorFactoryProvider());
+            IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+            ISearchOperationCallbackFactory searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
             RTreeSearchOperatorDescriptor rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
-                    appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first,
-                    typeTraits, comparatorFactories, keyFields, new RTreeDataflowHelperFactory(valueProviderFactories),
-                    retainInput, NoOpOperationCallbackProvider.INSTANCE);
+                    appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+                    typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(
+                            valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+                                    nestedKeyType.getTypeTag(), comparatorFactories.length),
+                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
+                    retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
+
         } catch (MetadataException me) {
             throw new AlgebricksException(me);
         }
@@ -651,11 +723,13 @@
         int numKeys = keys.size();
         // move key fields to front
         int[] fieldPermutation = new int[numKeys + 1];
+        int[] bloomFilterKeyFields = new int[numKeys];
         // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
         int i = 0;
         for (LogicalVariable varKey : keys) {
             int idx = propagatedSchema.findVariable(varKey);
             fieldPermutation[i] = idx;
+            bloomFilterKeyFields[i] = i;
             i++;
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
@@ -673,7 +747,6 @@
             String itemTypeName = dataset.getItemTypeName();
             ARecordType itemType = (ARecordType) MetadataManager.INSTANCE.getDatatype(mdTxnCtx,
                     dataset.getDataverseName(), itemTypeName).getDatatype();
-
             ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
             IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
                     itemType, context.getBinaryComparatorFactoryProvider());
@@ -681,31 +754,49 @@
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataSource.getId().getDataverseName(), datasetName, indexName);
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+
+            String numElementsHintString = dataset.getHints().get("CARDINALITY");
+            long numElementsHint;
+            if (numElementsHintString == null) {
+                numElementsHint = DatasetCardinalityHint.DEFAULT;
+            } else {
+                numElementsHint = Long.parseLong(dataset.getHints().get("CARDINALITY"));
+            }
+
+            //TODO
+            //figure out the right behavior of the bulkload and then give the right callback
+            //(ex. what's the expected behavior when there is an error during bulkload?)
             TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                    appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                    splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
-                    GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, new BTreeDataflowHelperFactory(),
-                    NoOpOperationCallbackProvider.INSTANCE);
+                    appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(),
+                    splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
+                    GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, new LSMBTreeDataflowHelperFactory(
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
+                    NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
-
         } catch (MetadataException me) {
             throw new AlgebricksException(me);
         }
     }
 
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOp indexOp,
-            IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
-            LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
-            throws AlgebricksException {
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOperation indexOp,
+            IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+            List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification spec) throws AlgebricksException {
         String datasetName = dataSource.getId().getDatasetName();
         int numKeys = keys.size();
         // Move key fields to front.
         int[] fieldPermutation = new int[numKeys + 1];
+        int[] bloomFilterKeyFields = new int[numKeys];
         int i = 0;
         for (LogicalVariable varKey : keys) {
             int idx = propagatedSchema.findVariable(varKey);
             fieldPermutation[i] = idx;
+            bloomFilterKeyFields[i] = i;
             i++;
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
@@ -731,12 +822,32 @@
                     itemType, context.getBinaryComparatorFactoryProvider());
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataSource.getId().getDataverseName(), datasetName, indexName);
-            TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                    spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                    splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
-                    new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, jobTxnId);
+
+            //prepare callback
+            JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+            int datasetId = dataset.getDatasetId();
+            int[] primaryKeyFields = new int[numKeys];
+            for (i = 0; i < numKeys; i++) {
+                primaryKeyFields[i] = i;
+            }
+            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+            PrimaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new PrimaryIndexModificationOperationCallbackFactory(
+                    jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE);
+
+            LSMTreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+                    spec, recordDesc, appContext.getStorageManagerInterface(),
+                    appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                    comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
+                    new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), null,
+                    modificationCallbackFactory);
+
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
+
         } catch (MetadataException me) {
             throw new AlgebricksException(me);
         }
@@ -744,27 +855,27 @@
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
-            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,
-                context, spec);
+            IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+            List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification spec) throws AlgebricksException {
+        return getInsertOrDeleteRuntime(IndexOperation.INSERT, dataSource, propagatedSchema, typeEnv, keys, payload,
+                recordDesc, context, spec);
     }
 
     @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(
-            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,
-                context, spec);
+            IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+            List<LogicalVariable> keys, LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification spec) throws AlgebricksException {
+        return getInsertOrDeleteRuntime(IndexOperation.DELETE, dataSource, propagatedSchema, typeEnv, keys, payload,
+                recordDesc, context, spec);
     }
 
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOp indexOp,
-            IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
-            IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
-            List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec) throws AlgebricksException {
+    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, JobGenContext context, JobSpecification spec) throws AlgebricksException {
         String indexName = dataSourceIndex.getId();
         String dataverseName = dataSourceIndex.getDataSource().getId().getDataverseName();
         String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
@@ -783,12 +894,20 @@
         AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
         switch (secondaryIndex.getIndexType()) {
             case BTREE: {
-                return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
-                        secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
+                return getBTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
+                        primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
             }
             case RTREE: {
-                return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, primaryKeys,
-                        secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
+                return getRTreeDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
+                        primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp);
+            }
+            case WORD_INVIX:
+            case NGRAM_INVIX:
+            case FUZZY_WORD_INVIX:
+            case FUZZY_NGRAM_INVIX: {
+                return getInvertedIndexDmlRuntime(dataverseName, datasetName, indexName, propagatedSchema, typeEnv,
+                        primaryKeys, secondaryKeys, filterFactory, recordDesc, context, spec, indexOp,
+                        secondaryIndex.getIndexType());
             }
             default: {
                 throw new AlgebricksException("Insert and delete not implemented for index type: "
@@ -803,8 +922,8 @@
             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,
-                primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
+        return getIndexInsertOrDeleteRuntime(IndexOperation.INSERT, dataSourceIndex, propagatedSchema, inputSchemas,
+                typeEnv, primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
     @Override
@@ -813,8 +932,8 @@
             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,
-                primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
+        return getIndexInsertOrDeleteRuntime(IndexOperation.DELETE, dataSourceIndex, propagatedSchema, inputSchemas,
+                typeEnv, primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
     private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
@@ -831,16 +950,20 @@
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getBTreeDmlRuntime(String dataverseName,
-            String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
-            List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+            String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+            List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+            AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
+
         int numKeys = primaryKeys.size() + secondaryKeys.size();
         // generate field permutations
         int[] fieldPermutation = new int[numKeys];
+        int[] bloomFilterKeyFields = new int[secondaryKeys.size()];
         int i = 0;
         for (LogicalVariable varKey : secondaryKeys) {
             int idx = propagatedSchema.findVariable(varKey);
             fieldPermutation[i] = idx;
+            bloomFilterKeyFields[i] = i;
             i++;
         }
         for (LogicalVariable varKey : primaryKeys) {
@@ -892,20 +1015,171 @@
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataverseName, datasetName, indexName);
-            TreeIndexInsertUpdateDeleteOperatorDescriptor btreeInsert = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                    spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                    splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
-                    new BTreeDataflowHelperFactory(), filterFactory, NoOpOperationCallbackProvider.INSTANCE, jobTxnId);
-            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeInsert, splitsAndConstraint.second);
-        } catch (MetadataException | IOException e) {
+
+            //prepare callback
+            JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+            int datasetId = dataset.getDatasetId();
+            int[] primaryKeyFields = new int[primaryKeys.size()];
+            i = 0;
+            for (LogicalVariable varKey : primaryKeys) {
+                int idx = propagatedSchema.findVariable(varKey);
+                primaryKeyFields[i] = idx;
+                i++;
+            }
+            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+            SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
+                    jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_BTREE);
+
+            LSMTreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+                    spec, recordDesc, appContext.getStorageManagerInterface(),
+                    appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                    comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
+                    new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
+                    filterFactory, modificationCallbackFactory);
+            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
+                    splitsAndConstraint.second);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+    private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInvertedIndexDmlRuntime(String dataverseName,
+            String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+            List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+            AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification spec, IndexOperation indexOp, IndexType indexType) throws AlgebricksException {
+
+        // Sanity checks.
+        if (primaryKeys.size() > 1) {
+            throw new AlgebricksException("Cannot create inverted index on dataset with composite primary key.");
+        }
+        if (secondaryKeys.size() > 1) {
+            throw new AlgebricksException("Cannot create composite inverted index on multiple fields.");
+        }
+
+        int numKeys = primaryKeys.size() + secondaryKeys.size();
+        // generate field permutations
+        int[] fieldPermutation = new int[numKeys];
+        int i = 0;
+        for (LogicalVariable varKey : secondaryKeys) {
+            int idx = propagatedSchema.findVariable(varKey);
+            fieldPermutation[i] = idx;
+            i++;
+        }
+        for (LogicalVariable varKey : primaryKeys) {
+            int idx = propagatedSchema.findVariable(varKey);
+            fieldPermutation[i] = idx;
+            i++;
+        }
+
+        boolean isPartitioned;
+        if (indexType == IndexType.FUZZY_WORD_INVIX || indexType == IndexType.FUZZY_NGRAM_INVIX) {
+            isPartitioned = true;
+        } else {
+            isPartitioned = false;
+        }
+
+        Dataset dataset = findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
+        }
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType;
+        try {
+            itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), itemTypeName)
+                    .getDatatype();
+
+            if (itemType.getTypeTag() != ATypeTag.RECORD) {
+                throw new AlgebricksException("Only record types can be indexed.");
+            }
+
+            ARecordType recType = (ARecordType) itemType;
+
+            // Index parameters.
+            Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+                    dataset.getDatasetName(), indexName);
+
+            List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
+
+            int numTokenFields = (!isPartitioned) ? secondaryKeys.size() : secondaryKeys.size() + 1;
+            ITypeTraits[] tokenTypeTraits = new ITypeTraits[numTokenFields];
+            ITypeTraits[] invListsTypeTraits = new ITypeTraits[primaryKeys.size()];
+            IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numTokenFields];
+            IBinaryComparatorFactory[] invListComparatorFactories = new IBinaryComparatorFactory[primaryKeys.size()];
+
+            IAType secondaryKeyType = null;
+            for (i = 0; i < secondaryKeys.size(); ++i) {
+                Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(i)
+                        .toString(), recType);
+                secondaryKeyType = keyPairType.first;
+            }
+            List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            i = 0;
+            for (String partitioningKey : partitioningKeys) {
+                IAType keyType = recType.getFieldType(partitioningKey);
+                invListsTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+                ++i;
+            }
+
+            tokenComparatorFactories[0] = NonTaggedFormatUtil.getTokenBinaryComparatorFactory(secondaryKeyType);
+            tokenTypeTraits[0] = NonTaggedFormatUtil.getTokenTypeTrait(secondaryKeyType);
+            if (isPartitioned) {
+                // The partitioning field is hardcoded to be a short *without* an Asterix type tag.
+                tokenComparatorFactories[1] = PointableBinaryComparatorFactory.of(ShortPointable.FACTORY);
+                tokenTypeTraits[1] = ShortPointable.TYPE_TRAITS;
+            }
+            IBinaryTokenizerFactory tokenizerFactory = NonTaggedFormatUtil.getBinaryTokenizerFactory(
+                    secondaryKeyType.getTypeTag(), indexType, secondaryIndex.getGramLength());
+
+            IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
+                    dataverseName, datasetName, indexName);
+
+            //prepare callback
+            JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+            int datasetId = dataset.getDatasetId();
+            int[] primaryKeyFields = new int[primaryKeys.size()];
+            i = 0;
+            for (LogicalVariable varKey : primaryKeys) {
+                int idx = propagatedSchema.findVariable(varKey);
+                primaryKeyFields[i] = idx;
+                i++;
+            }
+            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+            SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
+                    jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_INVERTED_INDEX);
+
+            LSMInvertedIndexInsertUpdateDeleteOperator insertDeleteOp = new LSMInvertedIndexInsertUpdateDeleteOperator(
+                    spec, recordDesc, appContext.getStorageManagerInterface(), splitsAndConstraint.first,
+                    appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
+                    invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
+                    new LSMInvertedIndexDataflowHelperFactory(
+                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
+                    filterFactory, modificationCallbackFactory);
+            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
+                    splitsAndConstraint.second);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        } catch (IOException e) {
             throw new AlgebricksException(e);
         }
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getRTreeDmlRuntime(String dataverseName,
-            String datasetName, String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
-            List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
-            JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
+            String datasetName, String indexName, IOperatorSchema propagatedSchema, IVariableTypeEnvironment typeEnv,
+            List<LogicalVariable> primaryKeys, List<LogicalVariable> secondaryKeys,
+            AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc, JobGenContext context,
+            JobSpecification spec, IndexOperation indexOp) throws AlgebricksException {
         try {
             Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
             String itemTypeName = dataset.getItemTypeName();
@@ -955,28 +1229,66 @@
                 ++i;
             }
 
+            IBinaryComparatorFactory[] primaryComparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
+                    dataset, recType, context.getBinaryComparatorFactoryProvider());
             IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
             Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
                     dataverseName, datasetName, indexName);
-            TreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                    spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                    splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
-                    new RTreeDataflowHelperFactory(valueProviderFactories), filterFactory,
-                    NoOpOperationCallbackProvider.INSTANCE, jobTxnId);
+
+            //prepare callback
+            JobId jobId = ((JobEventListenerFactory) spec.getJobletEventListenerFactory()).getJobId();
+            int datasetId = dataset.getDatasetId();
+            int[] primaryKeyFields = new int[numPrimaryKeys];
+            i = 0;
+            for (LogicalVariable varKey : primaryKeys) {
+                int idx = propagatedSchema.findVariable(varKey);
+                primaryKeyFields[i] = idx;
+                i++;
+            }
+            TransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+            SecondaryIndexModificationOperationCallbackFactory modificationCallbackFactory = new SecondaryIndexModificationOperationCallbackFactory(
+                    jobId, datasetId, primaryKeyFields, txnSubsystemProvider, indexOp, ResourceType.LSM_RTREE);
+
+            LSMTreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new LSMTreeIndexInsertUpdateDeleteOperatorDescriptor(
+                    spec, recordDesc, appContext.getStorageManagerInterface(),
+                    appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
+                    comparatorFactories, null, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
+                            valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
+                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+                                    nestedKeyType.getTypeTag(), comparatorFactories.length),
+                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
+                    filterFactory, modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
         } catch (MetadataException | IOException e) {
             throw new AlgebricksException(e);
         }
     }
 
-    public long getJobTxnId() {
-        return jobTxnId;
+    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 8f1e378..91a18f3 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
@@ -38,14 +38,19 @@
     private final IDatasetDetails datasetDetails;
     // Hints related to cardinatlity of dataset, avg size of tuples etc.
     private final Map<String, String> hints;
+    private final int datasetId;
+    // Type of pending operations with respect to atomic DDL operation
+    private final int pendingOp;
 
     public Dataset(String dataverseName, String datasetName, String itemTypeName, IDatasetDetails datasetDetails,
-            Map<String, String> hints, DatasetType datasetType) {
+            Map<String, String> hints, DatasetType datasetType, int datasetId, int pendingOp) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.itemTypeName = itemTypeName;
         this.datasetType = datasetType;
         this.datasetDetails = datasetDetails;
+        this.datasetId = datasetId;
+        this.pendingOp = pendingOp;
         this.hints = hints;
     }
 
@@ -73,6 +78,14 @@
         return hints;
     }
 
+    public int getDatasetId() {
+        return datasetId;
+    }
+
+    public int getPendingOp() {
+        return pendingOp;
+    }
+
     @Override
     public Object addToCache(MetadataCache cache) {
         return cache.addDatasetIfNotExists(this);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataverse.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataverse.java
index bacd812..3902282 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataverse.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataverse.java
@@ -27,10 +27,12 @@
     // Enforced to be unique within an Asterix cluster..
     private final String dataverseName;
     private final String dataFormat;
+    private final int pendingOp;
 
-    public Dataverse(String dataverseName, String format) {
+    public Dataverse(String dataverseName, String format, int pendingOp) {
         this.dataverseName = dataverseName;
         this.dataFormat = format;
+        this.pendingOp = pendingOp;
     }
 
     public String getDataverseName() {
@@ -40,6 +42,10 @@
     public String getDataFormat() {
         return dataFormat;
     }
+    
+    public int getPendingOp() {
+        return pendingOp;
+    }
 
     @Override
     public Object addToCache(MetadataCache cache) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
index 4265630..6d65730 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
@@ -19,6 +19,8 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.AUnionType;
@@ -29,7 +31,7 @@
 /**
  * Metadata describing an index.
  */
-public class Index implements Serializable {
+public class Index implements IMetadataEntity {
 
     private static final long serialVersionUID = 1L;
 
@@ -43,9 +45,11 @@
     private final boolean isPrimaryIndex;
     // Specific to NGRAM indexes.
     private final int gramLength;
+    // Type of pending operations with respect to atomic DDL operation
+    private final int pendingOp;
 
     public Index(String dataverseName, String datasetName, String indexName, IndexType indexType,
-            List<String> keyFieldNames, int gramLength, boolean isPrimaryIndex) {
+            List<String> keyFieldNames, int gramLength, boolean isPrimaryIndex, int pendingOp) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.indexName = indexName;
@@ -53,10 +57,11 @@
         this.keyFieldNames = keyFieldNames;
         this.gramLength = gramLength;
         this.isPrimaryIndex = isPrimaryIndex;
+        this.pendingOp = pendingOp;
     }
 
     public Index(String dataverseName, String datasetName, String indexName, IndexType indexType,
-            List<String> keyFieldNames, boolean isPrimaryIndex) {
+            List<String> keyFieldNames, boolean isPrimaryIndex, int pendingOp) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.indexName = indexName;
@@ -64,6 +69,7 @@
         this.keyFieldNames = keyFieldNames;
         this.gramLength = -1;
         this.isPrimaryIndex = isPrimaryIndex;
+        this.pendingOp = pendingOp;
     }
 
     public String getDataverseName() {
@@ -93,6 +99,10 @@
     public boolean isPrimaryIndex() {
         return isPrimaryIndex;
     }
+    
+    public int getPendingOp() {
+        return pendingOp;
+    }
 
     public boolean isSecondaryIndex() {
         return !isPrimaryIndex();
@@ -146,4 +156,14 @@
         }
         return true;
     }
+
+    @Override
+    public Object addToCache(MetadataCache cache) {
+        return cache.addIndexIfNotExists(this);
+    }
+
+    @Override
+    public Object dropFromCache(MetadataCache cache) {
+        return cache.dropIndex(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 a0aa36e..61f856a 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
@@ -43,6 +43,8 @@
 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.ANull;
 import edu.uci.ics.asterix.om.base.AOrderedList;
@@ -72,9 +74,14 @@
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(MetadataRecordTypes.DATASET_RECORDTYPE);
+    private AMutableInt32 aInt32;
+    protected ISerializerDeserializer<AInt32> aInt32Serde;
 
+    @SuppressWarnings("unchecked")
     public DatasetTupleTranslator(boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.DATASET_DATASET.getFieldCount());
+        aInt32 = new AMutableInt32(-1);
+        aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
     }
 
     @Override
@@ -89,6 +96,7 @@
     }
 
     private Dataset createDatasetFromARecord(ARecord datasetRecord) {
+
         String dataverseName = ((AString) datasetRecord
                 .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_DATAVERSENAME_FIELD_INDEX)).getStringValue();
         String datasetName = ((AString) datasetRecord
@@ -97,6 +105,10 @@
                 .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();
+        int pendingOp = ((AInt32) datasetRecord
+                .getValueByPos(MetadataRecordTypes.DATASET_ARECORD_PENDINGOP_FIELD_INDEX)).getIntegerValue();
         switch (datasetType) {
             case FEED: {
                 ARecord datasetDetailsRecord = (ARecord) datasetRecord
@@ -209,8 +221,10 @@
                 }
                 datasetDetails = new ExternalDatasetDetails(adapter, properties);
         }
+        
         Map<String, String> hints = getDatasetHints(datasetRecord);
-        return new Dataset(dataverseName, datasetName, typeName, datasetDetails, hints, datasetType);
+        
+        return new Dataset(dataverseName, datasetName, typeName, datasetDetails, hints, datasetType, datasetId, pendingOp);
     }
 
     @Override
@@ -278,6 +292,18 @@
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
 
+        // write field 9
+        fieldValue.reset();
+        aInt32.setValue(dataset.getDatasetId());
+        aInt32Serde.serialize(aInt32, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATASETID_FIELD_INDEX, fieldValue);
+
+        // write field 10
+        fieldValue.reset();
+        aInt32.setValue(dataset.getPendingOp());
+        aInt32Serde.serialize(aInt32, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_PENDINGOP_FIELD_INDEX, fieldValue);
+
         // write record
         try {
             recordBuilder.write(tupleBuilder.getDataOutput(), true);
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 185f35b..7cfa6d0 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.NotImplementedException;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -80,11 +81,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;
     }
 
@@ -185,7 +186,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);
             }
@@ -429,11 +430,11 @@
         try {
             if (typeName == null) {
                 typeName = suggestedTypeName;
-                metadataNode.addDatatype(txnId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType,
+                metadataNode.addDatatype(jobId, new Datatype(topLevelType.getDataverseName(), typeName, nestedType,
                         true));
 
             }
-            mn.insertIntoDatatypeSecondaryIndex(txnId, topLevelType.getDataverseName(), typeName,
+            mn.insertIntoDatatypeSecondaryIndex(jobId, topLevelType.getDataverseName(), typeName,
                     topLevelType.getDatatypeName());
 
         } catch (BTreeDuplicateKeyException e) {
@@ -441,7 +442,6 @@
             // a previous nested type.
         }
         return typeName;
-
     }
 
     private boolean isDerivedType(ATypeTag tag) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
index 1e32004..23e42d6 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DataverseTupleTranslator.java
@@ -27,8 +27,11 @@
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
 import edu.uci.ics.asterix.om.base.ARecord;
 import edu.uci.ics.asterix.om.base.AString;
+import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 
@@ -42,12 +45,18 @@
     // Payload field containing serialized Dataverse.
     public static final int DATAVERSE_PAYLOAD_TUPLE_FIELD_INDEX = 1;
 
+    private AMutableInt32 aInt32;
+    protected ISerializerDeserializer<AInt32> aInt32Serde;
+
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(MetadataRecordTypes.DATAVERSE_RECORDTYPE);
 
+    @SuppressWarnings("unchecked")
     public DataverseTupleTranslator(boolean getTuple) {
         super(getTuple, MetadataPrimaryIndexes.DATAVERSE_DATASET.getFieldCount());
+        aInt32 = new AMutableInt32(-1);
+        aInt32Serde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
     }
 
     @Override
@@ -59,7 +68,8 @@
         DataInput in = new DataInputStream(stream);
         ARecord dataverseRecord = recordSerDes.deserialize(in);
         return new Dataverse(((AString) dataverseRecord.getValueByPos(0)).getStringValue(),
-                ((AString) dataverseRecord.getValueByPos(1)).getStringValue());
+                ((AString) dataverseRecord.getValueByPos(1)).getStringValue(),
+                ((AInt32) dataverseRecord.getValueByPos(3)).getIntegerValue());
     }
 
     @Override
@@ -90,6 +100,12 @@
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATAVERSE_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
 
+        // write field 3
+        fieldValue.reset();
+        aInt32.setValue(instance.getPendingOp());
+        aInt32Serde.serialize(aInt32, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.DATAVERSE_ARECORD_PENDINGOP_FIELD_INDEX, fieldValue);
+
         try {
             recordBuilder.write(tupleBuilder.getDataOutput(), true);
         } catch (AsterixException e) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
index 07d7cc9..e6f0462 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/IndexTupleTranslator.java
@@ -98,13 +98,15 @@
         }
         Boolean isPrimaryIndex = ((ABoolean) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_ISPRIMARY_FIELD_INDEX))
                 .getBoolean();
+        int pendingOp = ((AInt32) rec.getValueByPos(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX))
+                .getIntegerValue();
         // Check if there is a gram length as well.
         int gramLength = -1;
         int gramLenPos = rec.getType().findFieldPosition(GRAM_LENGTH_FIELD_NAME);
         if (gramLenPos >= 0) {
             gramLength = ((AInt32) rec.getValueByPos(gramLenPos)).getIntegerValue();
         }
-        return new Index(dvName, dsName, indexName, indexStructure, searchKey, gramLength, isPrimaryIndex);
+        return new Index(dvName, dsName, indexName, indexStructure, searchKey, gramLength, isPrimaryIndex, pendingOp);
     }
 
     @Override
@@ -176,7 +178,12 @@
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.INDEX_ARECORD_TIMESTAMP_FIELD_INDEX, fieldValue);
 
-        // write optional field 7        
+        // write field 7
+        fieldValue.reset();
+        intSerde.serialize(new AInt32(instance.getPendingOp()), fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.INDEX_ARECORD_PENDINGOP_FIELD_INDEX, fieldValue);
+
+        // write optional field 8        
         if (instance.getGramLength() > 0) {
             fieldValue.reset();
             nameValue.reset();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
index df41c6e..820f277 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
@@ -52,6 +52,18 @@
         return bcfs;
     }
 
+    public static int[] createBloomFilterKeyFields(Dataset dataset) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            throw new AlgebricksException("not implemented");
+        }
+        List<String> partitioningKeys = getPartitioningKeys(dataset);
+        int[] bloomFilterKeyFields = new int[partitioningKeys.size()];
+        for (int i = 0; i < partitioningKeys.size(); ++i) {
+            bloomFilterKeyFields[i] = i;
+        }
+        return bloomFilterKeyFields;
+    }
+
     public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(Dataset dataset, ARecordType itemType,
             IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
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 a74d233..83819dc 100644
--- a/asterix-om/pom.xml
+++ b/asterix-om/pom.xml
@@ -32,21 +32,18 @@
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-invertedindex</artifactId>
+			<artifactId> hyracks-storage-am-lsm-invertedindex </artifactId>
+			<version>0.2.4-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>
+        <dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>algebricks-compiler</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
 		</dependency>
 		<dependency>
 			<groupId>edu.uci.ics.hyracks</groupId>
-			<artifactId>hyracks-storage-am-rtree</artifactId>
+			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
 		</dependency>
 	</dependencies>
 </project>
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/builders/OrderedListBuilder.java b/asterix-om/src/main/java/edu/uci/ics/asterix/builders/OrderedListBuilder.java
index 1fdd7df..caad260 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/builders/OrderedListBuilder.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/builders/OrderedListBuilder.java
@@ -3,7 +3,7 @@
 import java.io.IOException;
 
 import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IToken;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IToken;
 
 public class OrderedListBuilder extends AbstractListBuilder {
 
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 df3b43e..d1ee8c2 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.IOException;
 
 import edu.uci.ics.hyracks.data.std.util.GrowableArray;
-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
deleted file mode 100644
index 2a7fdb5..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifier.java
+++ /dev/null
@@ -1,36 +0,0 @@
-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;
-
-// TODO: Should go into hyracks.
-public class ListEditDistanceSearchModifier implements IInvertedIndexSearchModifier {
-
-    private int edThresh;
-
-    public ListEditDistanceSearchModifier(int edThresh) {
-        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;
-    }
-
-    public void setEdThresh(int edThresh) {
-        this.edThresh = edThresh;
-    }
-}
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
deleted file mode 100644
index b117a36..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/common/ListEditDistanceSearchModifierFactory.java
+++ /dev/null
@@ -1,21 +0,0 @@
-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;
-
-// TODO: Should go into hyracks.
-public class ListEditDistanceSearchModifierFactory implements IInvertedIndexSearchModifierFactory {
-
-    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 9dfe7df..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
@@ -5,11 +5,11 @@
 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-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index e8a9719..24649d9 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -181,7 +181,7 @@
     public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN = new FunctionIdentifier(
             FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even", 1);
     public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN2 = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even2", 2);
+            FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even", 2);
     // String funcitons
     public final static FunctionIdentifier STRING_EQUAL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "string-equal", 2);
@@ -192,17 +192,17 @@
     public final static FunctionIdentifier STRING_MATCHES = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "matches", 2);
     public final static FunctionIdentifier STRING_MATCHES_WITH_FLAG = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "matches2", 3);
+            FunctionConstants.ASTERIX_NS, "matches", 3);
     public final static FunctionIdentifier STRING_LOWERCASE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "lowercase", 1);
     public final static FunctionIdentifier STRING_REPLACE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "replace", 3);
     public final static FunctionIdentifier STRING_REPLACE_WITH_FLAG = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "replace2", 4);
+            FunctionConstants.ASTERIX_NS, "replace", 4);
     public final static FunctionIdentifier STRING_LENGTH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "string-length", 1);
     public final static FunctionIdentifier SUBSTRING2 = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "substring2", 2);
+            "substring", 2);
     public final static FunctionIdentifier SUBSTRING_BEFORE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "substring-before", 2);
     public final static FunctionIdentifier SUBSTRING_AFTER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
@@ -494,12 +494,17 @@
     public final static FunctionIdentifier ADJUST_DATETIME_FOR_TIMEZONE = new FunctionIdentifier(
             FunctionConstants.ASTERIX_NS, "adjust-datetime-for-timezone", 2);
 
-    public final static FunctionIdentifier GET_POINT_X_COORDINATE_ACCESSOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-x", 1);
-    public final static FunctionIdentifier GET_POINT_Y_COORDINATE_ACCESSOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-y", 1);
-    public final static FunctionIdentifier GET_CIRCLE_RADIUS_ACCESSOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-radius", 1);
-    public final static FunctionIdentifier GET_CIRCLE_CENTER_ACCESSOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-center", 1);
-    public final static FunctionIdentifier GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-points", 1);
-    
+    public final static FunctionIdentifier GET_POINT_X_COORDINATE_ACCESSOR = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "get-x", 1);
+    public final static FunctionIdentifier GET_POINT_Y_COORDINATE_ACCESSOR = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "get-y", 1);
+    public final static FunctionIdentifier GET_CIRCLE_RADIUS_ACCESSOR = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "get-radius", 1);
+    public final static FunctionIdentifier GET_CIRCLE_CENTER_ACCESSOR = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "get-center", 1);
+    public final static FunctionIdentifier GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR = new FunctionIdentifier(
+            FunctionConstants.ASTERIX_NS, "get-points", 1);
+
     public static final FunctionIdentifier EQ = AlgebricksBuiltinFunctions.EQ;
     public static final FunctionIdentifier LE = AlgebricksBuiltinFunctions.LE;
     public static final FunctionIdentifier GE = AlgebricksBuiltinFunctions.GE;
@@ -995,7 +1000,7 @@
         funTypeComputer.put(functionInfo, typeComputer);
         finfoRepo.put(fi);
     }
-    
+
     private static IFunctionInfo addPrivateFunction(FunctionIdentifier fi, IResultTypeComputer typeComputer) {
         IFunctionInfo functionInfo = getAsterixFunctionInfo(fi);
         builtinFunctionsSet.put(functionInfo, functionInfo);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
index b8b1c61..4655c37 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixFunctionInfo.java
@@ -56,12 +56,14 @@
             return false;
         }
         AsterixFunctionInfo info = (AsterixFunctionInfo) o;
-        return functionIdentifier.equals(info.getFunctionIdentifier());
+        return functionIdentifier.equals(info.getFunctionIdentifier())
+                && functionIdentifier.getArity() == info.getFunctionIdentifier().getArity();
     }
 
     @Override
     public String toString() {
-        return this.functionIdentifier.getNamespace() + ":" + this.functionIdentifier.getName();
+        return this.functionIdentifier.getNamespace() + ":" + this.functionIdentifier.getName() + "@"
+                + this.functionIdentifier.getArity();
     }
 
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
index a8c0485..047c459 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
@@ -2,18 +2,29 @@
 
 import java.util.List;
 
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARecordSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlBinaryTokenizerFactoryProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.EnumDeserializer;
 import edu.uci.ics.asterix.om.types.IAType;
+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.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 public final class NonTaggedFormatUtil {
 
@@ -160,4 +171,58 @@
                 throw new NotImplementedException(typeTag + " is not a supported spatial data type.");
         }
     }
+
+    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
+            int gramLength) throws AlgebricksException {
+        switch (indexType) {
+            case WORD_INVIX:
+            case FUZZY_WORD_INVIX: {
+                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(keyType, false);
+            }
+            case NGRAM_INVIX:
+            case FUZZY_NGRAM_INVIX: {
+                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType, gramLength, true,
+                        false);
+            }
+            default: {
+                throw new AlgebricksException("Tokenizer not applicable to index type '" + indexType + "'.");
+            }
+        }
+    }
+
+    public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
+        IAType type = keyType;
+        ATypeTag typeTag = keyType.getTypeTag();
+        // Extract item type from list.
+        if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
+            AbstractCollectionType listType = (AbstractCollectionType) keyType;
+            if (!listType.isTyped()) {
+                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
+            }
+            type = listType.getItemType();
+        }
+        // Ignore case for string types.
+        return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
+    }
+
+    public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
+        IAType type = keyType;
+        ATypeTag typeTag = keyType.getTypeTag();
+        // Extract item type from list.
+        if (typeTag == ATypeTag.UNORDEREDLIST) {
+            AUnorderedListType ulistType = (AUnorderedListType) keyType;
+            if (!ulistType.isTyped()) {
+                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
+            }
+            type = ulistType.getItemType();
+        }
+        if (typeTag == ATypeTag.ORDEREDLIST) {
+            AOrderedListType olistType = (AOrderedListType) keyType;
+            if (!olistType.isTyped()) {
+                throw new AlgebricksException("Cannot build an inverted index on untyped lists.)");
+            }
+            type = olistType.getItemType();
+        }
+        return AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
+    }
 }
diff --git a/asterix-runtime/pom.xml b/asterix-runtime/pom.xml
index 245f577..119d447 100644
--- a/asterix-runtime/pom.xml
+++ b/asterix-runtime/pom.xml
@@ -132,6 +132,11 @@
 			<type>jar</type>
 			<scope>compile</scope>
 		</dependency>
+		<dependency>
+		         <groupId>edu.uci.ics.hyracks</groupId>
+		         <artifactId>hyracks-api</artifactId>
+		         <version>0.2.4-SNAPSHOT</version>
+	        </dependency>
 	</dependencies>
 
 </project>
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java
index b5b6f29..6f834a7 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java
@@ -24,31 +24,34 @@
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.functions.IFunctionManager;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class FunctionManagerImpl implements IFunctionManager {
-    private final Map<FunctionIdentifier, IFunctionDescriptorFactory> functions;
+    private final Map<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory> functions;
 
     public FunctionManagerImpl() {
-        functions = new HashMap<FunctionIdentifier, IFunctionDescriptorFactory>();
+        functions = new HashMap<Pair<FunctionIdentifier, Integer>, IFunctionDescriptorFactory>();
     }
 
     @Override
     public synchronized IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException {
-        return functions.get(fid).createFunctionDescriptor();
+        Pair<FunctionIdentifier, Integer> key = new Pair<FunctionIdentifier, Integer>(fid, fid.getArity());
+        return functions.get(key).createFunctionDescriptor();
     }
 
     @Override
     public synchronized void registerFunction(IFunctionDescriptorFactory descriptorFactory) throws AlgebricksException {
         FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
-        functions.put(fid, descriptorFactory);
+        functions.put(new Pair<FunctionIdentifier, Integer>(fid, fid.getArity()), descriptorFactory);
     }
 
     @Override
     public synchronized void unregisterFunction(IFunctionDescriptorFactory descriptorFactory)
             throws AlgebricksException {
         FunctionIdentifier fid = descriptorFactory.createFunctionDescriptor().getIdentifier();
-        functions.remove(fid);
+        Pair<FunctionIdentifier, Integer> key = new Pair<FunctionIdentifier, Integer>(fid, fid.getArity());
+        functions.remove(key);
     }
 
     @Override
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 261039a..d336c05 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
@@ -14,8 +14,8 @@
 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.NGramUTF8StringBinaryTokenizer;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizer;
+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 c316fb5..a98b754 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
@@ -12,7 +12,7 @@
 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.lsm.invertedindex.tokenizers.IBinaryTokenizer;
 
 public class WordTokensEvaluator implements ICopyEvaluator {
     private 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 c4e8387..0bdb7ff 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
@@ -11,9 +11,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 90a4293..be982c0 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
@@ -11,10 +11,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 20a2977..1993453 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
@@ -11,9 +11,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 e86f86a6..43fda6f 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
@@ -11,9 +11,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 d6da522..554044a 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
@@ -11,10 +11,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/SpatialCellDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
index 9b8d1a4..9c344ad 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialCellDescriptor.java
@@ -19,7 +19,6 @@
 import edu.uci.ics.asterix.om.types.EnumDeserializer;
 import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
@@ -78,9 +77,16 @@
                         eval3.evaluate(tuple);
 
                         try {
-                            ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER
+                            ATypeTag tag0 = EnumDeserializer.ATYPETAGDESERIALIZER
                                     .deserialize(outInput0.getByteArray()[0]);
-                            if (tag == ATypeTag.POINT) {
+                            ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER
+                                    .deserialize(outInput1.getByteArray()[0]);
+                            ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER
+                                    .deserialize(outInput2.getByteArray()[0]);
+                            ATypeTag tag3 = EnumDeserializer.ATYPETAGDESERIALIZER
+                                    .deserialize(outInput3.getByteArray()[0]);
+                            if (tag0 == ATypeTag.POINT && tag1 == ATypeTag.POINT && tag2 == ATypeTag.DOUBLE
+                                    && tag3 == ATypeTag.DOUBLE) {
                                 double xLoc = ADoubleSerializerDeserializer.getDouble(outInput0.getByteArray(),
                                         APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
                                 double yLoc = ADoubleSerializerDeserializer.getDouble(outInput0.getByteArray(),
@@ -100,11 +106,24 @@
                                 aPoint[1].setValue(x + xInc, y + yInc);
                                 aRectangle.setValue(aPoint[0], aPoint[1]);
                                 rectangleSerde.serialize(aRectangle, out);
-                            } else if (tag == ATypeTag.NULL) {
+                            } else if (tag0 == ATypeTag.NULL || tag1 == ATypeTag.NULL || tag2 == ATypeTag.NULL
+                                    || tag3 == ATypeTag.NULL) {
                                 nullSerde.serialize(ANull.NULL, out);
                             } else {
-                                throw new NotImplementedException(AsterixBuiltinFunctions.SPATIAL_CELL.getName()
-                                        + ": does not support the type: " + tag + "; it is only implemented for POINT.");
+                                throw new AlgebricksException(
+                                        AsterixBuiltinFunctions.SPATIAL_CELL.getName()
+                                                + ": expects input type: (POINT, POINT, DOUBLE, DOUBLE) but got ("
+                                                + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outInput0
+                                                        .getByteArray()[0])
+                                                + ", "
+                                                + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outInput1
+                                                        .getByteArray()[0])
+                                                + ", "
+                                                + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outInput2
+                                                        .getByteArray()[0])
+                                                + ", "
+                                                + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outInput3
+                                                        .getByteArray()[0]) + ").");
                             }
                         } catch (IOException e1) {
                             throw new AlgebricksException(e1);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index 55acc01..96d8a3c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -67,26 +67,11 @@
                     private final DoubleArray trianglesX1 = new DoubleArray();
                     private final DoubleArray trianglesY1 = new DoubleArray();
 
-                    private boolean pointOnLine(byte[] bytes0, byte[] bytes1) throws HyracksDataException {
-
-                        double startX = ADoubleSerializerDeserializer.getDouble(bytes0,
-                                ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X));
-                        double startY = ADoubleSerializerDeserializer.getDouble(bytes0,
-                                ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y));
-                        double endX = ADoubleSerializerDeserializer.getDouble(bytes0,
-                                ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X));
-                        double endY = ADoubleSerializerDeserializer.getDouble(bytes0,
-                                ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.Y));
-
-                        double pX = ADoubleSerializerDeserializer.getDouble(bytes1,
-                                APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
-                        double pY = ADoubleSerializerDeserializer.getDouble(bytes1,
-                                APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
-
+                    private boolean pointOnLine(double pX, double pY, double startX, double startY, double endX,
+                            double endY) throws HyracksDataException {
                         double crossProduct = SpatialUtils.crossProduct(pY - startY, pX - startX, endY - startY, endX
                                 - startX);
-                        if (Math.abs(crossProduct) > SpatialUtils.doubleEpsilon()) { // crossProduct
-                            // != 0
+                        if (Math.abs(crossProduct) > SpatialUtils.doubleEpsilon()) { // crossProduct != 0
                             return false;
                         }
 
@@ -103,8 +88,7 @@
                         return true;
                     }
 
-                    private boolean pointInPolygon(byte[] bytes0, byte[] bytes1) throws HyracksDataException { // ray
-                        // casting
+                    private boolean pointInPolygon(byte[] bytes0, byte[] bytes1) throws HyracksDataException { // ray casting
 
                         double pX = ADoubleSerializerDeserializer.getDouble(bytes0,
                                 APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
@@ -138,13 +122,16 @@
                                 y2 = ADoubleSerializerDeserializer.getDouble(bytes1,
                                         APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.Y));
                             }
-                            if (pY > Math.min(y1, y2)) {
-                                if (pY <= Math.max(y1, y2)) {
-                                    if (pX <= Math.max(x1, x2)) {
-                                        if (y1 != y2) {
-                                            xInters = (pY - y1) * (x2 - x1) / (y2 - y1) + x1;
-                                            if (x1 == x2 || pX <= xInters) {
-                                                counter++;
+
+                            if (!pointOnLine(pX, pY, x1, y1, x2, y2)) {
+                                if (pY > Math.min(y1, y2)) {
+                                    if (pY <= Math.max(y1, y2)) {
+                                        if (pX <= Math.max(x1, x2)) {
+                                            if (y1 != y2) {
+                                                xInters = (pY - y1) * (x2 - x1) / (y2 - y1) + x1;
+                                                if (x1 == x2 || pX <= xInters) {
+                                                    counter++;
+                                                }
                                             }
                                         }
                                     }
@@ -173,7 +160,7 @@
                         double radius = ADoubleSerializerDeserializer.getDouble(bytes1,
                                 ACircleSerializerDeserializer.getRadiusOffset());
 
-                        if ((x - cX) * (x - cX) + (y - cY) * (y - cY) < (radius * radius)) {
+                        if ((x - cX) * (x - cX) + (y - cY) * (y - cY) <= (radius * radius)) {
                             return true;
                         }
                         return false;
@@ -190,9 +177,7 @@
                         double C2 = A2 * startX2 + B2 * startY2;
 
                         double det = (A1 * B2) - (A2 * B1);
-                        if (Math.abs(det) > SpatialUtils.doubleEpsilon()) { // det
-                                                                            // !=
-                                                                            // 0
+                        if (Math.abs(det) > SpatialUtils.doubleEpsilon()) { // det != 0
                             double x = (B2 * C1 - B1 * C2) / det;
                             double y = (A1 * C2 - A2 * C1) / det;
 
@@ -313,7 +298,7 @@
                         dx = (startX + t * (endX - startX)) - cX;
                         dy = (startY + t * (endY - startY)) - cY;
                         double rt = (dx * dx) + (dy * dy);
-                        if (rt < (radius * radius)) {
+                        if (rt <= (radius * radius)) {
                             return true;
                         }
                         return false;
@@ -362,8 +347,7 @@
 
                     private int triangulatePolygon(byte[] bytes, int numOfPoints, IntArray pointsOffsets,
                             DoubleArray trianglesX, DoubleArray trianglesY, int triangleId,
-                            int nonSimplePolygonDetection, int middleVertex) throws HyracksDataException { // Ear
-                                                                                                           // clipping
+                            int nonSimplePolygonDetection, int middleVertex) throws HyracksDataException { // Ear clipping
 
                         if (numOfPoints < 3) {
                             return -1;
@@ -431,8 +415,7 @@
 
                     private boolean triangleTriangleIntersection(DoubleArray trianglesX0, DoubleArray trianglesY0,
                             int triangleId0, DoubleArray trianglesX1, DoubleArray trianglesY1, int triangleId1)
-                            throws HyracksDataException { // separating axis
-                                                          // theorem
+                            throws HyracksDataException { // separating axis theorem
 
                         for (int side = 0; side < 3; side++) {
                             spatialUtils.findNormals(trianglesX0, trianglesY0, triangleId0, side);
@@ -452,10 +435,8 @@
                         return true;
                     }
 
-                    private final boolean pointInsideTriangle(double x1, double y1, double x2, double y2, double x3,
-                            double y3, double pX, double pY)
-
-                    {
+                    private boolean pointInsideTriangle(double x1, double y1, double x2, double y2, double x3,
+                            double y3, double pX, double pY) {
                         return pointsOnSameSide(pX, pY, x1, y1, x2, y2, x3, y3)
                                 && pointsOnSameSide(pX, pY, x2, y2, x1, y1, x3, y3)
                                 && pointsOnSameSide(pX, pY, x3, y3, x1, y1, x2, y2);
@@ -469,9 +450,7 @@
                     }
 
                     private boolean circleTriangleIntersection(byte[] bytes0, DoubleArray trianglesX,
-                            DoubleArray trianglesY, int triangleId) throws HyracksDataException { // separating
-                                                                                                  // axis
-                                                                                                  // theorem
+                            DoubleArray trianglesY, int triangleId) throws HyracksDataException { // separating axis theorem
 
                         double cX = ADoubleSerializerDeserializer.getDouble(bytes0,
                                 ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.X));
@@ -779,7 +758,27 @@
                                             }
                                             break;
                                         case LINE:
-                                            res = pointOnLine(outInput1.getByteArray(), outInput0.getByteArray());
+                                            double pX = ADoubleSerializerDeserializer.getDouble(
+                                                    outInput0.getByteArray(),
+                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+                                            double pY = ADoubleSerializerDeserializer.getDouble(
+                                                    outInput0.getByteArray(),
+                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+
+                                            double startX = ADoubleSerializerDeserializer.getDouble(outInput1
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY = ADoubleSerializerDeserializer.getDouble(outInput1
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX = ADoubleSerializerDeserializer.getDouble(outInput1
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY = ADoubleSerializerDeserializer.getDouble(outInput1
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+
+                                            res = pointOnLine(pX, pY, startX, startY, endX, endY);
                                             break;
                                         case POLYGON:
                                             res = pointInPolygon(outInput0.getByteArray(), outInput1.getByteArray());
@@ -804,7 +803,27 @@
                                 case LINE:
                                     switch (tag1) {
                                         case POINT:
-                                            res = pointOnLine(outInput0.getByteArray(), outInput1.getByteArray());
+                                            double pX = ADoubleSerializerDeserializer.getDouble(
+                                                    outInput1.getByteArray(),
+                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.X));
+                                            double pY = ADoubleSerializerDeserializer.getDouble(
+                                                    outInput1.getByteArray(),
+                                                    APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y));
+
+                                            double startX = ADoubleSerializerDeserializer.getDouble(outInput0
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getStartPointCoordinateOffset(Coordinate.X));
+                                            double startY = ADoubleSerializerDeserializer.getDouble(outInput0
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getStartPointCoordinateOffset(Coordinate.Y));
+                                            double endX = ADoubleSerializerDeserializer.getDouble(outInput0
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.X));
+                                            double endY = ADoubleSerializerDeserializer.getDouble(outInput0
+                                                    .getByteArray(), ALineSerializerDeserializer
+                                                    .getEndPointCoordinateOffset(Coordinate.Y));
+
+                                            res = pointOnLine(pX, pY, startX, startY, endX, endY);
                                             break;
                                         case LINE:
                                             double startX1 = ADoubleSerializerDeserializer.getDouble(outInput0
@@ -1072,5 +1091,4 @@
     public FunctionIdentifier getIdentifier() {
         return AsterixBuiltinFunctions.SPATIAL_INTERSECT;
     }
-
 }
\ No newline at end of file
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 791ee6b..80cb6db 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
@@ -11,10 +11,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..45e19c6 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
@@ -2,7 +2,9 @@
 
 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.DatasetId;
 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,13 +15,17 @@
 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;
     }
+    
+    public JobId getJobId() {
+        return jobId;
+    }
 
     @Override
     public IJobletEventListener createListener(final IHyracksJobletContext jobletContext) {
@@ -29,11 +35,11 @@
             public void jobletFinish(JobStatus jobStatus) {
                 try {
                     ITransactionManager txnManager = ((AsterixAppRuntimeContext) jobletContext.getApplicationContext()
-                            .getApplicationObject()).getTransactionProvider().getTransactionManager();
-                    TransactionContext txnContext = txnManager.getTransactionContext(txnId);
+                            .getApplicationObject()).getTransactionSubsystem().getTransactionManager();
+                    TransactionContext txnContext = txnManager.getTransactionContext(jobId);
                     txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
                             : TransactionType.READ);
-                    txnManager.completedTransaction(txnContext, !(jobStatus == JobStatus.FAILURE));
+                    txnManager.completedTransaction(txnContext, new DatasetId(-1), -1, !(jobStatus == JobStatus.FAILURE));
                 } catch (ACIDException e) {
                     throw new Error(e);
                 }
@@ -41,7 +47,12 @@
 
             @Override
             public void jobletStart() {
-
+                try {
+                    ((AsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
+                            .getTransactionSubsystem().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 58d779e..3d28851 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
@@ -47,6 +47,7 @@
 import edu.uci.ics.asterix.om.types.TypeSignature;
 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;
@@ -930,7 +931,7 @@
         List<Statement> statements = 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, statements);
         dmlt.translate();
         typeMap = dmlt.getTypeMap();
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java
index 7b91fcb..526580e 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/translator/ADGenDmlTranslator.java
@@ -19,58 +19,54 @@
 
 public class ADGenDmlTranslator extends AbstractAqlTranslator {
 
-	private final MetadataTransactionContext mdTxnCtx;
-	private final List<Statement> aqlStatements;
-	private Map<TypeSignature, IAType> types;
-	private Map<TypeSignature, TypeDataGen> typeDataGenMap;
+    private final MetadataTransactionContext mdTxnCtx;
+    private final List<Statement> aqlStatements;
+    private Map<TypeSignature, IAType> types;
+    private Map<TypeSignature, TypeDataGen> typeDataGenMap;
 
-	public ADGenDmlTranslator(MetadataTransactionContext mdTxnCtx,
-			List<Statement> aqlStatements) {
-		this.mdTxnCtx = mdTxnCtx;
-		this.aqlStatements = aqlStatements;
-	}
+    public ADGenDmlTranslator(MetadataTransactionContext mdTxnCtx, List<Statement> aqlStatements) {
+        this.mdTxnCtx = mdTxnCtx;
+        this.aqlStatements = aqlStatements;
+    }
 
-	public void translate() throws AsterixException, MetadataException,
-			AlgebricksException {
-		String defaultDataverse = getDefaultDataverse();
-		types = new HashMap<TypeSignature, IAType>();
-		typeDataGenMap = new HashMap<TypeSignature, TypeDataGen>();
+    public void translate() throws AsterixException, MetadataException, AlgebricksException {
+        String defaultDataverse = getDefaultDataverse();
+        types = new HashMap<TypeSignature, IAType>();
+        typeDataGenMap = new HashMap<TypeSignature, TypeDataGen>();
 
-		for (Statement stmt : aqlStatements) {
-			if (stmt.getKind().equals(Statement.Kind.TYPE_DECL)) {
-				TypeDecl td = (TypeDecl) stmt;
-				String typeDataverse = td.getDataverseName() == null ? defaultDataverse
-						: td.getDataverseName().getValue();
+        for (Statement stmt : aqlStatements) {
+            if (stmt.getKind().equals(Statement.Kind.TYPE_DECL)) {
+                TypeDecl td = (TypeDecl) stmt;
+                String typeDataverse = td.getDataverseName() == null ? defaultDataverse : td.getDataverseName()
+                        .getValue();
 
-				Map<TypeSignature, IAType> typeInStmt = TypeTranslator
-						.computeTypes(mdTxnCtx, td, typeDataverse, types);
-				types.putAll(typeInStmt);
+                Map<TypeSignature, IAType> typeInStmt = TypeTranslator.computeTypes(mdTxnCtx, td, typeDataverse, types);
+                types.putAll(typeInStmt);
 
-				TypeSignature signature = new TypeSignature(typeDataverse, td
-						.getIdent().getValue());
-				TypeDataGen tdg = td.getDatagenAnnotation();
-				if (tdg != null) {
-					typeDataGenMap.put(signature, tdg);
-				}
-			}
-		}
-	}
+                TypeSignature signature = new TypeSignature(typeDataverse, td.getIdent().getValue());
+                TypeDataGen tdg = td.getDatagenAnnotation();
+                if (tdg != null) {
+                    typeDataGenMap.put(signature, tdg);
+                }
+            }
+        }
+    }
 
-	private String getDefaultDataverse() {
-		for (Statement stmt : aqlStatements) {
-			if (stmt.getKind().equals(Statement.Kind.DATAVERSE_DECL)) {
-				return ((DataverseDecl) stmt).getDataverseName().getValue();
-			}
-		}
-		return null;
-	}
+    private String getDefaultDataverse() {
+        for (Statement stmt : aqlStatements) {
+            if (stmt.getKind().equals(Statement.Kind.DATAVERSE_DECL)) {
+                return ((DataverseDecl) stmt).getDataverseName().getValue();
+            }
+        }
+        return null;
+    }
 
-	public Map<TypeSignature, IAType> getTypeMap() {
-		return types;
-	}
+    public Map<TypeSignature, IAType> getTypeMap() {
+        return types;
+    }
 
-	public Map<TypeSignature, TypeDataGen> getTypeDataGenMap() {
-		return typeDataGenMap;
-	}
+    public Map<TypeSignature, TypeDataGen> getTypeDataGenMap() {
+        return typeDataGenMap;
+    }
 
 }
diff --git a/asterix-transactions/pom.xml b/asterix-transactions/pom.xml
index bc008cd..3b0f0ed 100644
--- a/asterix-transactions/pom.xml
+++ b/asterix-transactions/pom.xml
@@ -6,9 +6,7 @@
 		<groupId>edu.uci.ics.asterix</groupId>
 		<version>0.0.4-SNAPSHOT</version>
 	</parent>
-	<groupId>edu.uci.ics.asterix</groupId>
 	<artifactId>asterix-transactions</artifactId>
-	<version>0.0.4-SNAPSHOT</version>
 
 	<build>
 		<plugins>
@@ -31,5 +29,21 @@
 			<groupId>edu.uci.ics.hyracks</groupId>
 			<artifactId>hyracks-storage-am-common</artifactId>
 		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-btree</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-rtree</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
+		</dependency>
+		<dependency>
+			<groupId>edu.uci.ics.hyracks</groupId>
+			<artifactId>hyracks-storage-am-lsm-invertedindex</artifactId>
+			<version>0.2.4-SNAPSHOT</version>
+		</dependency>
 	</dependencies>
+
 </project>
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/exception/ACIDException.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/exception/ACIDException.java
index 339e349..2bc9935 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/exception/ACIDException.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/exception/ACIDException.java
@@ -56,5 +56,9 @@
     public ACIDException(String message) {
         super(message);
     }
+    
+    public ACIDException(Throwable cause) {
+        super(cause);
+    }
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java
new file mode 100644
index 0000000..74f39ad
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -0,0 +1,83 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+
+public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
+
+    protected final IndexOperationTracker opTracker;
+
+    public AbstractLSMIOOperationCallback(IndexOperationTracker opTracker) {
+        this.opTracker = opTracker;
+    }
+
+    @Override
+    public void beforeOperation() {
+        // Do nothing.
+    }
+
+    @Override
+    public void afterFinalize(ILSMComponent newComponent) {
+        opTracker.resetLSNs();
+    }
+
+    protected abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
+
+    protected void putLSNIntoMetadata(ITreeIndex treeIndex, List<ILSMComponent> oldComponents)
+            throws HyracksDataException {
+        long componentLSN = getComponentLSN(oldComponents);
+        int fileId = treeIndex.getFileId();
+        IBufferCache bufferCache = treeIndex.getBufferCache();
+        ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+        int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
+        ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
+        metadataPage.acquireWriteLatch();
+        try {
+            metadataFrame.setPage(metadataPage);
+            metadataFrame.setLSN(componentLSN);
+        } finally {
+            metadataPage.releaseWriteLatch();
+            bufferCache.unpin(metadataPage);
+        }
+    }
+
+    protected long getTreeIndexLSN(ITreeIndex treeIndex) throws HyracksDataException {
+        int fileId = treeIndex.getFileId();
+        IBufferCache bufferCache = treeIndex.getBufferCache();
+        ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+        int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
+        ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
+        metadataPage.acquireReadLatch();
+        try {
+            metadataFrame.setPage(metadataPage);
+            return metadataFrame.getLSN();
+        } finally {
+            metadataPage.releaseReadLatch();
+            bufferCache.unpin(metadataPage);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java
new file mode 100644
index 0000000..fb444c3
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+
+public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
+
+    public LSMBTreeIOOperationCallback(IndexOperationTracker opTracker) {
+        super(opTracker);
+    }
+
+    @Override
+    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+            throws HyracksDataException {
+        LSMBTreeImmutableComponent btreeComponent = (LSMBTreeImmutableComponent) newComponent;
+        putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
+    }
+
+    @Override
+    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
+        if (oldComponents == null) {
+            // Implies a flush IO operation.
+            return opTracker.getLastLSN();
+        }
+        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        long maxLSN = -1;
+        for (ILSMComponent c : oldComponents) {
+            BTree btree = ((LSMBTreeImmutableComponent) c).getBTree();
+            maxLSN = Math.max(getTreeIndexLSN(btree), maxLSN);
+        }
+        return maxLSN;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
new file mode 100644
index 0000000..a51da07
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.ioopcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+
+public class LSMBTreeIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    
+    public static LSMBTreeIOOperationCallbackFactory INSTANCE = new LSMBTreeIOOperationCallbackFactory();
+    
+    private LSMBTreeIOOperationCallbackFactory() {
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
+        return new LSMBTreeIOOperationCallback((IndexOperationTracker) syncObj);
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
new file mode 100644
index 0000000..56d4f80
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -0,0 +1,52 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexImmutableComponent;
+
+public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
+
+    public LSMInvertedIndexIOOperationCallback(IndexOperationTracker opTracker) {
+        super(opTracker);
+    }
+
+    @Override
+    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+            throws HyracksDataException {
+        LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) newComponent;
+        putLSNIntoMetadata(invIndexComponent.getDeletedKeysBTree(), oldComponents);
+    }
+
+    @Override
+    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
+        if (oldComponents == null) {
+            // Implies a flush IO operation.
+            return opTracker.getLastLSN();
+        }
+        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        long maxLSN = -1;
+        for (Object o : oldComponents) {
+            LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) o;
+            maxLSN = Math.max(getTreeIndexLSN(invIndexComponent.getDeletedKeysBTree()), maxLSN);
+        }
+        return maxLSN;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
new file mode 100644
index 0000000..790c60c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.ioopcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+
+public class LSMInvertedIndexIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static LSMInvertedIndexIOOperationCallbackFactory INSTANCE = new LSMInvertedIndexIOOperationCallbackFactory();
+    
+    private LSMInvertedIndexIOOperationCallbackFactory() {
+    }
+    
+    @Override
+    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
+        return new LSMInvertedIndexIOOperationCallback((IndexOperationTracker) syncObj);
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java
new file mode 100644
index 0000000..df1d945
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ioopcallbacks;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeImmutableComponent;
+
+public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
+
+    public LSMRTreeIOOperationCallback(IndexOperationTracker opTracker) {
+        super(opTracker);
+    }
+
+    @Override
+    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+            throws HyracksDataException {
+        LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) newComponent;
+        putLSNIntoMetadata(rtreeComponent.getRTree(), oldComponents);
+        putLSNIntoMetadata(rtreeComponent.getBTree(), oldComponents);
+    }
+
+    @Override
+    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
+        if (oldComponents == null) {
+            // Implies a flush IO operation.
+            return opTracker.getLastLSN();
+        }
+        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        long maxLSN = -1;
+        for (Object o : oldComponents) {
+            LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) o;
+            maxLSN = Math.max(getTreeIndexLSN(rtreeComponent.getRTree()), maxLSN);
+        }
+        return maxLSN;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
new file mode 100644
index 0000000..4b47a95
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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.ioopcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+
+public class LSMRTreeIOOperationCallbackFactory implements ILSMIOOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    
+    public static LSMRTreeIOOperationCallbackFactory INSTANCE = new LSMRTreeIOOperationCallbackFactory();
+    
+    private LSMRTreeIOOperationCallbackFactory() {
+    }
+
+    @Override
+    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
+        return new LSMRTreeIOOperationCallback((IndexOperationTracker) syncObj);
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java
new file mode 100644
index 0000000..fdbb707
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallback.java
@@ -0,0 +1,64 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.DatasetId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
+
+public abstract class AbstractOperationCallback {
+    
+    private final static long SEED = 0L;
+    
+    protected final DatasetId datasetId;
+    protected final int[] primaryKeyFields;
+    protected final ILockManager lockManager;
+    protected final TransactionContext txnCtx;
+    protected int transactorLocalNumActiveOperations = 0;
+    protected final long[] longHashes;
+
+    public AbstractOperationCallback(int datasetId, int[] primaryKeyFields,
+            TransactionContext txnCtx, ILockManager lockManager) {
+        this.datasetId = new DatasetId(datasetId);
+        this.primaryKeyFields = primaryKeyFields;
+        this.txnCtx = txnCtx;
+        this.lockManager = lockManager;
+        this.longHashes= new long[2];
+    }
+
+    public int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {
+        MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
+        return Math.abs((int) longHashes[0]); 
+    }
+
+    public TransactionContext getTransactionContext() {
+        return txnCtx;
+    }
+
+    public int getLocalNumActiveOperations() {
+        return transactorLocalNumActiveOperations;
+    }
+
+    public void incrementLocalNumActiveOperations() {
+        transactorLocalNumActiveOperations++;
+    }
+
+    public void decrementLocalNumActiveOperations() {
+        transactorLocalNumActiveOperations--;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java
new file mode 100644
index 0000000..386dce5
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractOperationCallbackFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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.opcallbacks;
+
+import java.io.Serializable;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobId;
+
+public abstract class AbstractOperationCallbackFactory implements Serializable {
+    private static final long serialVersionUID = 1L;
+
+    protected final JobId jobId;
+    protected final int datasetId;
+    protected final int[] primaryKeyFields;
+    protected final ITransactionSubsystemProvider txnSubsystemProvider;
+    protected final byte resourceType;
+
+    public AbstractOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+            ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.primaryKeyFields = primaryKeyFields;
+        this.txnSubsystemProvider = txnSubsystemProvider;
+        this.resourceType = resourceType;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
new file mode 100644
index 0000000..d0071e7
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
@@ -0,0 +1,133 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
+
+public class IndexOperationTracker implements ILSMOperationTracker {
+
+    // Number of active operations on a ILSMIndex instance.
+    private int numActiveOperations = 0;
+    private long lastLSN;
+    private long firstLSN;
+    private final ILSMIndex index;
+    private final ILSMIOOperationCallback ioOpCallback;
+    private ILSMIndexAccessor accessor;
+
+    public IndexOperationTracker(ILSMIndex index, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+        this.index = index;
+        //TODO 
+        //This code is added to avoid NullPointException when the index's comparatorFactory is null.
+        //The null comparator factory is set in the constructor of the IndexDropOperatorDescriptor.
+        if (ioOpCallbackFactory != null) {
+            ioOpCallback = ioOpCallbackFactory.createIOOperationCallback(this);
+        } else {
+            ioOpCallback = NoOpIOOperationCallback.INSTANCE;
+        }
+        resetLSNs();
+    }
+
+    @Override
+    public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        if (opType != LSMOperationType.FORCE_MODIFICATION) {
+            numActiveOperations++;
+
+            // Increment transactor-local active operations count.
+            AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
+            if (opCallback != null) {
+                opCallback.incrementLocalNumActiveOperations();
+            }
+        }
+    }
+
+    @Override
+    public void afterOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        // Searches are immediately considered complete, because they should not prevent the execution of flushes.
+        if (searchCallback != null) {
+            completeOperation(opType, searchCallback, modificationCallback);
+        }
+    }
+
+    @Override
+    public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        numActiveOperations--;
+
+        // Decrement transactor-local active operations count.
+        AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
+        if (opCallback != null) {
+            opCallback.decrementLocalNumActiveOperations();
+        }
+        // If we need a flush, and this is the last completing operation, then schedule the flush.
+        // Once the flush has completed notify all waiting operations.
+        if (index.getFlushStatus(index) && numActiveOperations == 0 && opType != LSMOperationType.FLUSH) {
+            if (accessor == null) {
+                accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+                        NoOpOperationCallback.INSTANCE);
+            }
+            accessor.scheduleFlush(ioOpCallback);
+        }
+    }
+
+    private AbstractOperationCallback getOperationCallback(ISearchOperationCallback searchCallback,
+            IModificationOperationCallback modificationCallback) {
+
+        if (searchCallback == NoOpOperationCallback.INSTANCE || modificationCallback == NoOpOperationCallback.INSTANCE) {
+            return null;
+        }
+        if (searchCallback != null) {
+            return (AbstractOperationCallback) searchCallback;
+        } else {
+            return (AbstractOperationCallback) modificationCallback;
+        }
+    }
+
+    public ILSMIOOperationCallback getIOOperationCallback() {
+        return ioOpCallback;
+    }
+
+    public long getLastLSN() {
+        return lastLSN;
+    }
+
+    public long getFirstLSN() {
+        return firstLSN;
+    }
+
+    public void updateLastLSN(long lastLSN) {
+        if (firstLSN == -1) {
+            firstLSN = lastLSN;
+        }
+        this.lastLSN = Math.max(this.lastLSN, lastLSN);
+    }
+
+    public void resetLSNs() {
+        lastLSN = -1;
+        firstLSN = -1;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java
new file mode 100644
index 0000000..032a4f9
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTrackerFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
+
+public class IndexOperationTrackerFactory implements ILSMOperationTrackerFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
+    
+    public IndexOperationTrackerFactory(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+        this.ioOpCallbackFactory = ioOpCallbackFactory;
+    }
+    
+    @Override
+    public ILSMOperationTracker createOperationTracker(ILSMIndex index) {
+        return new IndexOperationTracker(index, ioOpCallbackFactory);
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallback.java
new file mode 100644
index 0000000..718ea3f
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallback.java
@@ -0,0 +1,61 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+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.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+/**
+ * Assumes LSM-BTrees as primary indexes. Implements try/locking and unlocking on primary keys.
+ */
+public class PrimaryIndexInstantSearchOperationCallback extends AbstractOperationCallback implements
+        ISearchOperationCallback {
+
+    public PrimaryIndexInstantSearchOperationCallback(int datasetId, int[] entityIdFields, ILockManager lockManager,
+            TransactionContext txnCtx) {
+        super(datasetId, entityIdFields, txnCtx, lockManager);
+    }
+
+    @Override
+    public boolean proceed(ITupleReference tuple) throws HyracksDataException {
+        int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
+        try {
+            return lockManager.instantTryLock(datasetId, pkHash, LockMode.S, txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void reconcile(ITupleReference tuple) throws HyracksDataException {
+        int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
+        try {
+            lockManager.instantLock(datasetId, pkHash, LockMode.S, txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void cancel(ITupleReference tuple) throws HyracksDataException {
+        //no op
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
new file mode 100644
index 0000000..3773950
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+
+public class PrimaryIndexInstantSearchOperationCallbackFactory extends AbstractOperationCallbackFactory implements
+        ISearchOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public PrimaryIndexInstantSearchOperationCallbackFactory(JobId jobId, int datasetId, int[] entityIdFields,
+            ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
+        super(jobId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
+    }
+
+    @Override
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+            throws HyracksDataException {
+        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        try {
+            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            return new PrimaryIndexInstantSearchOperationCallback(datasetId, primaryKeyFields,
+                    txnSubsystem.getLockManager(), txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
new file mode 100644
index 0000000..824a324
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -0,0 +1,81 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.tuples.LSMBTreeTupleReference;
+
+/**
+ * Assumes LSM-BTrees as primary indexes.
+ * Performs locking on primary keys, and also logs before/after images.
+ */
+public class PrimaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+        IModificationOperationCallback {
+
+    protected final long resourceId;
+    protected final byte resourceType;
+    protected final IndexOperation indexOp;
+    protected final TransactionSubsystem txnSubsystem;
+
+    public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+            TransactionContext txnCtx, ILockManager lockManager,
+            TransactionSubsystem txnSubsystem, long resourceId, byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager);
+        this.resourceId = resourceId;
+        this.resourceType = resourceType;
+        this.indexOp = indexOp;
+        this.txnSubsystem = txnSubsystem;
+    }
+
+    @Override
+    public void before(ITupleReference tuple) throws HyracksDataException {
+        int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
+        try {
+            lockManager.lock(datasetId, pkHash, LockMode.X, txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
+        IndexLogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+        int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
+        LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
+        IndexOperation oldOp = IndexOperation.INSERT;
+        if (before == null) {
+            oldOp = IndexOperation.NOOP;
+        }
+        if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
+            oldOp = IndexOperation.DELETE;
+        }
+        try {
+            logger.generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
+                    oldOp, before);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
new file mode 100644
index 0000000..c75ab6f
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -0,0 +1,69 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+/**
+ * Assumes LSM-BTrees as primary indexes.
+ */
+public class PrimaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
+        IModificationOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final IndexOperation indexOp;
+
+    public PrimaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+            ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+        super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
+        this.indexOp = indexOp;
+    }
+
+    @Override
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+
+        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+                .getIndexLifecycleManager();
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        if (index == null) {
+            throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
+        }
+
+        try {
+            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
+                    primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
+                    indexOp);
+            txnCtx.registerIndexAndCallback(index, (AbstractOperationCallback) modCallback);
+            return modCallback;
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java
new file mode 100644
index 0000000..4760307
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallback.java
@@ -0,0 +1,65 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+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.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+/**
+ * Assumes LSM-BTrees as primary indexes. Implements try/locking and unlocking on primary keys.
+ */
+public class PrimaryIndexSearchOperationCallback extends AbstractOperationCallback implements ISearchOperationCallback {
+
+    public PrimaryIndexSearchOperationCallback(int datasetId, int[] entityIdFields,
+            ILockManager lockManager, TransactionContext txnCtx) {
+        super(datasetId, entityIdFields, txnCtx, lockManager);
+    }
+
+    @Override
+    public boolean proceed(ITupleReference tuple) throws HyracksDataException {
+        int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
+        try {
+            return lockManager.tryLock(datasetId, pkHash, LockMode.S, txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void reconcile(ITupleReference tuple) throws HyracksDataException {
+        int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
+        try {
+            lockManager.lock(datasetId, pkHash, LockMode.S, txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void cancel(ITupleReference tuple) throws HyracksDataException {
+        int pkHash = computePrimaryKeyHashValue(tuple, primaryKeyFields);
+        try {
+            lockManager.unlock(datasetId, pkHash, txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
new file mode 100644
index 0000000..fc62b90
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -0,0 +1,51 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+
+public class PrimaryIndexSearchOperationCallbackFactory extends AbstractOperationCallbackFactory implements
+        ISearchOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public PrimaryIndexSearchOperationCallbackFactory(JobId jobId, int datasetId, int[] entityIdFields,
+            ITransactionSubsystemProvider txnSubsystemProvider, byte resourceType) {
+        super(jobId, datasetId, entityIdFields, txnSubsystemProvider, resourceType);
+    }
+
+    @Override
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+            throws HyracksDataException {
+        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        try {
+            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            return new PrimaryIndexSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
+                    txnCtx);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
new file mode 100644
index 0000000..092f99c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -0,0 +1,69 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+
+/**
+ * Secondary-index modifications do not require any locking.
+ * We assume that the modification of the corresponding primary index has already taken an appropriate lock.
+ * This callback performs logging of the before and/or after images for secondary indexes.
+ */
+public class SecondaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+        IModificationOperationCallback {
+
+    protected final long resourceId;
+    protected final byte resourceType;
+    protected final IndexOperation indexOp;
+    protected final IndexOperation oldOp;
+    protected final TransactionSubsystem txnSubsystem;
+
+    public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+            TransactionContext txnCtx, ILockManager lockManager,
+            TransactionSubsystem txnSubsystem, long resourceId, byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager);
+        this.resourceId = resourceId;
+        this.resourceType = resourceType;
+        this.indexOp = indexOp;
+        oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
+        this.txnSubsystem = txnSubsystem;
+    }
+
+    @Override
+    public void before(ITupleReference tuple) throws HyracksDataException {
+        // Do nothing.
+    }
+
+    @Override
+    public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
+        IndexLogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
+        int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
+        try {
+            logger.generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId, indexOp, after,
+                    oldOp, before);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
new file mode 100644
index 0000000..672b434
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -0,0 +1,63 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.ITransactionSubsystemProvider;
+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.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+public class SecondaryIndexModificationOperationCallbackFactory extends AbstractOperationCallbackFactory implements
+        IModificationOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+    private final IndexOperation indexOp;
+
+    public SecondaryIndexModificationOperationCallbackFactory(JobId jobId, int datasetId, int[] primaryKeyFields,
+            ITransactionSubsystemProvider txnSubsystemProvider, IndexOperation indexOp, byte resourceType) {
+        super(jobId, datasetId, primaryKeyFields, txnSubsystemProvider, resourceType);
+        this.indexOp = indexOp;
+    }
+
+    @Override
+    public IModificationOperationCallback createModificationOperationCallback(long resourceId, Object resource,
+            IHyracksTaskContext ctx) throws HyracksDataException {
+
+        TransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
+        IIndexLifecycleManager indexLifeCycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+                .getIndexLifecycleManager();
+        ILSMIndex index = (ILSMIndex) indexLifeCycleManager.getIndex(resourceId);
+        if (index == null) {
+            throw new HyracksDataException("Index(id:" + resourceId + ") is not registered.");
+        }
+
+        try {
+            TransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            return new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
+                    txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
+        } catch (ACIDException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java
new file mode 100644
index 0000000..c53b651
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallback.java
@@ -0,0 +1,47 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+
+/**
+ * Secondary index searches perform no locking at all.
+ */
+public class SecondaryIndexSearchOperationCallback extends AbstractOperationCallback implements
+        ISearchOperationCallback {
+
+    public SecondaryIndexSearchOperationCallback() {
+        super(-1, null, null, null);
+    }
+
+    @Override
+    public boolean proceed(ITupleReference tuple) throws HyracksDataException {
+        return true;
+    }
+
+    @Override
+    public void reconcile(ITupleReference tuple) throws HyracksDataException {
+        // Do nothing.
+    }
+
+    @Override
+    public void cancel(ITupleReference tuple) throws HyracksDataException {
+        // Do nothing.
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
new file mode 100644
index 0000000..7172e06
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexSearchOperationCallbackFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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.opcallbacks;
+
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
+
+public class SecondaryIndexSearchOperationCallbackFactory implements ISearchOperationCallbackFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public ISearchOperationCallback createSearchOperationCallback(long resourceId, IHyracksTaskContext ctx)
+            throws HyracksDataException {
+        return new SecondaryIndexSearchOperationCallback();
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java
new file mode 100644
index 0000000..d2aa164
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/ILocalResourceMetadata.java
@@ -0,0 +1,14 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import java.io.Serializable;
+
+import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+public interface ILocalResourceMetadata extends Serializable {
+
+    public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
+            int partition) throws HyracksDataException;
+    
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
new file mode 100644
index 0000000..e037e95
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -0,0 +1,58 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import java.io.File;
+
+import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+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.api.ILSMIndex;
+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.file.TransientFileMapManager;
+
+public class LSMBTreeLocalResourceMetadata implements ILocalResourceMetadata {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ITypeTraits[] typeTraits;
+    private final IBinaryComparatorFactory[] cmpFactories;
+    private final int[] bloomFilterKeyFields;
+    private final int memPageSize;
+    private final int memNumPages;
+
+    public LSMBTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] cmpFactories,
+            int[] bloomFilterKeyFields, boolean isPrimary, int memPageSize, int memNumPages) {
+        this.typeTraits = typeTraits;
+        this.cmpFactories = cmpFactories;
+        this.bloomFilterKeyFields = bloomFilterKeyFields;
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+    }
+
+    @Override
+    public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
+            int partition) {
+        FileReference file = new FileReference(new File(filePath));
+        IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
+                memNumPages, new TransientFileMapManager());
+        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+        IInMemoryFreePageManager memFreePageManager = new InMemoryFreePageManager(memNumPages, metaDataFrameFactory);
+        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager,
+                runtimeContextProvider.getIOManager(), file, runtimeContextProvider.getBufferCache(),
+                runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories, bloomFilterKeyFields,
+                runtimeContextProvider.getLSMMergePolicy(),
+                runtimeContextProvider.getLSMBTreeOperationTrackerFactory(),
+                runtimeContextProvider.getLSMIOScheduler(),
+                runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(), partition);
+        return lsmBTree;
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
new file mode 100644
index 0000000..894fc16
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -0,0 +1,77 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.tokenizers.IBinaryTokenizerFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.util.InvertedIndexUtils;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+
+public class LSMInvertedIndexLocalResourceMetadata implements ILocalResourceMetadata {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ITypeTraits[] invListTypeTraits;
+    private final IBinaryComparatorFactory[] invListCmpFactories;
+    private final ITypeTraits[] tokenTypeTraits;
+    private final IBinaryComparatorFactory[] tokenCmpFactories;
+    private final IBinaryTokenizerFactory tokenizerFactory;
+    private final int memPageSize;
+    private final int memNumPages;
+    private final boolean isPartitioned;
+
+    public LSMInvertedIndexLocalResourceMetadata(ITypeTraits[] invListTypeTraits,
+            IBinaryComparatorFactory[] invListCmpFactories, ITypeTraits[] tokenTypeTraits,
+            IBinaryComparatorFactory[] tokenCmpFactories, IBinaryTokenizerFactory tokenizerFactory, int memPageSize,
+            int memNumPages, boolean isPartitioned) {
+        this.invListTypeTraits = invListTypeTraits;
+        this.invListCmpFactories = invListCmpFactories;
+        this.tokenTypeTraits = tokenTypeTraits;
+        this.tokenCmpFactories = tokenCmpFactories;
+        this.tokenizerFactory = tokenizerFactory;
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+        this.isPartitioned = isPartitioned;
+    }
+
+    @Override
+    public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
+            int partition) throws HyracksDataException {
+
+        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+        IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
+                memNumPages);
+        IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
+                metaDataFrameFactory);
+        try {
+            if (isPartitioned) {
+                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(memBufferCache, memFreePageManager,
+                        runtimeContextProvider.getFileMapManager(), invListTypeTraits, invListCmpFactories,
+                        tokenTypeTraits, tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
+                        runtimeContextProvider.getIOManager(), filePath, runtimeContextProvider.getLSMMergePolicy(),
+                        runtimeContextProvider.getLSMInvertedIndexOperationTrackerFactory(),
+                        runtimeContextProvider.getLSMIOScheduler(),
+                        runtimeContextProvider.getLSMInvertedIndexIOOperationCallbackProvider(), partition);
+            } else {
+                return InvertedIndexUtils.createLSMInvertedIndex(memBufferCache, memFreePageManager,
+                        runtimeContextProvider.getFileMapManager(), invListTypeTraits, invListCmpFactories,
+                        tokenTypeTraits, tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(),
+                        runtimeContextProvider.getIOManager(), filePath, runtimeContextProvider.getLSMMergePolicy(),
+                        runtimeContextProvider.getLSMInvertedIndexOperationTrackerFactory(),
+                        runtimeContextProvider.getLSMIOScheduler(),
+                        runtimeContextProvider.getLSMInvertedIndexIOOperationCallbackProvider(), partition);
+            }
+        } catch (IndexException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
new file mode 100644
index 0000000..1705dd3
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -0,0 +1,73 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import java.io.File;
+
+import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
+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.ITypeTraits;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.FileReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IPrimitiveValueProviderFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.common.api.TreeIndexException;
+import edu.uci.ics.hyracks.storage.am.common.frames.LIFOMetaDataFrameFactory;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.IInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.freepage.DualIndexInMemoryFreePageManager;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.utils.LSMRTreeUtils;
+import edu.uci.ics.hyracks.storage.am.rtree.frames.RTreePolicyType;
+import edu.uci.ics.hyracks.storage.common.buffercache.HeapBufferAllocator;
+
+public class LSMRTreeLocalResourceMetadata implements ILocalResourceMetadata {
+
+    private static final long serialVersionUID = 1L;
+
+    private final ITypeTraits[] typeTraits;
+    private final IBinaryComparatorFactory[] rtreeCmpFactories;
+    private final IBinaryComparatorFactory[] btreeCmpFactories;
+    private final IPrimitiveValueProviderFactory[] valueProviderFactories;
+    private final RTreePolicyType rtreePolicyType;
+    private final ILinearizeComparatorFactory linearizeCmpFactory;
+    private final int memPageSize;
+    private final int memNumPages;
+
+    public LSMRTreeLocalResourceMetadata(ITypeTraits[] typeTraits, IBinaryComparatorFactory[] rtreeCmpFactories,
+            IBinaryComparatorFactory[] btreeCmpFactories, IPrimitiveValueProviderFactory[] valueProviderFactories,
+            RTreePolicyType rtreePolicyType, ILinearizeComparatorFactory linearizeCmpFactory, int memPageSize,
+            int memNumPages) {
+        this.typeTraits = typeTraits;
+        this.rtreeCmpFactories = rtreeCmpFactories;
+        this.btreeCmpFactories = btreeCmpFactories;
+        this.valueProviderFactories = valueProviderFactories;
+        this.rtreePolicyType = rtreePolicyType;
+        this.linearizeCmpFactory = linearizeCmpFactory;
+        this.memPageSize = memPageSize;
+        this.memNumPages = memNumPages;
+    }
+
+    @Override
+    public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
+            int partition) throws HyracksDataException {
+        FileReference file = new FileReference(new File(filePath));
+        ITreeIndexMetaDataFrameFactory metaDataFrameFactory = new LIFOMetaDataFrameFactory();
+        IInMemoryBufferCache memBufferCache = new DualIndexInMemoryBufferCache(new HeapBufferAllocator(), memPageSize,
+                memNumPages);
+        IInMemoryFreePageManager memFreePageManager = new DualIndexInMemoryFreePageManager(memNumPages,
+                metaDataFrameFactory);
+
+        try {
+            return LSMRTreeUtils.createLSMTree(memBufferCache, memFreePageManager,
+                    runtimeContextProvider.getIOManager(), file, runtimeContextProvider.getBufferCache(),
+                    runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
+                    valueProviderFactories, rtreePolicyType, runtimeContextProvider.getLSMMergePolicy(),
+                    runtimeContextProvider.getLSMRTreeOperationTrackerFactory(),
+                    runtimeContextProvider.getLSMIOScheduler(),
+                    runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(), linearizeCmpFactory, partition);
+        } catch (TreeIndexException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
new file mode 100644
index 0000000..ed0f79f
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactory.java
@@ -0,0 +1,20 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
+
+public class PersistentLocalResourceFactory implements ILocalResourceFactory {
+    
+    private final ILocalResourceMetadata localResourceMetadata;
+    private final int resourceType;
+
+    public PersistentLocalResourceFactory(ILocalResourceMetadata localResourceMetadata, int resourceType) {
+        this.localResourceMetadata = localResourceMetadata;
+        this.resourceType = resourceType;
+    }
+
+    @Override
+    public LocalResource createLocalResource(long resourceId, String resourceName, int partition) {
+        return new LocalResource(resourceId, resourceName, partition, resourceType, localResourceMetadata);
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
new file mode 100644
index 0000000..4157c32
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceFactoryProvider.java
@@ -0,0 +1,21 @@
+package edu.uci.ics.asterix.transaction.management.resource;
+
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactory;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceFactoryProvider;
+
+public class PersistentLocalResourceFactoryProvider implements ILocalResourceFactoryProvider {
+
+    private static final long serialVersionUID = 1L;
+    private final ILocalResourceMetadata localResourceMetadata;
+    private final int resourceType;
+
+    public PersistentLocalResourceFactoryProvider(ILocalResourceMetadata localResourceMetadata, int resourceType) {
+        this.localResourceMetadata = localResourceMetadata;
+        this.resourceType = resourceType;
+    }
+
+    @Override
+    public ILocalResourceFactory getLocalResourceFactory() {
+        return new PersistentLocalResourceFactory(localResourceMetadata, resourceType);
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
new file mode 100644
index 0000000..838dc6d
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepository.java
@@ -0,0 +1,256 @@
+/*
+ * 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.resource;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
+import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
+
+public class PersistentLocalResourceRepository implements ILocalResourceRepository {
+
+    private final String mountPoint;
+    private static final String ROOT_METADATA_DIRECTORY = "asterix_root_metadata/";
+    private static final String ROOT_METADATA_FILE_NAME_PREFIX = ".asterix_root_metadata_";
+    private static final long ROOT_LOCAL_RESOURCE_ID = -4321;
+    private static final String METADATA_FILE_NAME = ".metadata";
+    private Map<String, LocalResource> name2ResourceMap = new HashMap<String, LocalResource>();
+    private Map<Long, LocalResource> id2ResourceMap = new HashMap<Long, LocalResource>();
+    private String rootMetadataFileName;
+    private String rootDir;
+
+    public PersistentLocalResourceRepository(String mountPoint) throws HyracksDataException {
+        File mountPointDir = new File(mountPoint);
+        if (!mountPointDir.exists()) {
+            throw new HyracksDataException(mountPointDir.getAbsolutePath() + "doesn't exist.");
+        }
+        if (!mountPoint.endsWith(System.getProperty("file.separator"))) {
+            this.mountPoint = new String(mountPoint + System.getProperty("file.separator"));
+        } else {
+            this.mountPoint = new String(mountPoint);
+        }
+    }
+
+    public void initialize(String nodeId, String rootDir, boolean isNewUniverse, ResourceIdFactory resourceIdFactory)
+            throws HyracksDataException {
+        LocalResource rootLocalResource = null;
+
+        //#. if the rootMetadataFile doesn't exist, create it and return.
+        rootMetadataFileName = new String(mountPoint + ROOT_METADATA_DIRECTORY + ROOT_METADATA_FILE_NAME_PREFIX
+                + nodeId);
+        File rootMetadataFile = new File(rootMetadataFileName);
+        if (isNewUniverse) {
+            File rootMetadataDir = new File(mountPoint + ROOT_METADATA_DIRECTORY);
+            if (!rootMetadataDir.exists()) {
+                rootMetadataDir.mkdir();
+            }
+
+            rootMetadataFile.delete();
+            if (rootDir.startsWith(System.getProperty("file.separator"))) {
+                this.rootDir = new String(mountPoint + rootDir.substring(System.getProperty("file.separator").length()));
+            } else {
+                this.rootDir = new String(mountPoint + rootDir);
+            }
+            rootLocalResource = new LocalResource(ROOT_LOCAL_RESOURCE_ID, rootMetadataFileName, 0, 0, this.rootDir);
+            insert(rootLocalResource);
+            return;
+        }
+
+        //#. if the rootMetadataFile exists, read it and set this.rootDir.
+        rootLocalResource = readLocalResource(rootMetadataFile);
+        this.rootDir = (String) rootLocalResource.getResourceObject();
+
+        //#. load all local resources. 
+        File rootDirFile = new File(this.rootDir);
+        if (!rootDirFile.exists()) {
+            throw new HyracksDataException(rootDirFile.getAbsolutePath() + "doesn't exist.");
+        }
+
+        FilenameFilter filter = new FilenameFilter() {
+            public boolean accept(File dir, String name) {
+                if (name.equalsIgnoreCase(METADATA_FILE_NAME)) {
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        long maxResourceId = 0;
+        File[] dataverseFileList = rootDirFile.listFiles();
+        if (dataverseFileList == null) {
+            throw new HyracksDataException("Metadata dataverse doesn't exist.");
+        }
+        for (File dataverseFile : dataverseFileList) {
+            if (dataverseFile.isDirectory()) {
+                File[] indexFileList = dataverseFile.listFiles();
+                if (indexFileList != null) {
+                    for (File indexFile : indexFileList) {
+                        if (indexFile.isDirectory()) {
+                            File[] metadataFiles = indexFile.listFiles(filter);
+                            if (metadataFiles != null) {
+                                for (File metadataFile : metadataFiles) {
+                                    LocalResource localResource = readLocalResource(metadataFile);
+                                    id2ResourceMap.put(localResource.getResourceId(), localResource);
+                                    name2ResourceMap.put(localResource.getResourceName(), localResource);
+                                    maxResourceId = Math.max(localResource.getResourceId(), maxResourceId);
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        }
+        resourceIdFactory.initId(maxResourceId + 1);
+    }
+
+    @Override
+    public LocalResource getResourceById(long id) throws HyracksDataException {
+        return id2ResourceMap.get(id);
+    }
+
+    @Override
+    public LocalResource getResourceByName(String name) throws HyracksDataException {
+        return name2ResourceMap.get(name);
+    }
+
+    @Override
+    public synchronized void insert(LocalResource resource) throws HyracksDataException {
+        long id = resource.getResourceId();
+
+        if (id2ResourceMap.containsKey(id)) {
+            throw new HyracksDataException("Duplicate resource");
+        }
+
+        if (resource.getResourceId() != ROOT_LOCAL_RESOURCE_ID) {
+            id2ResourceMap.put(id, resource);
+            name2ResourceMap.put(resource.getResourceName(), resource);
+        }
+
+        FileOutputStream fos = null;
+        ObjectOutputStream oosToFos = null;
+        try {
+            fos = new FileOutputStream(getFileName(mountPoint, resource.getResourceName(), resource.getResourceId()));
+            oosToFos = new ObjectOutputStream(fos);
+            oosToFos.writeObject(resource);
+            oosToFos.flush();
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        } finally {
+            if (oosToFos != null) {
+                try {
+                    oosToFos.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+            if (oosToFos == null && fos != null) {
+                try {
+                    fos.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        }
+    }
+
+    @Override
+    public synchronized void deleteResourceById(long id) throws HyracksDataException {
+        LocalResource resource = id2ResourceMap.get(id);
+        if (resource == null) {
+            throw new HyracksDataException("Resource doesn't exist");
+        }
+        id2ResourceMap.remove(id);
+        name2ResourceMap.remove(resource.getResourceName());
+        File file = new File(getFileName(mountPoint, resource.getResourceName(), resource.getResourceId()));
+        file.delete();
+    }
+
+    @Override
+    public synchronized void deleteResourceByName(String name) throws HyracksDataException {
+        LocalResource resource = name2ResourceMap.get(name);
+        if (resource == null) {
+            throw new HyracksDataException("Resource doesn't exist");
+        }
+        id2ResourceMap.remove(resource.getResourceId());
+        name2ResourceMap.remove(name);
+        File file = new File(getFileName(mountPoint, resource.getResourceName(), resource.getResourceId()));
+        file.delete();
+    }
+
+    @Override
+    public List<LocalResource> getAllResources() throws HyracksDataException {
+        List<LocalResource> resources = new ArrayList<LocalResource>();
+        for (LocalResource resource : id2ResourceMap.values()) {
+            resources.add(resource);
+        }
+        return resources;
+    }
+
+    private String getFileName(String mountPoint, String baseDir, long resourceId) {
+
+        if (resourceId == ROOT_LOCAL_RESOURCE_ID) {
+            return baseDir;
+        } else {
+            String fileName = new String(mountPoint);
+            if (!baseDir.endsWith(System.getProperty("file.separator"))) {
+                baseDir += System.getProperty("file.separator");
+            }
+            fileName += baseDir + METADATA_FILE_NAME;
+            return fileName;
+        }
+    }
+
+    private LocalResource readLocalResource(File file) throws HyracksDataException {
+        FileInputStream fis = null;
+        ObjectInputStream oisFromFis = null;
+
+        try {
+            fis = new FileInputStream(file);
+            oisFromFis = new ObjectInputStream(fis);
+            LocalResource resource = (LocalResource) oisFromFis.readObject();
+            return resource;
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        } finally {
+            if (oisFromFis != null) {
+                try {
+                    oisFromFis.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+            if (oisFromFis == null && fis != null) {
+                try {
+                    fis.close();
+                } catch (IOException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
new file mode 100644
index 0000000..c7efca5
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/PersistentLocalResourceRepositoryFactory.java
@@ -0,0 +1,37 @@
+/*
+ * 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.resource;
+
+import java.util.List;
+
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.api.io.IODeviceHandle;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepositoryFactory;
+
+public class PersistentLocalResourceRepositoryFactory implements ILocalResourceRepositoryFactory {
+    private IIOManager ioManager;
+
+    public PersistentLocalResourceRepositoryFactory(IIOManager ioManager) {
+        this.ioManager = ioManager;
+    }
+
+    @Override
+    public ILocalResourceRepository createRepository() throws HyracksDataException {
+        List<IODeviceHandle> devices = ioManager.getIODevices();
+        return new PersistentLocalResourceRepository(devices.get(0).getPath().getPath());
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceManagerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceManagerRepository.java
new file mode 100644
index 0000000..d38226b
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceManagerRepository.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2010 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.resource;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+
+/**
+ * Represents a repository containing Resource Managers and Resources in the
+ * transaction eco-system. Operations on a resource require acquiring
+ * appropriate locks (for isolation) and writing logs (durability). Every
+ * resource is managed by an associated resource manager that contains the logic
+ * to interpret the logs and take necessary action(s) during roll back or
+ * recovery. An example of resource is a @see ITreeIndex that is managed by a
+ * resource manager @see TreeResourceManager
+ */
+public class TransactionalResourceManagerRepository {
+
+    private Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
+
+    public void registerTransactionalResourceManager(byte id, IResourceManager resourceMgr) {
+        synchronized (resourceMgrRepository) {
+            if (resourceMgrRepository.get(id) == null) {
+                resourceMgrRepository.put(id, resourceMgr);
+            }
+        }
+    }
+
+    public IResourceManager getTransactionalResourceMgr(byte id) {
+        synchronized (resourceMgrRepository) {
+            return resourceMgrRepository.get(id);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
deleted file mode 100644
index 25309e1..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/TransactionalResourceRepository.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Copyright 2009-2010 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.resource;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-
-/**
- * Represents a repository containing Resource Managers and Resources in the
- * transaction eco-system. Operations on a resource require acquiring
- * appropriate locks (for isolation) and writing logs (durability). Every
- * resource is managed by an associated resource manager that contains the logic
- * to interpret the logs and take necessary action(s) during roll back or
- * recovery. An example of resource is a @see ITreeIndex that is managed by a
- * resource manager @see TreeResourceManager
- */
-public class TransactionalResourceRepository {
-
-    private Map<ByteBuffer, Object> resourceRepository = new HashMap<ByteBuffer, Object>(); // repository
-
-    private Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
-
-    public void registerTransactionalResource(byte[] resourceBytes, Object resource) {
-        // convert to ByteBuffer so that a byte[] can be used as a key in a hash map.
-        ByteBuffer resourceId = ByteBuffer.wrap(resourceBytes); // need to
-
-        synchronized (resourceRepository) {
-            if (resourceRepository.get(resourceId) == null) {
-                resourceRepository.put(resourceId, resource);
-                
-                // wake up threads waiting for the resource
-                resourceRepository.notifyAll();
-            }
-        }
-    }
-
-    public void registerTransactionalResourceManager(byte id, IResourceManager resourceMgr) {
-        synchronized (resourceMgrRepository) {
-            if (resourceMgrRepository.get(id) == null) {
-                resourceMgrRepository.put(id, resourceMgr);
-                
-                // wake up threads waiting for the resource manager
-                resourceMgrRepository.notifyAll();
-            }
-        }
-    }
-
-    public Object getTransactionalResource(byte[] resourceIdBytes) {
-        ByteBuffer buffer = ByteBuffer.wrap(resourceIdBytes);
-        synchronized (resourceRepository) {
-            while (resourceRepository.get(buffer) == null) {
-                try {
-                    resourceRepository.wait();
-                } catch (InterruptedException ie) {
-                    ie.printStackTrace();
-                    break; // the thread might be interrupted due to other
-                    // failures occurring elsewhere, break from the loop
-                }
-            }
-            return resourceRepository.get(buffer);
-        }
-    }
-
-    public IResourceManager getTransactionalResourceMgr(byte id) {
-        synchronized (resourceMgrRepository) {
-            return resourceMgrRepository.get(id);
-        }
-
-    }
-
-}
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..900725b 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 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);
+    public boolean isSafeToAdd(DatasetLockInfo dLockInfo, int eLockInfo, int entityInfo, boolean isDatasetLockInfo,
+            boolean isUpgrade) {
+        int holder;
+        int visitedHolder;
+        int callerId = entityInfoManager.getJobId(entityInfo);
+        int datasetId = entityInfoManager.getDatasetId(entityInfo);
+        int hashValue = entityInfoManager.getPKHashVal(entityInfo);
+        int resource;
+        PrimitiveIntHashMap tempHolderList;
+
+        holderList.clear(true);
+        visitedHolderList.clear(true);
+
+        //holderlist contains jobId
+        //resourceList contains entityInfo's slot numbers instead of resourceId in order to avoid object creation 
+        //since resourceId consists of datasetId and PKHashValue
+
+        //get holder list(jobId list)
+        if (isDatasetLockInfo) {
+            getHolderList(datasetId, -1, holderList);
+        } else {
+            getHolderList(datasetId, hashValue, holderList);
         }
+
+        //check whether this caller is upgrader or not
+        //if it is upgrader, then handle it as special case in the following manner
+        //if there is another upgrader or waiter of which lock mode is not compatible with the caller's lock mode,
+        //then this caller's wait causes deadlock.
+        if (holderList.get(callerId) != -1) {
+            if (isUpgrade && dLockInfo.getFirstUpgrader() != -1) {
+                return false;
+            }
+            //there is no case such that while a job is holding any mode of lock on a dataset and waits for the same dataset as an waiter. 
+            //But the job may wait for the same dataset as an upgrader.
+        }
+
+        //TODO
+        //check whether when there are multiple resources, the waiter and upgrader should be distinguished or not.
+        //The current logic doesn't distinguish these two types of waiter.
+
+        //while holderList is not empty
+        holderList.beginIterate();
+        holder = holderList.getNextKey();
+        while (holder != -1) {
+
+            nextHolderList.clear(true);
+
+            while (holder != -1) {
+                resourceList.clear(true);
+                getWaitingResourceList(holder, resourceList);
+                resourceList.beginIterate();
+                resource = resourceList.getNextKey();
+
+                while (resource != -1) {
+                    //get dataset holder
+                    getHolderList(entityInfoManager.getDatasetId(resource), -1, nextHolderList);
+                    //get entity holder
+                    getHolderList(entityInfoManager.getDatasetId(resource), entityInfoManager.getPKHashVal(resource),
+                            nextHolderList);
+                    if (nextHolderList.get(callerId) != -1) {
+                        return false;
+                    }
+                    resource = resourceList.getNextKey();
+                }
+
+                visitedHolderList.put(holder, -1);
+                holder = holderList.getNextKey();
+            }
+
+            //remove visitedHolder for nextHolderList;
+            visitedHolderList.beginIterate();
+            visitedHolder = visitedHolderList.getNextKey();
+            while (visitedHolder != -1) {
+                nextHolderList.remove(visitedHolder);
+                visitedHolder = visitedHolderList.getNextKey();
+            }
+
+            //swap holder list
+            //set holderList to nextHolderList and nextHolderList to holderList
+            tempHolderList = holderList;
+            holderList = nextHolderList;
+            nextHolderList = tempHolderList;
+            holderList.beginIterate();
+            holder = holderList.getNextKey();
+        }
+
         return true;
     }
 
-    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..59c20f2
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/EntityLockInfoManager.java
@@ -0,0 +1,801 @@
+/*
+ * 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.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..1341cc1 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,72 @@
      * 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;
+    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) 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
+     * @param datasetId
+     * @param entityHashValue
+     * @param txnContext
+     * @throws ACIDException TODO
      */
-    public boolean convertLock(TransactionContext context, byte[] resourceID, int mode) throws ACIDException;
-
+    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext, boolean commitFlag) 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..75ff349
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/JobInfo.java
@@ -0,0 +1,305 @@
+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 datasetISLockHT; //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;
+        if (LockManager.ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+            //This table maintains the number of locks acquired by this jobInfo.
+            //[Notice] But this doesn't decrease the count even if the lock is released.
+            this.datasetISLockHT = new PrimitiveIntHashMap(1 << 4, 1 << 2, Integer.MAX_VALUE);
+        }
+    }
+
+    public void addHoldingResource(int resource) {
+
+        if (LockManager.IS_DEBUG_MODE) {
+            if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
+                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+                        + entityInfoManager.getJobId(resource) + "'s resource!!!");
+            }
+            //System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddHolder:"+ resource);
+        }
+
+        if (lastHoldingResource != -1) {
+            entityInfoManager.setNextJobResource(lastHoldingResource, resource);
+        }
+        entityInfoManager.setPrevJobResource(resource, lastHoldingResource);
+        entityInfoManager.setNextJobResource(resource, -1);
+        lastHoldingResource = resource;
+    }
+
+    public void removeHoldingResource(int resource) {
+        int current = lastHoldingResource;
+        int prev;
+        int next;
+
+        if (LockManager.IS_DEBUG_MODE) {
+            if (entityInfoManager.getJobId(resource) != jobCtx.getJobId().getId()) {
+                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+                        + entityInfoManager.getJobId(resource) + "'s resource!!!");
+            }
+            //System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveHolder:"+ resource);
+        }
+
+        while (current != resource) {
+
+            if (LockManager.IS_DEBUG_MODE) {
+                if (current == -1) {
+                    //shouldn't occur: debugging purpose
+                    try {
+                        throw new Exception();
+                    } catch (Exception e) {
+                        // TODO Auto-generated catch block
+                        e.printStackTrace();
+                    }
+                }
+            }
+
+            current = entityInfoManager.getPrevJobResource(current);
+        }
+
+        prev = entityInfoManager.getPrevJobResource(current);
+        next = entityInfoManager.getNextJobResource(current);
+        //update prev->next = next
+        if (prev != -1) {
+            entityInfoManager.setNextJobResource(prev, next);
+        }
+        if (next != -1) {
+            entityInfoManager.setPrevJobResource(next, prev);
+        }
+        if (lastHoldingResource == resource) {
+            lastHoldingResource = prev;
+        }
+
+        //decreaseDatasetLockCount(resource);
+    }
+
+    public void addWaitingResource(int waiterObjId) {
+        int lastObjId;
+        LockWaiter lastObj = null;
+
+        if (firstWaitingResource != -1) {
+            //find the lastWaiter
+            lastObjId = firstWaitingResource;
+            while (lastObjId != -1) {
+                lastObj = lockWaiterManager.getLockWaiter(lastObjId);
+                if (LockManager.IS_DEBUG_MODE) {
+                    int entityInfo = lastObj.getEntityInfoSlot();
+                    if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
+                        throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId()
+                                + ") has diffrent Job(JID:" + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
+                    }
+                }
+                lastObjId = lastObj.getNextWaitingResourceObjId();
+            }
+            //last->next = new_waiter
+            lastObj.setNextWaitingResourceObjId(waiterObjId);
+        } else {
+            firstWaitingResource = waiterObjId;
+        }
+        //new_waiter->next = -1
+        lastObj = lockWaiterManager.getLockWaiter(waiterObjId);
+        if (LockManager.IS_DEBUG_MODE) {
+            int entityInfo = lastObj.getEntityInfoSlot();
+            if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
+                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+                        + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
+            }
+        }
+        lastObj.setNextWaitingResourceObjId(-1);
+
+        //        if (LockManager.IS_DEBUG_MODE) {
+        //            System.out.println(Thread.currentThread().getName()+"\tJobInfo_AddWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);            
+        //        }
+    }
+
+    public void removeWaitingResource(int waiterObjId) {
+        int currentObjId = firstWaitingResource;
+        LockWaiter currentObj;
+        LockWaiter prevObj = null;
+        int prevObjId = -1;
+        int nextObjId;
+
+        while (currentObjId != waiterObjId) {
+
+            if (LockManager.IS_DEBUG_MODE) {
+                if (currentObjId == -1) {
+                    //shouldn't occur: debugging purpose
+                    try {
+                        throw new Exception();
+                    } catch (Exception e) {
+                        // TODO Auto-generated catch block
+                        e.printStackTrace();
+                    }
+                }
+            }
+
+            prevObj = lockWaiterManager.getLockWaiter(currentObjId);
+            prevObjId = currentObjId;
+            currentObjId = prevObj.getNextWaitingResourceObjId();
+        }
+
+        //get current waiter object
+        currentObj = lockWaiterManager.getLockWaiter(currentObjId);
+
+        if (LockManager.IS_DEBUG_MODE) {
+            int entityInfo = currentObj.getEntityInfoSlot();
+            if (entityInfoManager.getJobId(entityInfo) != jobCtx.getJobId().getId()) {
+                throw new IllegalStateException("JobInfo(" + jobCtx.getJobId().getId() + ") has diffrent Job(JID:"
+                        + entityInfoManager.getJobId(entityInfo) + "'s resource!!!");
+            }
+        }
+
+        //get next waiterObjId
+        nextObjId = currentObj.getNextWaitingResourceObjId();
+
+        if (prevObjId != -1) {
+            //prev->next = next
+            prevObj.setNextWaitingResourceObjId(nextObjId);
+        } else {
+            //removed first waiter. firstWaiter = current->next
+            firstWaitingResource = nextObjId;
+        }
+
+        //        if (LockManager.IS_DEBUG_MODE) {
+        //            System.out.println(Thread.currentThread().getName()+"\tJobInfo_RemoveWaiter:"+ waiterObjId + ", FirstWaiter:"+firstWaitingResource);            
+        //        }
+    }
+
+    public void increaseDatasetISLockCount(int datasetId) {
+        int count = datasetISLockHT.get(datasetId);
+        if (count == -1) {
+            datasetISLockHT.upsert(datasetId, 1);
+        } else {
+            datasetISLockHT.upsert(datasetId, count + 1);
+        }
+    }
+
+    public void decreaseDatasetISLockCount(int datasetId) {
+        int count = datasetISLockHT.get(datasetId);
+        if (count >= LockManager.ESCALATE_TRHESHOLD_ENTITY_TO_DATASET) {
+            //do not decrease the count since it is already escalated.
+        } else if (count > 1) {
+            datasetISLockHT.upsert(datasetId, count - 1);
+        } else if (count == 1) {
+            datasetISLockHT.remove(datasetId);
+        } else if (count <= 0) {
+            throw new IllegalStateException("Illegal state of datasetLock count in JobInfo's dLockHT");
+        }
+    }
+
+    public int getDatasetISLockCount(int datasetId) {
+        int count = datasetISLockHT.get(datasetId);
+        if (count == -1) {
+            return 0;
+        } else {
+            return count;
+        }
+    }
+
+    /**********************************************************************************
+     * public boolean isDatasetLockGranted(int datasetId) {
+     * return dLockHT.get(datasetId) == -1 ? false : true;
+     * }
+     **********************************************************************************/
+
+    public boolean isDatasetLockGranted(int datasetId, byte lockMode) {
+        int entityInfo = lastHoldingResource;
+        byte datasetLockMode;
+
+        while (entityInfo != -1) {
+            datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+            datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? datasetLockMode
+                    : datasetLockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+            if (entityInfoManager.getDatasetId(entityInfo) == datasetId
+                    && isStrongerOrEqualToLockMode(datasetLockMode, lockMode)) {
+                return true;
+            }
+            entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
+        }
+        return false;
+    }
+
+    //check whether LockMode modeA is stronger than or equal to LockMode modeB
+    private boolean isStrongerOrEqualToLockMode(byte modeA, byte modeB) {
+        switch (modeB) {
+            case LockMode.X:
+                return modeA == LockMode.X;
+
+            case LockMode.IX:
+                return modeA == LockMode.IX || modeA == LockMode.X;
+
+            case LockMode.S:
+                return modeA == LockMode.S || modeA == LockMode.X;
+
+            case LockMode.IS:
+                return true;
+
+            default:
+                throw new IllegalStateException("Unsupported dataset lock mode.");
+        }
+    }
+
+    public String printHoldingResource() {
+        StringBuilder s = new StringBuilder();
+        int entityInfo = lastHoldingResource;
+
+        while (entityInfo != -1) {
+            s.append("entityInfo[").append(entityInfo).append("] ");
+            s.append(entityInfoManager.getJobId(entityInfo)).append(" ");
+            s.append(entityInfoManager.getDatasetId(entityInfo)).append(" ");
+            s.append(entityInfoManager.getPKHashVal(entityInfo)).append(" ");
+            s.append(entityInfoManager.getDatasetLockMode(entityInfo)).append(" ");
+            s.append(entityInfoManager.getDatasetLockCount(entityInfo)).append(" ");
+            s.append(entityInfoManager.getEntityLockCount(entityInfo)).append(" ");
+            s.append(entityInfoManager.getEntityLockMode(entityInfo)).append(" ");
+            s.append("\n");
+            entityInfo = entityInfoManager.getPrevJobResource(entityInfo);
+        }
+        return s.toString();
+    }
+
+    /////////////////////////////////////////////////////////
+    //  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..1d47c0b 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,1851 @@
+/*
+ * 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.util.HashMap;
 import java.util.Iterator;
-import java.util.Properties;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.logging.Level;
-import java.util.logging.Logger;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+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.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.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 /**
- * @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());
+    public static final boolean IS_DEBUG_MODE = false;//true
 
-    final int INIT_TABLE_SIZE = 50;
-    private LMTables lmTables;
-    ILockMatrix lMtx;
+    public static final boolean ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET = true;
+    public static final int ESCALATE_TRHESHOLD_ENTITY_TO_DATASET = 1000;
+    private static final int DO_ESCALATE = 0;
+    private static final int ESCALATED = 1;
+    private static final int DONOT_ESCALATE = 2;
 
-    WaitObjectManager woManager;
-    TimeOutDetector toutDetector;
-    DeadlockDetector deadlockDetector;
+    private TransactionSubsystem txnSubsystem;
 
-    public LockManager(TransactionProvider factory) throws ACIDException {
-        Properties p = new Properties();
-        InputStream is = null;
-        ILockMatrix lockMatrix = null;
-        int[] confTab = null;
-        int[] convTab = null;
+    //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;
 
-        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 EntityLockInfoManager entityLockInfoManager;
+    private EntityInfoManager entityInfoManager;
+    private LockWaiterManager lockWaiterManager;
 
-    private static int[] getConfTab(Properties properties) {
-        return null;
-    }
+    private DeadlockDetector deadlockDetector;
+    private TimeOutDetector toutDetector;
+    private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
 
-    private static int[] getConvTab(Properties properties) {
-        return null;
-    }
+    private int tryLockDatasetGranuleRevertOperation;
 
-    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);
+    private LockRequestTracker lockRequestTracker; //for debugging
+    private ConsecutiveWakeupContext consecutiveWakeupContext;
+
+    //TODO 
+    //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds. 
+    private LogicalLogLocator logicalLogLocator;
+
+    public LockManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+        this.txnSubsystem = txnSubsystem;
+        this.lockTableLatch = new ReentrantReadWriteLock(true);
+        this.waiterLatch = new ReentrantReadWriteLock(true);
+        this.jobHT = new HashMap<JobId, JobInfo>();
+        this.datasetResourceHT = new HashMap<DatasetId, DatasetLockInfo>();
+        this.entityInfoManager = new EntityInfoManager();
+        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();
+
+        this.logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
+
+        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;
+        boolean isEscalated = false;
 
-            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;
-                    }
-                    // ---------------------------
+        if (IS_DEBUG_MODE) {
+            trackLockRequest("Requested", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext,
+                    dLockInfo, eLockInfo);
+        }
 
-                    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());
+        dLockInfo = datasetResourceHT.get(datasetId);
+        jobInfo = jobHT.get(jobId);
 
-                    }
-                } else { // No converter(s) waiting
-                    int mask = lInfo.getMask();
-                    if (lMtx.conflicts(mask, mode)) { // If There is conflict,
-                                                      // Txr needs to wait
+        if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+            if (datasetLockMode == LockMode.IS && jobInfo != null && dLockInfo != null) {
+                int upgradeStatus = needUpgradeFromEntityToDataset(jobInfo, dId, lockMode);
+                switch (upgradeStatus) {
+                    case DO_ESCALATE:
+                        entityHashValue = -1;
+                        isEscalated = true;
+                        break;
 
-                        // ----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;
-                    }
-                }
-            }
+                    case ESCALATED:
+                        unlatchLockTable();
+                        return;
 
-            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
-                    }
-
+                    default:
+                        break;
                 }
             }
         }
 
-        if (isConverting) {
+        //#. if the datasetLockInfo doesn't exist in datasetResourceHT 
+        if (dLockInfo == null || dLockInfo.isNoHolder()) {
+            if (dLockInfo == null) {
+                dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
+                datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
+            }
+            entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
+
+            //if dataset-granule lock
+            if (entityHashValue == -1) { //-1 stands for dataset-granule
+                entityInfoManager.increaseDatasetLockCount(entityInfo);
+                dLockInfo.increaseLockCount(datasetLockMode);
+                dLockInfo.addHolder(entityInfo);
+            } else {
+                entityInfoManager.increaseDatasetLockCount(entityInfo);
+                dLockInfo.increaseLockCount(datasetLockMode);
+                //add entityLockInfo
+                eLockInfo = entityLockInfoManager.allocate();
+                dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
+                entityInfoManager.increaseEntityLockCount(entityInfo);
+                entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
+                entityLockInfoManager.addHolder(eLockInfo, entityInfo);
+            }
+
+            if (jobInfo == null) {
+                jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
+                jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
+            }
+            jobInfo.addHoldingResource(entityInfo);
+
+            if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+                if (datasetLockMode == LockMode.IS) {
+                    jobInfo.increaseDatasetISLockCount(dId);
+                }
+            }
+
+            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 (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+            if (isEscalated) {
+                releaseDatasetISLocks(jobInfo, jobId, datasetId, txnContext);
+            }
+            if (datasetLockMode == LockMode.IS) {
+                jobInfo.increaseDatasetISLockCount(dId);
+            }
+        }
+
+        if (IS_DEBUG_MODE) {
+            trackLockRequest("Granted", RequestType.LOCK, datasetId, entityHashValue, lockMode, txnContext, dLockInfo,
+                    eLockInfo);
+        }
+        unlatchLockTable();
+        return;
+    }
+
+    private void releaseDatasetISLocks(JobInfo jobInfo, JobId jobId, DatasetId datasetId, TransactionContext txnContext)
+            throws ACIDException {
+        int entityInfo;
+        int prevEntityInfo;
+        int entityHashValue;
+        int did;//int-type dataset id
+
+        //while traversing all holding resources, 
+        //release IS locks on the escalated dataset and
+        //release S locks on the corresponding enttites
+        //by calling unlock() method.
+        entityInfo = jobInfo.getLastHoldingResource();
+        while (entityInfo != -1) {
+            prevEntityInfo = entityInfoManager.getPrevJobResource(entityInfo);
+
+            //release a lock only if the datset is the escalated dataset and
+            //the entityHashValue is not -1("not -1" means a non-dataset-level lock)
+            did = entityInfoManager.getDatasetId(entityInfo);
+            entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
+            if (did == datasetId.getId() && entityHashValue != -1) {
+                this.unlock(datasetId, entityHashValue, txnContext);
+            }
+
+            entityInfo = prevEntityInfo;
+        }
+    }
+
+    private int needUpgradeFromEntityToDataset(JobInfo jobInfo, int datasetId, byte lockMode) {
+        //we currently allow upgrade only if the lockMode is S. 
+        if (lockMode != LockMode.S) {
+            return DONOT_ESCALATE;
+        }
+
+        int count = jobInfo.getDatasetISLockCount(datasetId);
+        if (count == ESCALATE_TRHESHOLD_ENTITY_TO_DATASET) {
+            return DO_ESCALATE;
+        } else if (count > ESCALATE_TRHESHOLD_ENTITY_TO_DATASET) {
+            return ESCALATED;
+        } else {
+            return DONOT_ESCALATE;
+        }
+    }
+
+    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] Mimicking SIX mode
+                //When the lock escalation from IS to S in dataset-level is allowed, the following case occurs
+                //DatasetLockInfo's SCount = 1 and the same job who carried out the escalation tries to insert,
+                //then the job should be able to insert without being blocked by itself. 
+                //Our approach is to introduce SIX mode, but we don't have currently, 
+                //so I simply mimicking SIX by allowing S and IX coexist in the dataset level 
+                //only if their job id is identical for the requests. 
+                if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+                    if (datasetLockMode == LockMode.IX && dLockInfo.getSCount() == 1
+                            && jobInfo.isDatasetLockGranted(dId, LockMode.S)) {
+                        entityInfoManager.increaseDatasetLockCount(entityInfo);
+                        //IX holders are implicitly handled without adding holder
+                        dLockInfo.increaseLockCount(datasetLockMode);
+                        //add entityInfo to JobInfo's holding-resource list
+                        jobInfo.addHoldingResource(entityInfo);
+                        return entityInfo;
+                    }
+                }
+                ///////////////////////////////////////////////////////////////////////////////////////////////
+
+                /////////////////////////////////////////////////////////////////////////////////////////////
+                //[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);
-
-            if (txrInfo == null) {
-                throw new ACIDException("No lock is granted to the transaction, to convert");
-            }
-
-            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");
-            }
-
-            curMode = tInfo.getMode();
-            if (mode == curMode) { // Redundant
-                return true; // We do not increment the counter, because it is a
-                // conversion
-            }
-
-            if (!lMtx.isConversion(curMode, mode)) {
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Transaction " + txId + " already has grabbed a stronger mode (" + curMode + ") than "
-                            + mode);
-                }
-
-                return true;
-            }
-
-            lInfo = lmTables.getLockInfo(resourceID);
-            if (lInfo == null) {
-                throw new ACIDException("No lock on the resource, to convert");
-            }
-        }
-
-        try {
-            return convertLockForNewTransaction(context, resourceID, curMode, mode);
-        } catch (InterruptedException e) {
-            e.printStackTrace();
-        }
-
-        throw new ACIDException("Problem in Lock Converting for Transaction " + txId
-                + " (We unexpectedly returned from convert lock for new transaction)");
+    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext) throws ACIDException {
+        internalUnlock(datasetId, entityHashValue, txnContext, false);
     }
 
-    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 ");
-                }
-                return true;
-            }
-            // --------------------------------------------
+    @Override
+    public void unlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext, boolean commitFlag)
+            throws ACIDException {
+        internalUnlock(datasetId, entityHashValue, txnContext, commitFlag);
+    }
 
-            int updatedMask = lInfo.getUpdatedMask(txId, curMode, LockInfo.UNKNOWN_IX);
-            if (lMtx.conflicts(updatedMask, reqMode)) { // if Conflicting, Txr
-                                                        // needs to wait
+    private void internalUnlock(DatasetId datasetId, int entityHashValue, TransactionContext txnContext,
+            boolean commitFlag) throws ACIDException {
+        JobId jobId = txnContext.getJobId();
+        int eLockInfo = -1;
+        DatasetLockInfo dLockInfo = null;
+        JobInfo jobInfo;
+        int entityInfo = -1;
+        byte datasetLockMode;
 
-                // ---- 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;
-                }
-                // ---------------------------
-
-                else {
-                    waitObj = woManager.allocate();
-                    if (waitObj == null) {
-                        throw new ACIDException("Invalid (null) object allocated as the WaitEntry for Txr " + txId);
-                    }
-                    lInfo.addToConvert(txId, reqMode, waitObj);
-                    txrInfo.setWaitOnRid(resourceId);
-                    context.setStartWaitTime(System.currentTimeMillis());
-                }
-            }
-
-            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 ");
-                }
-                return true;
+        if (IS_DEBUG_MODE) {
+            if (entityHashValue == -1) {
+                throw new UnsupportedOperationException(
+                        "Unsupported unlock request: dataset-granule unlock is not supported");
             }
         }
 
-        if (shouldAbort) {
-            requestTxrAbort(context);
-            return false;
+        latchLockTable();
+        validateJob(txnContext);
+
+        if (IS_DEBUG_MODE) {
+            trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, 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()) {
+        //find the resource to be unlocked
+        dLockInfo = datasetResourceHT.get(datasetId);
+        jobInfo = jobHT.get(jobId);
+        if (dLockInfo == null || jobInfo == null) {
+            unlatchLockTable();
+            throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
+        }
 
-                waitObj.wait();
-            }
+        eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
 
-            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);
+        if (eLockInfo == -1) {
+            unlatchLockTable();
+            throw new IllegalStateException("Invalid unlock request: Corresponding lock info doesn't exist.");
+        }
+
+        //find the corresponding entityInfo
+        entityInfo = entityLockInfoManager.findEntityInfoFromHolderList(eLockInfo, jobId.getId(), entityHashValue);
+        if (entityInfo == -1) {
+            unlatchLockTable();
+            throw new IllegalStateException("Invalid unlock request[" + jobId.getId() + "," + datasetId.getId() + ","
+                    + entityHashValue + "]: Corresponding lock info doesn't exist.");
+        }
+
+        datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo) == LockMode.S ? LockMode.IS : LockMode.IX;
+
+        //decrease the corresponding count of dLockInfo/eLockInfo/entityInfo
+        dLockInfo.decreaseLockCount(datasetLockMode);
+        entityLockInfoManager.decreaseLockCount(eLockInfo, entityInfoManager.getEntityLockMode(entityInfo));
+        entityInfoManager.decreaseDatasetLockCount(entityInfo);
+        entityInfoManager.decreaseEntityLockCount(entityInfo);
+
+        if (entityInfoManager.getEntityLockCount(entityInfo) == 0
+                && entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
+            int threadCount = 0; //number of threads(in the same job) waiting on the same resource 
+            int waiterObjId = jobInfo.getFirstWaitingResource();
+            int waitingEntityInfo;
+            LockWaiter waiterObj;
+
+            //TODO
+            //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds.
+            //This commit log is written here in order to avoid increasing the memory space for managing transactionIds
+            if (commitFlag) {
+                if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) {
+                    try {
+                        txnSubsystem.getLogManager().log(LogType.ENTITY_COMMIT, txnContext, datasetId.getId(),
+                                entityHashValue, -1, (byte) 0, 0, null, null, logicalLogLocator);
+                    } catch (ACIDException e) {
+                        try {
+                            requestAbort(txnContext);
+                        } finally {
+                            unlatchLockTable();
+                        }
+                    }
                 }
+
+                txnContext.updateLastLSNForIndexes(logicalLogLocator.getLsn());
+            }
+
+            //1) wake up waiters and remove holder
+            //wake up waiters of dataset-granule lock
+            wakeUpDatasetLockWaiters(dLockInfo);
+            //wake up waiters of entity-granule lock
+            wakeUpEntityLockWaiters(eLockInfo);
+            //remove the holder from eLockInfo's holder list and remove the holding resource from jobInfo's holding resource list
+            //this can be done in the following single function call.
+            entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
+
+            //2) if 
+            //      there is no waiting thread on the same resource (this can be checked through jobInfo)
+            //   then 
+            //      a) delete the corresponding entityInfo
+            //      b) write commit log for the unlocked resource(which is a committed txn).
+            while (waiterObjId != -1) {
+                waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+                waitingEntityInfo = waiterObj.getEntityInfoSlot();
+                if (entityInfoManager.getDatasetId(waitingEntityInfo) == datasetId.getId()
+                        && entityInfoManager.getPKHashVal(waitingEntityInfo) == entityHashValue) {
+                    threadCount++;
+                    break;
+                }
+                waiterObjId = waiterObj.getNextWaiterObjId();
+            }
+            if (threadCount == 0) {
+                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);
+            }
+        }
+
+        //deallocate entityLockInfo's slot if there is no txn referring to the entityLockInfo.
+        if (entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
+                && entityLockInfoManager.getLastHolder(eLockInfo) == -1
+                && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
+            dLockInfo.getEntityResourceHT().remove(entityHashValue);
+            entityLockInfoManager.deallocate(eLockInfo);
+        }
+
+        //we don't deallocate datasetLockInfo even if there is no txn referring to the datasetLockInfo
+        //since the datasetLockInfo is likely to be referred to again.
+
+        if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+            if (datasetLockMode == LockMode.IS) {
+                jobInfo.decreaseDatasetISLockCount(datasetId.getId());
+            }
+        }
+
+        if (IS_DEBUG_MODE) {
+            trackLockRequest("Granted", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
+                    dLockInfo, eLockInfo);
+        }
+        unlatchLockTable();
+    }
+
+    @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!!!");
+                }
+            }
+
+            //1. remove from waiter(or upgrader)'s list of dLockInfo or eLockInfo.
+            did = entityInfoManager.getDatasetId(entityInfo);
+            tempDatasetIdObj.setId(did);
+            dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
+
+            if (waiterObj.isWaitingOnEntityLock()) {
+                entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
+                eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+                if (waiterObj.isWaiter()) {
+                    entityLockInfoManager.removeWaiter(eLockInfo, waiterObjId);
+                } else {
+                    entityLockInfoManager.removeUpgrader(eLockInfo, waiterObjId);
+                }
+            } else {
+                if (waiterObj.isWaiter()) {
+                    dLockInfo.removeWaiter(waiterObjId);
+                } else {
+                    dLockInfo.removeUpgrader(waiterObjId);
+                }
+            }
+
+            //2. wake-up waiters
+            latchWaitNotify();
+            synchronized (waiterObj) {
+                unlatchWaitNotify();
+                waiterObj.setWait(false);
+                if (IS_DEBUG_MODE) {
+                    System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID(" + waiterObjId
+                            + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+                }
+                waiterObj.notifyAll();
+            }
+
+            //3. deallocate waiterObj
+            lockWaiterManager.deallocate(waiterObjId);
+
+            //4. deallocate entityInfo only if this waiter is not an upgrader
+            if (entityInfoManager.getDatasetLockCount(entityInfo) == 0
+                    && entityInfoManager.getEntityLockCount(entityInfo) == 0) {
+                entityInfoManager.deallocate(entityInfo);
+            }
+            waiterObjId = nextWaiterObjId;
+        }
+
+        //release holding resources
+        entityInfo = jobInfo.getLastHoldingResource();
+        while (entityInfo != -1) {
+            prevEntityInfo = entityInfoManager.getPrevJobResource(entityInfo);
+
+            //decrease lock count of datasetLock and entityLock
+            did = entityInfoManager.getDatasetId(entityInfo);
+            tempDatasetIdObj.setId(did);
+            dLockInfo = datasetResourceHT.get(tempDatasetIdObj);
+            entityHashValue = entityInfoManager.getPKHashVal(entityInfo);
+
+            if (entityHashValue == -1) {
+                //decrease datasetLockCount
+                lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+                datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+                if (datasetLockCount != 0) {
+                    dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
+
+                    //wakeup waiters of datasetLock and remove holder from datasetLockInfo
+                    wakeUpDatasetLockWaiters(dLockInfo);
+
+                    //remove the holder from datasetLockInfo only if the lock is dataset-granule lock.
+                    //--> this also removes the holding resource from jobInfo               
+                    //(Because the IX and IS lock's holders are handled implicitly, 
+                    //those are not in the holder list of datasetLockInfo.)
+                    dLockInfo.removeHolder(entityInfo, jobInfo);
+                }
+            } else {
+                //decrease datasetLockCount
+                lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+                lockMode = lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+                datasetLockCount = entityInfoManager.getDatasetLockCount(entityInfo);
+
+                if (datasetLockCount != 0) {
+                    dLockInfo.decreaseLockCount(lockMode, datasetLockCount);
+                }
+
+                //decrease entityLockCount
+                lockMode = entityInfoManager.getEntityLockMode(entityInfo);
+                entityLockCount = entityInfoManager.getEntityLockCount(entityInfo);
+                eLockInfo = dLockInfo.getEntityResourceHT().get(entityHashValue);
+                if (IS_DEBUG_MODE) {
+                    if (eLockInfo < 0) {
+                        System.out.println("eLockInfo:" + eLockInfo);
+                    }
+                }
+
+                if (entityLockCount != 0) {
+                    entityLockInfoManager.decreaseLockCount(eLockInfo, lockMode, (short) entityLockCount);
+                }
+
+                if (datasetLockCount != 0) {
+                    //wakeup waiters of datasetLock and don't remove holder from datasetLockInfo
+                    wakeUpDatasetLockWaiters(dLockInfo);
+                }
+
+                if (entityLockCount != 0) {
+                    //wakeup waiters of entityLock
+                    wakeUpEntityLockWaiters(eLockInfo);
+
+                    //remove the holder from entityLockInfo 
+                    //--> this also removes the holding resource from jobInfo
+                    entityLockInfoManager.removeHolder(eLockInfo, entityInfo, jobInfo);
+                }
+
+                //deallocate entityLockInfo if there is no holder and waiter.
+                if (entityLockInfoManager.getLastHolder(eLockInfo) == -1
+                        && entityLockInfoManager.getFirstWaiter(eLockInfo) == -1
+                        && entityLockInfoManager.getUpgrader(eLockInfo) == -1) {
+                    dLockInfo.getEntityResourceHT().remove(entityHashValue);
+                    entityLockInfoManager.deallocate(eLockInfo);
+                    //                    if (IS_DEBUG_MODE) {
+                    //                        System.out.println("removed PK["+entityHashValue+"]");
+                    //                    }
+                }
+            }
+
+            //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;
+        boolean isEscalated = 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 (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+            if (datasetLockMode == LockMode.IS && jobInfo != null && dLockInfo != null) {
+                int upgradeStatus = needUpgradeFromEntityToDataset(jobInfo, dId, lockMode);
+                switch (upgradeStatus) {
+                    case DO_ESCALATE:
+                        entityHashValue = -1;
+                        isEscalated = true;
+                        break;
+
+                    case ESCALATED:
+                        unlatchLockTable();
+                        return true;
+
+                    default:
+                        break;
+                }
+            }
+        }
+
+        //#. if the datasetLockInfo doesn't exist in datasetResourceHT 
+        if (dLockInfo == null || dLockInfo.isNoHolder()) {
+            if (dLockInfo == null) {
+                dLockInfo = new DatasetLockInfo(entityLockInfoManager, entityInfoManager, lockWaiterManager);
+                datasetResourceHT.put(new DatasetId(dId), dLockInfo); //datsetId obj should be created
+            }
+            entityInfo = entityInfoManager.allocate(jId, dId, entityHashValue, lockMode);
+
+            //if dataset-granule lock
+            if (entityHashValue == -1) { //-1 stands for dataset-granule
+                entityInfoManager.increaseDatasetLockCount(entityInfo);
+                dLockInfo.increaseLockCount(datasetLockMode);
+                dLockInfo.addHolder(entityInfo);
+            } else {
+                entityInfoManager.increaseDatasetLockCount(entityInfo);
+                dLockInfo.increaseLockCount(datasetLockMode);
+                //add entityLockInfo
+                eLockInfo = entityLockInfoManager.allocate();
+                dLockInfo.getEntityResourceHT().put(entityHashValue, eLockInfo);
+                entityInfoManager.increaseEntityLockCount(entityInfo);
+                entityLockInfoManager.increaseLockCount(eLockInfo, lockMode);
+                entityLockInfoManager.addHolder(eLockInfo, entityInfo);
+            }
+
+            if (jobInfo == null) {
+                jobInfo = new JobInfo(entityInfoManager, lockWaiterManager, txnContext);
+                jobHT.put(jobId, jobInfo); //jobId obj doesn't have to be created
+            }
+            jobInfo.addHoldingResource(entityInfo);
+
+            if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+                if (datasetLockMode == LockMode.IS) {
+                    jobInfo.increaseDatasetISLockCount(dId);
+                }
+            }
+
+            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);
+                }
+            }
+        }
+
+        if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
+            if (isEscalated) {
+                releaseDatasetISLocks(jobInfo, jobId, datasetId, txnContext);
+            }
+            if (datasetLockMode == LockMode.IS) {
+                jobInfo.increaseDatasetISLockCount(dId);
+            }
+        }
+
+        if (IS_DEBUG_MODE) {
+            if (isSuccess) {
+                trackLockRequest("Granted", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
+                        dLockInfo, eLockInfo);
+            } else {
+                trackLockRequest("Failed", RequestType.TRY_LOCK, datasetId, entityHashValue, lockMode, txnContext,
+                        dLockInfo, eLockInfo);
+            }
+        }
+
+        unlatchLockTable();
+
+        return isSuccess;
+    }
+
+    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(Thread.currentThread().getId() + ":");
+        s.append(msg);
+        if (msg.equals("Granted")) {
+            if (dLockInfo != null) {
+                s.append("\t|D| ");
+                s.append(dLockInfo.getIXCount()).append(",");
+                s.append(dLockInfo.getISCount()).append(",");
+                s.append(dLockInfo.getXCount()).append(",");
+                s.append(dLockInfo.getSCount()).append(",");
+                if (dLockInfo.getFirstUpgrader() != -1) {
+                    s.append("+");
+                } else {
+                    s.append("-");
+                }
+                s.append(",");
+                if (dLockInfo.getFirstWaiter() != -1) {
+                    s.append("+");
+                } else {
+                    s.append("-");
+                }
+            }
+
+            if (eLockInfo != -1) {
+                s.append("\t|E| ");
+                s.append(entityLockInfoManager.getXCount(eLockInfo)).append(",");
+                s.append(entityLockInfoManager.getSCount(eLockInfo)).append(",");
+                if (entityLockInfoManager.getUpgrader(eLockInfo) != -1) {
+                    s.append("+");
+                } else {
+                    s.append("-");
+                }
+                s.append(",");
+                if (entityLockInfoManager.getFirstWaiter(eLockInfo) != -1) {
+                    s.append("+");
+                } else {
+                    s.append("-");
+                }
+            }
+        }
+
+        lockRequestTracker.addEvent(s.toString(), request);
+        if (msg.equals("Requested")) {
+            lockRequestTracker.addRequest(request);
+        }
+        System.out.println(request.prettyPrint() + "--> " + s.toString());
+    }
+
+    public String getHistoryForAllJobs() {
+        if (IS_DEBUG_MODE) {
+            return lockRequestTracker.getHistoryForAllJobs();
+        }
+        return null;
+    }
+
+    public String getHistoryPerJob() {
+        if (IS_DEBUG_MODE) {
+            return lockRequestTracker.getHistoryPerJob();
+        }
+        return null;
+    }
+
+    public String getRequestHistoryForAllJobs() {
+        if (IS_DEBUG_MODE) {
+            return lockRequestTracker.getRequestHistoryForAllJobs();
+        }
+        return null;
+    }
+
+    private void revertTryLockDatasetGranuleOperation(DatasetId datasetId, int entityHashValue, byte lockMode,
+            int entityInfo, 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);
+                    }
+                }
             }
 
-            return true;
+            if (isInterruptedExceptionOccurred) {
+                throw new ACIDException("InterruptedException is caught");
+            }
+
+            //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)");
     }
 
-    @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);
+    /**
+     * 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() + ")");
                     }
-                    int wlMode = lInfo.getLockMode(wlModeIx);
-                    String wLModeRep = (wlMode == 0 ? "S" : "X");
-                    status += " for " + wLModeRep + " lock";
+                    waiterObj.notifyAll();
                 }
-
-                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";
+                waiterObjId = waiterObj.getNextWaiterObjId();
+            } else {
+                areAllUpgradersAwaken = false;
+                break;
             }
-
         }
+
+        if (areAllUpgradersAwaken) {
+            //wake up waiters
+            waiterObjId = dLockInfo.getFirstWaiter();
+            while (waiterObjId != -1) {
+                waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+                entityInfo = waiterObj.getEntityInfoSlot();
+                lockMode = entityInfoManager.getDatasetLockMode(entityInfo);
+                datasetLockMode = entityInfoManager.getPKHashVal(entityInfo) == -1 ? lockMode
+                        : lockMode == LockMode.S ? LockMode.IS : LockMode.IX;
+                if (dLockInfo.isCompatible(datasetLockMode) && consecutiveWakeupContext.isCompatible(datasetLockMode)) {
+                    consecutiveWakeupContext.setLockMode(datasetLockMode);
+                    //compatible waiter is waken up
+                    latchWaitNotify();
+                    synchronized (waiterObj) {
+                        unlatchWaitNotify();
+                        waiterObj.setWait(false);
+                        if (IS_DEBUG_MODE) {
+                            System.out.println("" + Thread.currentThread().getName() + "\twake-up(D): WID("
+                                    + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+                        }
+                        waiterObj.notifyAll();
+                    }
+                    waiterObjId = waiterObj.getNextWaiterObjId();
+                } else {
+                    break;
+                }
+            }
+        }
+    }
+
+    private void wakeUpEntityLockWaiters(int eLockInfo) {
+        boolean areAllUpgradersAwaken = true;
+        int waiterObjId = entityLockInfoManager.getUpgrader(eLockInfo);
+        int entityInfo;
+        LockWaiter waiterObj;
+        byte entityLockMode;
+
+        consecutiveWakeupContext.reset();
+        while (waiterObjId != -1) {
+            //wake up upgraders
+            waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+            entityInfo = waiterObj.getEntityInfoSlot();
+            if (entityLockInfoManager.isUpgradeCompatible(eLockInfo, LockMode.X, entityInfo)
+                    && consecutiveWakeupContext.isCompatible(LockMode.X)) {
+                consecutiveWakeupContext.setLockMode(LockMode.X);
+                latchWaitNotify();
+                synchronized (waiterObj) {
+                    unlatchWaitNotify();
+                    waiterObj.setWait(false);
+                    if (IS_DEBUG_MODE) {
+                        System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID(" + waiterObjId
+                                + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+                    }
+                    waiterObj.notifyAll();
+                }
+                waiterObjId = waiterObj.getNextWaiterObjId();
+            } else {
+                areAllUpgradersAwaken = false;
+                break;
+            }
+        }
+
+        if (areAllUpgradersAwaken) {
+            //wake up waiters
+            waiterObjId = entityLockInfoManager.getFirstWaiter(eLockInfo);
+            while (waiterObjId != -1) {
+                waiterObj = lockWaiterManager.getLockWaiter(waiterObjId);
+                entityInfo = waiterObj.getEntityInfoSlot();
+                entityLockMode = entityInfoManager.getEntityLockMode(entityInfo);
+                if (entityLockInfoManager.isCompatible(eLockInfo, entityLockMode)
+                        && consecutiveWakeupContext.isCompatible(entityLockMode)) {
+                    consecutiveWakeupContext.setLockMode(entityLockMode);
+                    //compatible waiter is waken up
+                    latchWaitNotify();
+                    synchronized (waiterObj) {
+                        unlatchWaitNotify();
+                        waiterObj.setWait(false);
+                        if (IS_DEBUG_MODE) {
+                            System.out.println("" + Thread.currentThread().getName() + "\twake-up(E): WID("
+                                    + waiterObjId + "),EID(" + waiterObj.getEntityInfoSlot() + ")");
+                        }
+                        waiterObj.notifyAll();
+                    }
+                } else {
+                    break;
+                }
+                waiterObjId = waiterObj.getNextWaiterObjId();
+            }
+        }
+    }
+
+    @Override
+    public String prettyPrint() throws ACIDException {
+        StringBuilder s = new StringBuilder("\n########### LockManager Status #############\n");
         return s + "\n";
     }
 
     public void sweepForTimeout() throws ACIDException {
-        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
-     */
-
-    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) {
-        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 int getId() {
-        return id;
-    }
-
-    public void setNext(int n) {
-        next = n;
-    }
-
-    public int getNext() {
-        return next;
-    }
-
-    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..c3b47bc
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManagerDeterministicUnitTest.java
@@ -0,0 +1,618 @@
+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.TransactionSubsystem;
+
+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;
+    TransactionSubsystem 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 TransactionSubsystem("LockManagerPredefinedUnitTest", null);;
+        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;
+    TransactionSubsystem txnProvider;
+    ILockManager lockMgr;
+    WorkerReadyQueue workerReadyQueue;
+    LockRequest lockRequest;
+    boolean needWait;
+    boolean isAwaken;
+    boolean isDone;
+
+    public LockRequestWorker(TransactionSubsystem 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..89a15ef
--- /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.TransactionSubsystem;
+
+/**
+ * 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;
+        TransactionSubsystem txnProvider = new TransactionSubsystem("LockManagerRandomUnitTest", null);
+        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(TransactionSubsystem txnProvider) {
+        Thread t;
+        int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
+        if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
+            childCount = 1;
+        }
+        TransactionContext txnContext = generateTxnContext(txnProvider);
+
+        for (int i = 0; i < childCount; i++) {
+            System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockThread..");
+            t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, false, false));
+            t.start();
+        }
+    }
+
+    private static void generateDatasetLockThread(TransactionSubsystem txnProvider) {
+        Thread t;
+        //        int childCount = rand.nextInt(MAX_NUM_OF_THREAD_IN_A_JOB);
+        //        if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
+        //            childCount = 1;
+        //        }
+        int childCount = 1;
+
+        TransactionContext txnContext = generateTxnContext(txnProvider);
+
+        for (int i = 0; i < childCount; i++) {
+            System.out.println("Creating " + txnContext.getJobId() + "," + i + "th DatasetLockThread..");
+            t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, true, false, false));
+            t.start();
+        }
+    }
+
+    private static void generateEntityLockUpgradeThread(TransactionSubsystem txnProvider) {
+        int i;
+        Thread t;
+        int childCount = MAX_NUM_OF_THREAD_IN_A_JOB;
+        if (MAX_NUM_OF_THREAD_IN_A_JOB != 0 && childCount == 0) {
+            childCount = 1;
+        }
+        TransactionContext txnContext = generateTxnContext(txnProvider);
+
+        for (i = 0; i < childCount - 1; i++) {
+            System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(false)..");
+            t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, false));
+            t.start();
+        }
+        System.out.println("Creating " + txnContext.getJobId() + "," + i + "th EntityLockUpgradeThread(true)..");
+        t = new Thread(new LockRequestProducer(txnProvider.getLockManager(), txnContext, false, true, true));
+        t.start();
+    }
+
+    private static TransactionContext generateTxnContext(TransactionSubsystem txnProvider) {
+        try {
+            return new TransactionContext(new JobId(jobId++), txnProvider);
+        } catch (ACIDException e) {
+            e.printStackTrace();
+            return null;
+        }
+    }
+
+}
+
+class LockRequestProducer implements Runnable {
+
+    private static final int MAX_DATASET_NUM = 10;//10
+    private static final int MAX_ENTITY_NUM = 30;//30
+    private static final int MAX_LOCK_MODE_NUM = 2;
+    private static final long DATASET_LOCK_THREAD_SLEEP_TIME = 1000;
+    private static final int MAX_LOCK_REQUEST_TYPE_NUM = 4;
+
+    private ILockManager lockMgr;
+    private TransactionContext txnContext;
+    private Random rand;
+    private boolean isDatasetLock; //dataset or entity
+    private ArrayList<LockRequest> requestQueue;
+    private StringBuilder requestHistory;
+    private int unlockIndex;
+    private int upgradeIndex;
+    private boolean isUpgradeThread;
+    private boolean isUpgradeThreadJob;
+    private boolean isDone;
+
+    public LockRequestProducer(ILockManager lockMgr, TransactionContext txnContext, boolean isDatasetLock,
+            boolean isUpgradeThreadJob, boolean isUpgradeThread) {
+        this.lockMgr = lockMgr;
+        this.txnContext = txnContext;
+        this.isDatasetLock = isDatasetLock;
+        this.isUpgradeThreadJob = isUpgradeThreadJob;
+        this.isUpgradeThread = isUpgradeThread;
+
+        this.rand = new Random(System.currentTimeMillis());
+        requestQueue = new ArrayList<LockRequest>();
+        requestHistory = new StringBuilder();
+        unlockIndex = 0;
+        upgradeIndex = 0;
+        isDone = false;
+    }
+
+    @Override
+    public void run() {
+        try {
+            if (isDatasetLock) {
+                System.out.println("DatasetLockThread(" + Thread.currentThread().getName() + ") is running...");
+                runDatasetLockTask();
+            } else {
+                System.out.println("EntityLockThread(" + Thread.currentThread().getName() + "," + isUpgradeThreadJob
+                        + "," + isUpgradeThread + ") is running...");
+                runEntityLockTask();
+            }
+        } catch (Exception e) {
+            e.printStackTrace();
+            return;
+        } finally {
+
+            /*
+            System.out.println("" + Thread.currentThread().getName() + "\n" + requestHistory.toString() + ""
+                    + Thread.currentThread().getName() + "\n");
+            System.out.println("RequestHistoryPerJobId\n" + ((LockManager) lockMgr).getLocalRequestHistory());
+            System.out.println("");
+            System.out.println("GlobalRequestHistory\n" + ((LockManager) lockMgr).getGlobalRequestHistory());
+            System.out.println("");
+            */
+        }
+    }
+
+    private void runDatasetLockTask() {
+        try {
+            produceDatasetLockRequest();
+            if (isDone) {
+                return;
+            }
+        } catch (ACIDException e) {
+            e.printStackTrace();
+            return;
+        }
+
+        try {
+            Thread.sleep(DATASET_LOCK_THREAD_SLEEP_TIME);
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+
+        try {
+            produceDatasetUnlockRequest();
+            if (isDone) {
+                return;
+            }
+        } catch (ACIDException e) {
+            e.printStackTrace();
+            return;
+        }
+    }
+
+    private void runEntityLockTask() {
+        int i;
+        byte lockMode;
+        int lockCount;
+        int upgradeCount;
+        int releaseCount;
+        boolean mayRelease = false;
+
+        lockCount = 1 + rand.nextInt(20);
+        if (isUpgradeThreadJob) {
+            if (isUpgradeThread) {
+                upgradeCount = 1; //rand.nextInt(4) + 1;
+                if (upgradeCount > lockCount) {
+                    upgradeCount = lockCount;
+                }
+            } else {
+                upgradeCount = 0;
+            }
+            lockMode = LockMode.S;
+        } else {
+            upgradeCount = 0;
+            lockMode = (byte) (this.txnContext.getJobId().getId() % 2);
+        }
+        releaseCount = rand.nextInt(5) % 3 == 0 ? 1 : 0;
+
+        //lock
+        for (i = 0; i < lockCount; i++) {
+            try {
+                produceEntityLockRequest(lockMode);
+                if (isDone) {
+                    return;
+                }
+            } catch (ACIDException e) {
+                e.printStackTrace();
+                return;
+            }
+        }
+
+        //upgrade
+        for (i = 0; i < upgradeCount; i++) {
+            try {
+                produceEntityLockUpgradeRequest();
+                if (isDone) {
+                    return;
+                }
+            } catch (ACIDException e) {
+                e.printStackTrace();
+                return;
+            }
+        }
+
+        //unlock or releaseLocks
+        if (releaseCount == 0) {
+            //unlock
+            for (i = 0; i < lockCount; i++) {
+                try {
+                    produceEntityUnlockRequest();
+                    if (isDone) {
+                        return;
+                    }
+                } catch (ACIDException e) {
+                    e.printStackTrace();
+                    return;
+                }
+            }
+        } else {
+            try {
+                synchronized (txnContext) {
+                    if (txnContext.getTxnState() != 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..a65f385
--- /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][i*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..6bc8c6b 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
@@ -3,10 +3,10 @@
 import java.util.LinkedList;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-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 +15,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 +30,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 +75,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..91ce7ed 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,24 @@
         return bytes;
     }
 
+    public static long byteArrayToLong(byte[] bytes, int offset) {
+        return ((bytes[offset] & 0xff) << 56) + ((bytes[offset + 1] & 0xff) << 48) + ((bytes[offset + 2] & 0xff) << 40)
+                + ((bytes[offset + 3] & 0xff) << 32) + ((bytes[offset + 4] & 0xff) << 24)
+                + ((bytes[offset + 5] & 0xff) << 16) + ((bytes[offset + 6] & 0xff) << 8)
+                + ((bytes[offset + 7] & 0xff) << 0);
+    }
+
+    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/FileBasedBuffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileBasedBuffer.java
index d3cb2d1..4319b8b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileBasedBuffer.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/FileBasedBuffer.java
@@ -36,7 +36,7 @@
         this.filePath = filePath;
         this.nextWritePosition = offset;
         buffer = ByteBuffer.allocate(size);
-        raf = new RandomAccessFile(new File(filePath), "rws");
+        raf = new RandomAccessFile(new File(filePath), "rw");
         raf.seek(offset);
         fileChannel = raf.getChannel();
         fileChannel.read(buffer);
@@ -75,6 +75,7 @@
         buffer.position(0);
         buffer.limit(size);
         fileChannel.write(buffer);
+        fileChannel.force(true);
         erase();
     }
 
@@ -125,9 +126,9 @@
     @Override
     public void reset(String filePath, long nextWritePosition, int size) throws IOException {
         if (!filePath.equals(this.filePath)) {
-            raf.close();
+            raf.close();//required?
             fileChannel.close();
-            raf = new RandomAccessFile(filePath, "rws");
+            raf = new RandomAccessFile(filePath, "rw");
             this.filePath = filePath;
         }
         this.nextWritePosition = nextWritePosition;
@@ -138,6 +139,23 @@
         buffer.limit(size);
         this.size = size;
     }
+    
+    @Override
+    public void close() throws IOException {
+        fileChannel.close();
+    }
+    
+    @Override
+    public void open(String filePath, long offset, int size) throws IOException {
+        raf = new RandomAccessFile(filePath, "rw");
+        this.nextWritePosition = offset;
+        fileChannel = raf.getChannel();
+        fileChannel.position(offset);
+        erase();
+        buffer.position(0);
+        buffer.limit(size);
+        this.size = size;
+    }
 
     public long getNextWritePosition() {
         return nextWritePosition;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IFileBasedBuffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IFileBasedBuffer.java
index a0620f5..e1f9f95 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IFileBasedBuffer.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IFileBasedBuffer.java
@@ -35,4 +35,8 @@
 
     public void setNextWritePosition(long writePosition);
 
+    public void close() throws IOException;
+    
+    public void open(String filePath, long offset, int size) throws IOException;
+
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
index 437c92b..991de1b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogCursor.java
@@ -23,7 +23,7 @@
  */
 public interface ILogCursor {
 
-    public boolean next(LogicalLogLocator next) throws IOException, ACIDException;
+    public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException;
 
     public ILogFilter getLogFilter();
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
index 1158029..c629d03 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogManager.java
@@ -15,51 +15,30 @@
 package edu.uci.ics.asterix.transaction.management.service.logging;
 
 import java.io.IOException;
-import java.util.Map;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public interface ILogManager {
 
     /**
-     * An API to write a log record.
-     * 
-     * @param logicalLogLocator
-     *            A reusable object passed in by the caller. When the call
-     *            returns, this object has the physical location of the log
-     *            record that was written.
-     * @param context
-     *            the transaction context associated with the transaction that
-     *            is writing the log record
-     * @param resourceMgrId
-     *            the unique identifier of the resource manager that would be
-     *            handling (interpreting) the log record if there is a need to
-     *            process and apply the log record during a redo/undo task.
-     * @param pageId
-     *            the unique identifier of the page where the operation
-     *            corresponding to the log record is applied
      * @param logType
-     *            the type of log record (@see LogType)
-     * @param logActionType
-     *            the action that needs to be taken when processing the log
-     *            record (@see LogActionType)
-     * @param length
-     *            the length of the content inside the log record. This does not
-     *            include the header or the checksum size.
+     * @param context
+     * @param datasetId
+     * @param PKHashValue
+     * @param resourceId
+     * @param resourceMgrId
+     * @param logContentSize
+     * @param reusableLogContentObject
      * @param logger
-     *            an implementation of the @see ILogger interface that is
-     *            invoked by the ILogManager instance to get the actual content
-     *            for the log record.
-     * @param loggerArguments
-     *            Represent any additional arguments that needs to be passed
-     *            back in the call the to ILogger interface APIs.
+     * @param logicalLogLocator
      * @throws ACIDException
      */
-    public void log(LogicalLogLocator logicalLogLocator, TransactionContext context, byte resourceMgrId, long pageId,
-            byte logType, byte logActionType, int length, ILogger logger, Map<Object, Object> loggerArguments)
-            throws ACIDException;
+    void log(byte logType, TransactionContext context, int datasetId, int PKHashValue, long resourceId,
+            byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
+            LogicalLogLocator logicalLogLocator) throws ACIDException;
 
     /**
      * @param physicalLogLocator
@@ -85,13 +64,12 @@
     public ILogCursor readLog(ILogFilter logFilter) throws ACIDException;
 
     /**
+     * @param logicalLogLocator TODO
      * @param PhysicalLogLocator
      *            specifies the location of the log record to be read
-     * @return LogicalLogLocator represents the in-memory location of the log
-     *         record that has been fetched
      * @throws ACIDException
      */
-    public LogicalLogLocator readLog(PhysicalLogLocator physicalLogLocator) throws ACIDException;
+    public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
 
     /**
      * Flushes the log records up to the lsn represented by the
@@ -123,8 +101,8 @@
      * Returns the Transaction Provider associated with this ILogManager
      * instance
      * 
-     * @return TransactionProvider
+     * @return TransactionSubsystem
      */
-    public TransactionProvider getTransactionProvider();
+    public TransactionSubsystem getTransactionSubsystem();
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
index ab4bfea..80f74cb 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogRecordHelper.java
@@ -25,38 +25,41 @@
 
 public interface ILogRecordHelper {
 
-    public byte getLogType(LogicalLogLocator logicalLogLocator);
+    byte getLogType(LogicalLogLocator logicalLogLocator);
 
-    public int getLogLength(LogicalLogLocator logicalLogLocator);
+    int getJobId(LogicalLogLocator logicalLogLocator);
 
-    public long getLogTimestamp(LogicalLogLocator logicalLogLocator);
+    int getDatasetId(LogicalLogLocator logicalLogLocator);
 
-    public long getLogChecksum(LogicalLogLocator logicalLogLocator);
+    int getPKHashValue(LogicalLogLocator logicalLogLocator);
 
-    public long getLogTransactionId(LogicalLogLocator logicalLogLocator);
+    PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
 
-    public byte getResourceMgrId(LogicalLogLocator logicalLogLocator);
+    boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
+    
+    long getResourceId(LogicalLogLocator logicalLogLocator);
+    
+    byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
 
-    public long getPageId(LogicalLogLocator logicalLogLocator);
+    int getLogContentSize(LogicalLogLocator logicalLogLocater);
 
-    public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
+    long getLogChecksum(LogicalLogLocator logicalLogLocator);
 
-    public int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
+    int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
 
-    public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
+    int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
 
-    public byte getLogActionType(LogicalLogLocator logicalLogLocator);
+    String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
 
-    public PhysicalLogLocator getPreviousLsnByTransaction(LogicalLogLocator logicalLogLocator);
+    void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, TransactionContext context, int datasetId,
+            int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId, int logRecordSize);
 
-    public boolean getPreviousLsnByTransaction(PhysicalLogLocator physicalLogLocator,
-            LogicalLogLocator logicalLogLocator);
+    boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
 
-    public void writeLogHeader(TransactionContext context, LogicalLogLocator logicalLogLocator, byte resourceMgrId,
-            long pageId, byte logType, byte logActionType, int logContentSize, long prevLsnValue);
+    int getLogRecordSize(byte logType, int logBodySize);
 
-    public void writeLogTail(LogicalLogLocator logicalLogLocator, ILogManager logManager);
+    int getLogHeaderSize(byte logType);
 
-    public boolean validateLogRecord(LogManagerProperties logManagerProperties, LogicalLogLocator logicalLogLocator);
+    int getLogChecksumSize();
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
index 92984cb..e26a3cc 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/ILogger.java
@@ -14,9 +14,9 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.logging;
 
-import java.util.Map;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 
 /**
@@ -25,11 +25,11 @@
  */
 public interface ILogger {
 
-    public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException;
+    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException;
 
-    public void log(TransactionContext context, final LogicalLogLocator logicalLogLocator, int logRecordSize,
-            Map<Object, Object> loggerArguments) throws ACIDException;
+    public void log(TransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
+            ReusableLogContentObject reusableLogContentObject) throws ACIDException;
 
-    public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException;
+    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException;
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
new file mode 100644
index 0000000..d69a36e
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
@@ -0,0 +1,273 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+public class IndexLogger implements ILogger, ICloseable {
+
+    private final Map<Object, Object> jobId2ReusableLogContentObjectRepositoryMap = new ConcurrentHashMap<Object, Object>();
+
+    public static final String TREE_INDEX = "TREE_INDEX";
+    public static final String TUPLE_REFERENCE = "TUPLE_REFERENCE";
+    public static final String TUPLE_WRITER = "TUPLE_WRITER";
+    public static final String INDEX_OPERATION = "INDEX_OPERATION";
+    public static final String RESOURCE_ID = "RESOURCE_ID";
+
+    private final long resourceId;
+    private final byte resourceType;
+    private final SimpleTupleWriter tupleWriter;
+
+    public class BTreeOperationCodes {
+        public static final byte INSERT = 0;
+        public static final byte DELETE = 1;
+    }
+
+    public IndexLogger(long resourceId, byte resourceType, IIndex index) {
+        this.resourceId = resourceId;
+        this.resourceType = resourceType;
+        this.tupleWriter = new SimpleTupleWriter();
+    }
+
+    public synchronized void close(TransactionContext context) {
+        ReusableLogContentObjectRepository txnThreadStateRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
+                .get(context.getJobId());
+        txnThreadStateRepository.remove(Thread.currentThread().getId());
+        jobId2ReusableLogContentObjectRepositoryMap.remove(context.getJobId());
+    }
+
+    public void generateLogRecord(TransactionSubsystem txnSubsystem, TransactionContext context, int datasetId,
+            int PKHashValue, long resourceId, IndexOperation newOperation, ITupleReference newValue,
+            IndexOperation oldOperation, ITupleReference oldValue) throws ACIDException {
+
+        if (this.resourceId != resourceId) {
+            throw new ACIDException("IndexLogger mistach");
+        }
+
+        context.addCloseableResource(this); // the close method would be called
+        // on this TreeLogger instance at
+        // the time of transaction
+        // commit/abort.
+        if (newOperation != IndexOperation.INSERT && newOperation != IndexOperation.DELETE) {
+            throw new ACIDException("Loging for Operation " + newOperation + " not supported");
+        }
+
+        ReusableLogContentObject reusableLogContentObject = null;
+        ReusableLogContentObjectRepository reusableLogContentObjectRepository = null;
+        reusableLogContentObjectRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
+                .get(context.getJobId());
+        if (reusableLogContentObjectRepository == null) {
+            synchronized (context) { // threads belonging to different
+                // transaction do not need to
+                // synchronize amongst them.
+                if (reusableLogContentObjectRepository == null) {
+                    reusableLogContentObjectRepository = new ReusableLogContentObjectRepository();
+                    jobId2ReusableLogContentObjectRepositoryMap.put(context.getJobId(),
+                            reusableLogContentObjectRepository);
+                }
+            }
+        }
+
+        reusableLogContentObject = reusableLogContentObjectRepository.getObject(Thread.currentThread().getId());
+        if (reusableLogContentObject == null) {
+            LogicalLogLocator logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
+            reusableLogContentObject = new ReusableLogContentObject(logicalLogLocator, newOperation, newValue,
+                    oldOperation, oldValue);
+            reusableLogContentObjectRepository.putObject(Thread.currentThread().getId(), reusableLogContentObject);
+        } else {
+            reusableLogContentObject.setNewOperation(newOperation);
+            reusableLogContentObject.setNewValue(newValue);
+            reusableLogContentObject.setOldOperation(oldOperation);
+            reusableLogContentObject.setOldValue(oldValue);
+        }
+
+        int logContentSize = 4/*TupleFieldCount*/+ 1/*NewOperation*/+ 4/*newValueLength*/;
+        if (newValue != null) {
+            logContentSize += tupleWriter.bytesRequired(newValue);
+        }
+
+        logContentSize += 1/*OldOperation*/+ 4/*oldValueLength*/;
+        if (oldValue != null) {
+            logContentSize += tupleWriter.bytesRequired(oldValue);
+        }
+
+        txnSubsystem.getLogManager().log(LogType.UPDATE, context, datasetId, PKHashValue, resourceId, resourceType,
+                logContentSize, reusableLogContentObject, this, reusableLogContentObject.getLogicalLogLocator());
+    }
+
+    @Override
+    public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logContentSize,
+            ReusableLogContentObject reusableLogContentObject) throws ACIDException {
+        int offset = 0;
+        int tupleSize = 0;
+
+        //tuple field count
+        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, reusableLogContentObject
+                .getNewValue().getFieldCount());
+        offset += 4;
+
+        //new operation
+        (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + offset,
+                (byte) reusableLogContentObject.getNewOperation().ordinal());
+        offset += 1;
+
+        //new tuple size
+        if (reusableLogContentObject.getNewValue() != null) {
+            tupleSize = tupleWriter.bytesRequired(reusableLogContentObject.getNewValue());
+        }
+        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, tupleSize);
+        offset += 4;
+
+        //new tuple
+        if (tupleSize != 0) {
+            tupleWriter.writeTuple(reusableLogContentObject.getNewValue(), logicalLogLocator.getBuffer().getArray(),
+                    logicalLogLocator.getMemoryOffset() + offset);
+            offset += tupleSize;
+        }
+
+        if (resourceType == ResourceType.LSM_BTREE) {
+            //old operation
+            (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + offset,
+                    (byte) reusableLogContentObject.getOldOperation().ordinal());
+            offset += 1;
+
+            if (reusableLogContentObject.getOldOperation() != IndexOperation.NOOP) {
+                //old tuple size
+                if (reusableLogContentObject.getOldValue() != null) {
+                    tupleSize = tupleWriter.bytesRequired(reusableLogContentObject.getOldValue());
+                } else {
+                    tupleSize = 0;
+                }
+                (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, tupleSize);
+                offset += 4;
+
+                if (tupleSize != 0) {
+                    //old tuple
+                    tupleWriter.writeTuple(reusableLogContentObject.getOldValue(), logicalLogLocator.getBuffer()
+                            .getArray(), logicalLogLocator.getMemoryOffset() + offset);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject)
+            throws ACIDException {
+    }
+
+    @Override
+    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject)
+            throws ACIDException {
+    }
+
+    /**
+     * Represents a utility class for generating log records corresponding to
+     * operations on a ITreeIndex implementation. A TreeLogger instance is thread
+     * safe and can be shared across multiple threads that may belong to same or
+     * different transactions.
+     */
+    public class ReusableLogContentObjectRepository {
+
+        private final Map<Long, ReusableLogContentObject> id2Object = new HashMap<Long, ReusableLogContentObject>();
+
+        public synchronized ReusableLogContentObject getObject(long threadId) {
+            return id2Object.get(threadId);
+        }
+
+        public synchronized void putObject(long threadId, ReusableLogContentObject reusableLogContentObject) {
+            this.id2Object.put(threadId, reusableLogContentObject);
+        }
+
+        public synchronized void remove(long threadId) {
+            id2Object.remove(threadId);
+        }
+    }
+
+    /**
+     * Represents the state of a transaction thread. The state contains information
+     * that includes the tuple being operated, the operation and the location of the
+     * log record corresponding to the operation.
+     */
+    public class ReusableLogContentObject {
+
+        private LogicalLogLocator logicalLogLocator;
+        private IndexOperation newOperation;
+        private ITupleReference newValue;
+        private IndexOperation oldOperation;
+        private ITupleReference oldValue;
+
+        public ReusableLogContentObject(LogicalLogLocator logicalLogLocator, IndexOperation newOperation,
+                ITupleReference newValue, IndexOperation oldOperation, ITupleReference oldValue) {
+            this.logicalLogLocator = logicalLogLocator;
+            this.newOperation = newOperation;
+            this.newValue = newValue;
+            this.oldOperation = oldOperation;
+            this.oldValue = oldValue;
+        }
+
+        public synchronized LogicalLogLocator getLogicalLogLocator() {
+            return logicalLogLocator;
+        }
+
+        public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
+            this.logicalLogLocator = logicalLogLocator;
+        }
+
+        public synchronized void setNewOperation(IndexOperation newOperation) {
+            this.newOperation = newOperation;
+        }
+
+        public synchronized IndexOperation getNewOperation() {
+            return newOperation;
+        }
+
+        public synchronized void setNewValue(ITupleReference newValue) {
+            this.newValue = newValue;
+        }
+
+        public synchronized ITupleReference getNewValue() {
+            return newValue;
+        }
+
+        public synchronized void setOldOperation(IndexOperation oldOperation) {
+            this.oldOperation = oldOperation;
+        }
+
+        public synchronized IndexOperation getOldOperation() {
+            return oldOperation;
+        }
+
+        public synchronized void setOldValue(ITupleReference oldValue) {
+            this.oldValue = oldValue;
+        }
+
+        public synchronized ITupleReference getOldValue() {
+            return oldValue;
+        }
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
new file mode 100644
index 0000000..da5042c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.MutableResourceId;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+
+public class IndexLoggerRepository {
+
+    private final Map<MutableResourceId, IndexLogger> loggers = new HashMap<MutableResourceId, IndexLogger>();
+    private final TransactionSubsystem txnSubsystem;
+    private MutableResourceId mutableResourceId;
+
+    public IndexLoggerRepository(TransactionSubsystem provider) {
+        this.txnSubsystem = provider;
+        mutableResourceId = new MutableResourceId(0);
+    }
+
+    public synchronized IndexLogger getIndexLogger(long resourceId, byte resourceType) {
+        mutableResourceId.setId(resourceId);
+        IndexLogger logger = loggers.get(mutableResourceId);
+        if (logger == null) {
+            MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
+            IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(resourceId);
+            logger = new IndexLogger(resourceId, resourceType, index);
+            loggers.put(newMutableResourceId, logger);
+        }
+        return logger;
+    }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
new file mode 100644
index 0000000..7ecfa69
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+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.tuples.SimpleTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+
+public class IndexResourceManager implements IResourceManager {
+
+    public final byte resourceType;
+
+    private final TransactionSubsystem txnSubsystem;
+
+    public IndexResourceManager(byte resourceType, TransactionSubsystem provider) {
+        this.resourceType = resourceType;
+        this.txnSubsystem = provider;
+    }
+
+    public byte getResourceManagerId() {
+        return resourceType;
+    }
+
+    public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
+        long resourceId = logRecordHelper.getResourceId(logLocator);
+        int offset = logRecordHelper.getLogContentBeginPos(logLocator);
+
+        //TODO
+        //replace TransactionResourceRepository with IndexLifeCycleManager
+        // look up the repository to obtain the resource object
+        IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                .getIndex(resourceId);
+
+        /* field count */
+        int fieldCount = logLocator.getBuffer().readInt(offset);
+        offset += 4;
+
+        /* new operation */
+        byte newOperation = logLocator.getBuffer().getByte(offset);
+        offset += 1;
+
+        /* new value size */
+        int newValueSize = logLocator.getBuffer().readInt(offset);
+        offset += 4;
+
+        /* new value */
+        SimpleTupleWriter tupleWriter = new SimpleTupleWriter();
+        SimpleTupleReference newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
+        newTuple.setFieldCount(fieldCount);
+        newTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
+        offset += newValueSize;
+
+        ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+
+        try {
+            if (resourceType == ResourceType.LSM_BTREE) {
+
+                /* old operation */
+                byte oldOperation = logLocator.getBuffer().getByte(offset);
+                offset += 1;
+
+                if (oldOperation != (byte) IndexOperation.NOOP.ordinal()) {
+                    /* old value size */
+                    int oldValueSize = logLocator.getBuffer().readInt(offset);
+                    offset += 4;
+
+                    /* old value */
+                    SimpleTupleReference oldTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
+                    oldTuple.setFieldCount(fieldCount);
+                    oldTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
+                    offset += oldValueSize;
+
+                    if (oldOperation == (byte) IndexOperation.DELETE.ordinal()) {
+                        indexAccessor.forceDelete(oldTuple);
+                    } else {
+                        indexAccessor.forceInsert(oldTuple);
+                    }
+                } else {
+                    indexAccessor.forcePhysicalDelete(newTuple);
+                }
+            } else {
+                if (newOperation == (byte) IndexOperation.DELETE.ordinal()) {
+                    indexAccessor.forceInsert(newTuple);
+                } else {
+                    indexAccessor.forceDelete(newTuple);
+                }
+            }
+        } catch (Exception e) {
+            throw new ACIDException("Undo failed", e);
+        }
+    }
+
+    public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
+        long resourceId = logRecordHelper.getResourceId(logLocator);
+        int offset = logRecordHelper.getLogContentBeginPos(logLocator);
+
+        IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                .getIndex(resourceId);
+
+        /* field count */
+        int fieldCount = logLocator.getBuffer().readInt(offset);
+        offset += 4;
+
+        /* new operation */
+        byte newOperation = logLocator.getBuffer().getByte(offset);
+        offset += 1;
+
+        /* new value size */
+        int newValueSize = logLocator.getBuffer().readInt(offset);
+        offset += 4;
+
+        /* new value */
+        SimpleTupleWriter tupleWriter = new SimpleTupleWriter();
+        SimpleTupleReference newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
+        newTuple.setFieldCount(fieldCount);
+        newTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
+        offset += newValueSize;
+
+        ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+                NoOpOperationCallback.INSTANCE);
+
+        try {
+            if (newOperation == IndexOperation.INSERT.ordinal()) {
+                indexAccessor.insert(newTuple);
+            } else if (newOperation == IndexOperation.DELETE.ordinal()) {
+                indexAccessor.delete(newTuple);
+            } else {
+                new ACIDException("Unsupported operation type for undo operation : " + newOperation);
+            }
+        } catch (Exception e) {
+            throw new ACIDException("Redo failed", e);
+        }
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
index 7ba9130..8a2b188 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
@@ -21,33 +21,34 @@
 
 public class LogCursor implements ILogCursor {
 
-    private final ILogManager logManager;
+    private final LogManager logManager;
     private final ILogFilter logFilter;
-    private IFileBasedBuffer readOnlyBuffer;
+    private IBuffer readOnlyBuffer;
     private LogicalLogLocator logicalLogLocator = null;
-    private int bufferIndex = 0;
+    private long bufferIndex = 0;
+    private boolean firstNext = true;
+    private boolean readMemory = false;
+    private long readLSN = 0;
+    private boolean needReloadBuffer = true;
 
     /**
      * @param logFilter
      */
-    public LogCursor(final ILogManager logManager, ILogFilter logFilter) throws ACIDException {
+    public LogCursor(final LogManager logManager, ILogFilter logFilter) throws ACIDException {
         this.logFilter = logFilter;
         this.logManager = logManager;
 
     }
 
-    public LogCursor(final ILogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter)
-            throws IOException {
+    public LogCursor(final LogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter)
+            throws IOException, ACIDException {
         this.logFilter = logFilter;
         this.logManager = logManager;
         initialize(startingPhysicalLogLocator);
     }
 
-    private void initialize(final PhysicalLogLocator startingPhysicalLogLocator) throws IOException {
-        readOnlyBuffer = getReadOnlyBuffer(startingPhysicalLogLocator.getLsn(), logManager.getLogManagerProperties()
-                .getLogBufferSize());
-        logicalLogLocator = new LogicalLogLocator(startingPhysicalLogLocator.getLsn(), readOnlyBuffer, 0, logManager);
-
+    private void initialize(final PhysicalLogLocator startingPhysicalLogLocator) throws IOException, ACIDException {
+        logicalLogLocator = new LogicalLogLocator(startingPhysicalLogLocator.getLsn(), null, 0, logManager);
     }
 
     private IFileBasedBuffer getReadOnlyBuffer(long lsn, int size) throws IOException {
@@ -55,7 +56,8 @@
         String filePath = LogUtil.getLogFilePath(logManager.getLogManagerProperties(), fileId);
         File file = new File(filePath);
         if (file.exists()) {
-            return FileUtil.getFileBasedBuffer(filePath, lsn, size);
+            return FileUtil.getFileBasedBuffer(filePath, lsn
+                    % logManager.getLogManagerProperties().getLogPartitionSize(), size);
         } else {
             return null;
         }
@@ -66,21 +68,49 @@
      * filter. The parameter nextLogLocator is set to the point to the next log
      * record.
      * 
-     * @param nextLogicalLogLocator
+     * @param currentLogLocator
      * @return true if the cursor was successfully moved to the next log record
      *         false if there are no more log records that satisfy the
      *         configured filter.
      */
     @Override
-    public boolean next(LogicalLogLocator nextLogicalLogLocator) throws IOException, ACIDException {
+    public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException {
 
+        //TODO
+        //Test the correctness when multiple log files are created
         int integerRead = -1;
         boolean logRecordBeginPosFound = false;
         long bytesSkipped = 0;
+
+        //if the lsn to read is greater than or equal to the most recent lsn, then return false
+        if (logicalLogLocator.getLsn() >= logManager.getCurrentLsn().get()) {
+            return false;
+        }
+
+        //if the lsn to read is greater than the last flushed lsn, then read from memory
+        if (logicalLogLocator.getLsn() > logManager.getLastFlushedLsn().get()) {
+            return readFromMemory(currentLogLocator);
+        }
+
+        //if the readOnlyBuffer should be reloaded, then load the log page from the log file.
+        //needReloadBuffer is set to true if the log record is read from the memory log page.
+        if (needReloadBuffer) {
+            //log page size doesn't exceed integer boundary
+            int offset = (int)(logicalLogLocator.getLsn() % logManager.getLogManagerProperties().getLogPageSize());
+            long adjustedLSN = logicalLogLocator.getLsn() - offset;
+            readOnlyBuffer = getReadOnlyBuffer(adjustedLSN, logManager.getLogManagerProperties()
+                    .getLogPageSize());
+            logicalLogLocator.setBuffer(readOnlyBuffer);
+            logicalLogLocator.setMemoryOffset(offset);
+            needReloadBuffer = false;
+        }
+
+        //check whether the currentOffset has enough space to have new log record by comparing
+        //the smallest log record type(which is commit)'s log header.
         while (logicalLogLocator.getMemoryOffset() <= readOnlyBuffer.getSize()
-                - logManager.getLogManagerProperties().getLogHeaderSize()) {
+                - logManager.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
             integerRead = readOnlyBuffer.readInt(logicalLogLocator.getMemoryOffset());
-            if (integerRead == logManager.getLogManagerProperties().logMagicNumber) {
+            if (integerRead == logManager.getLogManagerProperties().LOG_MAGIC_NUMBER) {
                 logRecordBeginPosFound = true;
                 break;
             }
@@ -93,36 +123,46 @@
                 // bytes without finding a log record, it
                 // indicates an absence of logs any further.
             }
+
+            if (logicalLogLocator.getLsn() > logManager.getLastFlushedLsn().get()) {
+                return next(currentLogLocator); //should read from memory if there is any further log
+            }
         }
 
         if (!logRecordBeginPosFound) {
             // need to reload the buffer
-            long lsnpos = (++bufferIndex * logManager.getLogManagerProperties().getLogBufferSize());
-            readOnlyBuffer = getReadOnlyBuffer(lsnpos, logManager.getLogManagerProperties().getLogBufferSize());
+            // TODO
+            // reduce IO by reading more pages(equal to logBufferSize) at a time.
+            long lsnpos = ((logicalLogLocator.getLsn() / logManager.getLogManagerProperties().getLogPageSize()) + 1)
+                    * logManager.getLogManagerProperties().getLogPageSize();
+
+            readOnlyBuffer = getReadOnlyBuffer(lsnpos, logManager.getLogManagerProperties().getLogPageSize());
             if (readOnlyBuffer != null) {
                 logicalLogLocator.setBuffer(readOnlyBuffer);
                 logicalLogLocator.setLsn(lsnpos);
                 logicalLogLocator.setMemoryOffset(0);
-                return next(nextLogicalLogLocator);
+                return next(currentLogLocator);
             } else {
                 return false;
             }
         }
 
-        int logLength = logManager.getLogRecordHelper().getLogLength(logicalLogLocator);
-        if (logManager.getLogRecordHelper().validateLogRecord(logManager.getLogManagerProperties(), logicalLogLocator)) {
-            if (nextLogicalLogLocator == null) {
-                nextLogicalLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
+        int logLength = logManager.getLogRecordHelper().getLogRecordSize(
+                logManager.getLogRecordHelper().getLogType(logicalLogLocator),
+                logManager.getLogRecordHelper().getLogContentSize(logicalLogLocator));
+        if (logManager.getLogRecordHelper().validateLogRecord(logicalLogLocator)) {
+            if (currentLogLocator == null) {
+                currentLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
             }
-            nextLogicalLogLocator.setLsn(logicalLogLocator.getLsn());
-            nextLogicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
-            nextLogicalLogLocator.setBuffer(readOnlyBuffer);
+            currentLogLocator.setLsn(logicalLogLocator.getLsn());
+            currentLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
+            currentLogLocator.setBuffer(readOnlyBuffer);
             logicalLogLocator.incrementLsn(logLength);
             logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
         } else {
             throw new ACIDException("Invalid Log Record found ! checksums do not match :( ");
         }
-        return logFilter.accept(readOnlyBuffer, nextLogicalLogLocator.getMemoryOffset(), logLength);
+        return logFilter.accept(readOnlyBuffer, currentLogLocator.getMemoryOffset(), logLength);
     }
 
     /**
@@ -135,4 +175,98 @@
         return logFilter;
     }
 
+    private boolean readFromMemory(LogicalLogLocator currentLogLocator) throws ACIDException, IOException {
+        byte[] logRecord = null;
+        long lsn = logicalLogLocator.getLsn();
+
+        //set the needReloadBuffer to true
+        needReloadBuffer = true;
+
+        int pageIndex = logManager.getLogPageIndex(lsn);
+        logicalLogLocator.setMemoryOffset(logManager.getLogPageOffset(lsn));
+
+        // take a lock on the log page so that the page is not flushed to
+        // disk interim
+        IFileBasedBuffer logPage = logManager.getLogPage(pageIndex);
+        synchronized (logPage) {
+            // need to check again if the log record in the log buffer or has reached the disk
+            if (lsn > logManager.getLastFlushedLsn().get()) {
+
+                //find the magic number to identify the start of the log record
+                //----------------------------------------------------------------
+                int readNumber = -1;
+                int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
+                int logMagicNumber = logManager.getLogManagerProperties().LOG_MAGIC_NUMBER;
+                int bytesSkipped = 0;
+                boolean logRecordBeginPosFound = false;
+                //check whether the currentOffset has enough space to have new log record by comparing
+                //the smallest log record type(which is commit)'s log header.
+                while (logicalLogLocator.getMemoryOffset() <= logPageSize
+                        - logManager.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
+                    readNumber = logPage.readInt(logicalLogLocator.getMemoryOffset());
+                    if (readNumber == logMagicNumber) {
+                        logRecordBeginPosFound = true;
+                        break;
+                    }
+                    logicalLogLocator.increaseMemoryOffset(1);
+                    logicalLogLocator.incrementLsn();
+                    bytesSkipped++;
+                    if (bytesSkipped > logPageSize) {
+                        return false; // the maximum size of a log record is limited to
+                        // a log page size. If we have skipped as many
+                        // bytes without finding a log record, it
+                        // indicates an absence of logs any further.
+                    }
+                }
+
+                if (!logRecordBeginPosFound) {
+                    // need to read the next log page
+                    readOnlyBuffer = null;
+                    logicalLogLocator.setBuffer(null);
+                    logicalLogLocator.setLsn(lsn / logPageSize + 1);
+                    logicalLogLocator.setMemoryOffset(0);
+                    return next(currentLogLocator);
+                }
+                //------------------------------------------------------
+
+                logicalLogLocator.setBuffer(logPage);
+                int logLength = logManager.getLogRecordHelper().getLogRecordSize(
+                        logManager.getLogRecordHelper().getLogType(logicalLogLocator),
+                        logManager.getLogRecordHelper().getLogContentSize(logicalLogLocator));
+                logRecord = new byte[logLength];
+
+                //copy the log record and set the buffer of logical log locator to the buffer of the copied log record.
+                System.arraycopy(logPage.getArray(), logicalLogLocator.getMemoryOffset(), logRecord, 0, logLength);
+                MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
+                readOnlyBuffer = memBuffer;
+                logicalLogLocator.setBuffer(readOnlyBuffer);
+                logicalLogLocator.setMemoryOffset(0);
+
+                if (logManager.getLogRecordHelper().validateLogRecord(logicalLogLocator)) {
+                    if (currentLogLocator == null) {
+                        currentLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
+                    }
+                    currentLogLocator.setLsn(logicalLogLocator.getLsn());
+                    currentLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
+                    currentLogLocator.setBuffer(readOnlyBuffer);
+                    logicalLogLocator.incrementLsn(logLength);
+                    logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
+                } else {
+                    //if the checksum doesn't match, there is two possible scenario. 
+                    //case1) the log file corrupted: there's nothing we can do for this case during abort. 
+                    //case2) the log record is partially written by another thread. So, we may ignore this log record 
+                    //       and continue to read the next log record
+                    //[NOTICE]
+                    //Only case2 is handled here. 
+                    logicalLogLocator.incrementLsn(logLength);
+                    logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
+                    return next(currentLogLocator);
+                }
+                return logFilter.accept(readOnlyBuffer, currentLogLocator.getMemoryOffset(), logLength);
+
+            } else {
+                return next(currentLogLocator);//read from disk
+            }
+        }
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 1d1e067..4e96d2e 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * 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
@@ -15,14 +15,17 @@
 package edu.uci.ics.asterix.transaction.management.service.logging;
 
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
@@ -30,29 +33,26 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogManager.PageOwnershipStatus;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogManager.PageState;
 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.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 public class LogManager implements ILogManager {
 
-    /*
-     * Log Record Structure HEADER
-     * <(log_magic_number,4)(log_length,8)(log_type,1
-     * )(log_action_type,1)(log_timestamp
-     * ,8)(log_transaction_id,8)(resource_manager_id
-     * ,1)(page_id,8)(previous_lsn,8) <CONTENT> TAIL <(checksum,8)>
-     */
-
+    public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
-    private TransactionProvider provider;
+    private final TransactionSubsystem provider;
     private LogManagerProperties logManagerProperties;
+    private LogPageFlushThread logPageFlusher;
 
     /*
      * the array of log pages. The number of log pages is configurable. Pages
      * taken together form an in-memory log buffer.
      */
-
     private IFileBasedBuffer[] logPages;
 
     private ILogRecordHelper logRecordHelper;
@@ -61,6 +61,7 @@
      * Number of log pages that constitute the in-memory log buffer.
      */
     private int numLogPages;
+
     /*
      * Initially all pages have an owner count of 1 that is the LogManager. When
      * a transaction requests to write in a log page, the owner count is
@@ -69,12 +70,11 @@
      * (covering the whole log record). When the content has been put, the log
      * manager computes the checksum and puts it after the content. At this
      * point, the ownership count is decremented as the transaction is done with
-     * using the page. When a page is full, the log manager decrements the count
-     * by one indicating that it has released its ownership of the log page.
-     * There could be other transaction(s) still owning the page (that is they
-     * could still be mid-way putting the log content). When the ownership count
-     * eventually reaches zero, the thread responsible for flushing the log page
-     * is notified and the page is flushed to disk.
+     * using the page. When a page is requested to be flushed, logPageFlusher
+     * set the count to 0(LOG_FLUSHER: meaning that the page is being flushed)
+     * only if the count is 1(LOG_WRITER: meaning that there is no other
+     * transactions who own the page to write logs.) After flushing the page,
+     * logPageFlusher set this count to 1.
      */
     private AtomicInteger[] logPageOwnerCount;
 
@@ -85,18 +85,17 @@
 
     /*
      * LogPageStatus: A page is either ACTIVE or INACTIVE. The status for each
-     * page is maintained in a map called logPageStatus. A page is ACTIVE when
-     * the LogManager can allocate space in the page for writing a log record.
-     * Initially all pages are ACTIVE. As transactions fill up space by writing
-     * log records, a page may not have sufficient space left for serving a
-     * request by a transaction. When this happens, the page is marked INACTIVE.
-     * An INACTIVE page with no owners ( logPageOwnerCount.get(<pageIndex>) ==
-     * 0) indicates that the page must be flushed to disk before any other log
-     * record is written on the page.F
+     * page is maintained in logPageStatus. A page is ACTIVE when the LogManager
+     * can allocate space in the page for writing a log record. Initially all
+     * pages are ACTIVE. As transactions fill up space by writing log records, a
+     * page may not have sufficient space left for serving a request by a
+     * transaction. When this happens, the page is flushed to disk by calling
+     * logPageFlusher.requestFlush(). In the requestFlush(), after
+     * groupCommitWaitTime, the page status is set to INACTIVE. Then, there is
+     * no more writer on the page(meaning the corresponding logPageOwnerCount is
+     * 1), the page is flushed by the logPageFlusher and the status is reset to
+     * ACTIVE by the logPageFlusher.
      */
-
-    // private Map<Integer, Integer> logPageStatus = new
-    // ConcurrentHashMap<Integer, Integer>();
     private AtomicInteger[] logPageStatus;
 
     static class PageState {
@@ -104,40 +103,7 @@
         public static final int ACTIVE = 1;
     }
 
-    private AtomicLong lastFlushedLsn = new AtomicLong(-1);
-    private AtomicInteger lastFlushedPage = new AtomicInteger(-1);
-
-    /*
-     * pendingFlushRequests is a map with key as Integer denoting the page
-     * index. When a (transaction) thread discovers the need to flush a page, it
-     * puts its Thread object into the corresponding value that is a
-     * LinkedBlockingQueue. The LogManager has a LogFlusher thread that scans
-     * this map in order of page index (and circling around). The flusher thread
-     * needs to flush pages in order and waits for a thread to deposit an object
-     * in the blocking queue corresponding to the next page in order. A request
-     * to flush a page is conveyed to the flush thread by simply depositing an
-     * object in to corresponding blocking queue. It is blocking in the sense
-     * that the flusher thread will continue to wait for an object to arrive in
-     * the queue. The object itself is ignored by the fliusher and just acts as
-     * a signal/event that a page needs to be flushed.
-     */
-
-    private LinkedBlockingQueue[] pendingFlushRequests;
-
-    /*
-     * ICommitResolver is an interface that provides an API that can answer a
-     * simple boolean - Given the commit requests so far, should a page be
-     * flushed. The implementation of the interface contains the logic (or you
-     * can say the policy) for commit. It could be group commit in which case
-     * the commit resolver may not return a true indicating that it wishes to
-     * delay flushing of the page.
-     */
-    private ICommitResolver commitResolver;
-
-    /*
-     * An object that keeps track of the submitted commit requests.
-     */
-    private CommitRequestStatistics commitRequestStatistics;
+    private AtomicLong lastFlushedLSN = new AtomicLong(-1);
 
     /*
      * When the transaction eco-system comes to life, the log manager positions
@@ -145,7 +111,7 @@
      * lsn value of the next log record to be written after a system (re)start.
      * The value is zero when the system is starting for the first time.
      */
-    private long startingLsn = 0;
+    private long startingLSN = 0;
 
     /*
      * lsn represents the monotonically increasing long value that can be broken
@@ -153,20 +119,14 @@
      */
     private AtomicLong lsn = new AtomicLong(0);
 
-    /*
-     * A map that tracks the flush requests submitted for each page. The
-     * requests for a page are cleared when the page is flushed.
-     */
-    public LinkedBlockingQueue<Thread> getPendingFlushRequests(int pageIndex) {
-        return pendingFlushRequests[pageIndex];
-    }
+    private List<HashMap<TransactionContext, Integer>> activeTxnCountMaps;
 
-    public void addFlushRequest(int pageIndex) {
-        pendingFlushRequests[pageIndex].add(Thread.currentThread());
+    public void addFlushRequest(int pageIndex, long lsn, boolean isSynchronous) {
+        logPageFlusher.requestFlush(pageIndex, lsn, isSynchronous);
     }
 
     public AtomicLong getLastFlushedLsn() {
-        return lastFlushedLsn;
+        return lastFlushedLSN;
     }
 
     public AtomicInteger getLogPageStatus(int pageIndex) {
@@ -178,18 +138,18 @@
     }
 
     public long incrementLastFlushedLsn(long delta) {
-        return lastFlushedLsn.addAndGet(delta);
+        return lastFlushedLSN.addAndGet(delta);
     }
 
-    public LogManager(TransactionProvider provider) throws ACIDException {
+    public LogManager(TransactionSubsystem provider) throws ACIDException {
         this.provider = provider;
-        initLogManagerProperties(null);
+        initLogManagerProperties(this.provider.getId());
         initLogManager();
     }
 
-    public LogManager(TransactionProvider provider, LogManagerProperties logConfiguration) throws ACIDException {
+    public LogManager(TransactionSubsystem provider, String nodeId) throws ACIDException {
         this.provider = provider;
-        initLogManagerProperties(logConfiguration);
+        initLogManagerProperties(nodeId);
         initLogManager();
     }
 
@@ -197,42 +157,30 @@
      * initialize the log manager properties either from the configuration file
      * on disk or with default values
      */
-    private void initLogManagerProperties(LogManagerProperties logConfiguration) throws ACIDException {
-        if (logConfiguration == null) {
-            InputStream is = null;
-            try {
-                File file = new File(TransactionManagementConstants.LogManagerConstants.LOG_CONF_DIR
-                        + File.pathSeparator + TransactionManagementConstants.LogManagerConstants.LOG_CONF_FILE);
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("Log Configuration file path is " + file.getAbsolutePath());
-                }
-                if (file.exists()) {
-                    is = new FileInputStream(TransactionManagementConstants.LogManagerConstants.LOG_CONF_DIR
-                            + File.pathSeparator + TransactionManagementConstants.LogManagerConstants.LOG_CONF_FILE);
-                    Properties configuredProperties = new Properties();
-                    configuredProperties.load(is);
-                    logConfiguration = new LogManagerProperties(configuredProperties);
-                } else {
-                    if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("Log configuration file not found, using defaults !");
-                    }
-                    Properties configuredProperties = new Properties();
-                    configuredProperties.setProperty(LogManagerProperties.LOG_DIR_KEY,
-                            TransactionManagementConstants.LogManagerConstants.DEFAULT_LOG_DIR + File.separator
-                                    + provider.getId());
-                    logConfiguration = new LogManagerProperties(configuredProperties);
-                }
-            } catch (IOException ioe) {
-                if (is != null) {
-                    try {
-                        is.close();
-                    } catch (IOException e) {
-                        throw new ACIDException("unable to close input stream ", e);
-                    }
+    private void initLogManagerProperties(String nodeId) throws ACIDException {
+        LogManagerProperties logProperties = null;
+        InputStream is = null;
+        try {
+            is = this.getClass().getClassLoader()
+                    .getResourceAsStream(TransactionManagementConstants.LogManagerConstants.LOG_CONF_FILE);
+
+            Properties p = new Properties();
+
+            if (is != null) {
+                p.load(is);
+            }
+            logProperties = new LogManagerProperties(p, nodeId);
+
+        } catch (IOException ioe) {
+            if (is != null) {
+                try {
+                    is.close();
+                } catch (IOException e) {
+                    throw new ACIDException("unable to close input stream ", e);
                 }
             }
         }
-        logManagerProperties = logConfiguration;
+        logManagerProperties = logProperties;
     }
 
     private void initLogManager() throws ACIDException {
@@ -240,38 +188,25 @@
         numLogPages = logManagerProperties.getNumLogPages();
         logPageOwnerCount = new AtomicInteger[numLogPages];
         logPageStatus = new AtomicInteger[numLogPages];
-        pendingFlushRequests = new LinkedBlockingQueue[numLogPages];
-        if (logManagerProperties.getGroupCommitWaitPeriod() > 0) { // configure
-            // the
-            // Commit
-            // Resolver
-            commitResolver = new GroupCommitResolver(); // Group Commit is
-            // enabled
-            commitRequestStatistics = new CommitRequestStatistics(numLogPages);
-        } else {
-            commitResolver = new BasicCommitResolver(); // the basic commit
-            // resolver
+
+        activeTxnCountMaps = new ArrayList<HashMap<TransactionContext, Integer>>(numLogPages);
+        for (int i = 0; i < numLogPages; i++) {
+            activeTxnCountMaps.add(new HashMap<TransactionContext, Integer>());
         }
-        this.commitResolver.init(this); // initialize the commit resolver
+
         logPages = new FileBasedBuffer[numLogPages];
 
         /*
          * place the log anchor at the end of the last log record written.
          */
-        PhysicalLogLocator nextPhysicalLsn = LogUtil.initializeLogAnchor(this);
-        startingLsn = nextPhysicalLsn.getLsn();
-        lastFlushedLsn.set(startingLsn - 1);
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" Starting lsn is : " + startingLsn);
-        }
-        lsn.set(startingLsn);
+        PhysicalLogLocator nextPhysicalLsn = initLSN();
+
         /*
          * initialize meta data for each log page.
          */
         for (int i = 0; i < numLogPages; i++) {
             logPageOwnerCount[i] = new AtomicInteger(PageOwnershipStatus.LOG_WRITER);
             logPageStatus[i] = new AtomicInteger(PageState.ACTIVE);
-            pendingFlushRequests[i] = new LinkedBlockingQueue<Thread>();
         }
 
         /*
@@ -285,13 +220,13 @@
          * daemon thread so that it does not stop the JVM from exiting when all
          * other threads are done with their work.
          */
-        LogPageFlushThread logFlusher = new LogPageFlushThread(this);
-        logFlusher.setDaemon(true);
-        logFlusher.start();
+        logPageFlusher = new LogPageFlushThread(this);
+        logPageFlusher.setDaemon(true);
+        logPageFlusher.start();
     }
 
     public int getLogPageIndex(long lsnValue) {
-        return (int) ((lsnValue - startingLsn) / logManagerProperties.getLogPageSize()) % numLogPages;
+        return (int) ((lsnValue - startingLSN) / logManagerProperties.getLogPageSize()) % numLogPages;
 
     }
 
@@ -309,7 +244,7 @@
      * record is (to be) placed.
      */
     public int getLogPageOffset(long lsnValue) {
-        return (int) (lsnValue - startingLsn) % logManagerProperties.getLogPageSize();
+        return (int) (lsnValue - startingLSN) % logManagerProperties.getLogPageSize();
     }
 
     /*
@@ -319,7 +254,7 @@
      */
     private void waitUntillPageIsAvailableForWritingLog(int pageIndex) throws ACIDException {
         if (logPageStatus[pageIndex].get() == PageState.ACTIVE
-                && getLogPageOwnershipCount(pageIndex).get() >= PageOwnershipStatus.LOG_WRITER) {
+                && logPageOwnerCount[pageIndex].get() >= PageOwnershipStatus.LOG_WRITER) {
             return;
         }
         try {
@@ -345,47 +280,47 @@
      */
     private long getLsn(int entrySize, byte logType) throws ACIDException {
         long pageSize = logManagerProperties.getLogPageSize();
-        boolean requiresFlushing = logType == LogType.COMMIT;
+
         while (true) {
             boolean forwardPage = false;
-            boolean shouldFlushPage = false;
             long old = lsn.get();
-            int pageIndex = getLogPageIndex(old); // get the log page
-            // corresponding to the
-            // current lsn value
+
+            // get the log page corresponding to the current lsn value
+            int pageIndex = getLogPageIndex(old);
             long retVal = old;
-            long next = old + entrySize; // the lsn value for the next request,
-            // if the current request is served.
+
+            // the lsn value for the next request if the current request is
+            // served.
+            long next = old + entrySize;
             int prevPage = -1;
-            if ((next - 1) / pageSize != old / pageSize // check if the log
-            // record will cross
-            // page boundaries, a
-            // case that is not
-            // allowed.
-                    || (next % pageSize == 0)) {
+
+            // check if the log record will cross page boundaries, a case that
+            // is not allowed.
+            if ((next - 1) / pageSize != old / pageSize || (next % pageSize == 0)) {
+
                 if ((old != 0 && old % pageSize == 0)) {
-                    retVal = old; // On second thought, this shall never be the
-                    // case as it means that the lsn is
-                    // currently at the beginning of a page and
-                    // we still need to forward the page which
-                    // means that the entrySize exceeds a log
-                    // page size. If this is the case, an
-                    // exception is thrown before calling this
-                    // API.
-                    // would remove this case.
+                    // On second thought, this shall never be the case as it
+                    // means that the lsn is
+                    // currently at the beginning of a page and we still need to
+                    // forward the page which
+                    // means that the entrySize exceeds a log page size. If this
+                    // is the case, an
+                    // exception is thrown before calling this API. would remove
+                    // this case.
+                    retVal = old;
 
                 } else {
-                    retVal = ((old / pageSize) + 1) * pageSize; // set the lsn
-                    // to point to
-                    // the beginning
-                    // of the next
-                    // page.
+                    // set the lsn to point to the beginning of the next page.
+                    retVal = ((old / pageSize) + 1) * pageSize;
                 }
+
                 next = retVal;
-                forwardPage = true; // as the log record shall cross log page
-                // boundary, we must re-assign the lsn (so
-                // that the log record begins on a different
-                // location.
+
+                // as the log record shall cross log page boundary, we must
+                // re-assign the lsn so
+                // that the log record begins on a different location.
+                forwardPage = true;
+
                 prevPage = pageIndex;
                 pageIndex = getNextPageInSequence(pageIndex);
             }
@@ -404,140 +339,92 @@
              */
             waitUntillPageIsAvailableForWritingLog(pageIndex);
 
-            if (!forwardPage && requiresFlushing) {
-                shouldFlushPage = commitResolver.shouldCommitPage(pageIndex, this, commitRequestStatistics);
-                if (shouldFlushPage) {
-                    next = ((next / pageSize) + 1) * pageSize; /*
-                                                                * next
-                                                                * represents the
-                                                                * next value of
-                                                                * lsn after this
-                                                                * log record has
-                                                                * been written.
-                                                                * If the page
-                                                                * needs to be
-                                                                * flushed, then
-                                                                * we do not give
-                                                                * any more LSNs
-                                                                * from this
-                                                                * page.
-                                                                */
-                }
-            }
-            if (!lsn.compareAndSet(old, next)) { // Atomic call -> returns true
-                // only when the value
-                // represented by lsn is same as
-                // "old". The value is updated
-                // to "next".
+            if (!lsn.compareAndSet(old, next)) {
+                // Atomic call -> returns true only when the value represented
+                // by lsn is same as
+                // "old". The value is updated to "next".
                 continue;
             }
 
             if (forwardPage) {
-                logPageStatus[prevPage].set(PageState.INACTIVE); // mark
-                // previous
-                // page
-                // inactive
+                addFlushRequest(prevPage, old, false);
 
-                /*
-                 * decrement on the behalf of the log manager. if there are no
-                 * more owners (count == 0) the page must be marked as a
-                 * candidate to be flushed.
-                 */
-                int pageDirtyCount = getLogPageOwnershipCount(prevPage).decrementAndGet();
-                if (pageDirtyCount == 0) {
-                    addFlushRequest(prevPage);
-                }
-
-                /*
-                 * The transaction thread that discovers the need to forward a
-                 * page is made to re-acquire a lsn.
-                 */
+                // The transaction thread that discovers the need to forward a
+                // page is made to re-acquire a lsn.
                 continue;
+
             } else {
-                /*
-                 * the transaction thread has been given a space in a log page,
-                 * but is made to wait until the page is available.
-                 */
+                // the transaction thread has been given a space in a log page,
+                // but is made to wait until the page is available.
+                // (Is this needed? when does this wait happen?)
                 waitUntillPageIsAvailableForWritingLog(pageIndex);
-                /*
-                 * increment the counter as the transaction thread now holds a
-                 * space in the log page and hence is an owner.
-                 */
+
+                // increment the counter as the transaction thread now holds a
+                // space in the log page and hence is an owner.
                 logPageOwnerCount[pageIndex].incrementAndGet();
-            }
-            if (requiresFlushing) {
-                if (!shouldFlushPage) {
-                    /*
-                     * the log record requires the page to be flushed but under
-                     * the commit policy, the flush task has been deferred. The
-                     * transaction thread submits its request to flush the page.
-                     */
-                    commitRequestStatistics.registerCommitRequest(pageIndex);
-                } else {
-                    /*
-                     * the flush request was approved by the commit resolver.
-                     * Thus the page is marked INACTIVE as no more logs will be
-                     * written on this page. The log manager needs to release
-                     * its ownership. Note that transaction threads may still
-                     * continue to be owners of the log page till they fill up
-                     * the space allocated to them.
-                     */
-                    logPageStatus[pageIndex].set(PageState.INACTIVE);
-                    logPageOwnerCount[pageIndex].decrementAndGet(); // on
-                    // the
-                    // behalf
-                    // of
-                    // log
-                    // manager
+
+                // Before the count is incremented, if the flusher flushed the
+                // allocated page,
+                // then retry to get new LSN. Otherwise, the log with allocated
+                // lsn will be lost.
+                if (lastFlushedLSN.get() >= retVal) {
+                    logPageOwnerCount[pageIndex].decrementAndGet();
+                    continue;
                 }
             }
+
             return retVal;
         }
     }
 
-    public void log(LogicalLogLocator logLocator, TransactionContext context, byte resourceMgrId, long pageId,
-            byte logType, byte logActionType, int requestedSpaceForLog, ILogger logger,
-            Map<Object, Object> loggerArguments) throws ACIDException {
-        /*
-         * logLocator is a re-usable object that is appropriately set in each
-         * invocation. If the reference is null, the log manager must throw an
-         * exception
-         */
-        if (logLocator == null) {
+    @Override
+    public void log(byte logType, TransactionContext txnCtx, int datasetId, int PKHashValue, long resourceId,
+            byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
+            LogicalLogLocator logicalLogLocator) throws ACIDException {
+
+        HashMap<TransactionContext, Integer> map = null;
+        int activeTxnCount;
+
+        // logLocator is a re-usable object that is appropriately set in each
+        // invocation.
+        // If the reference is null, the log manager must throw an exception.
+        if (logicalLogLocator == null) {
             throw new ACIDException(
                     " you need to pass in a non-null logLocator, if you dont have it, then pass in a dummy so that the +"
                             + "log manager can set it approporiately for you");
         }
 
         // compute the total log size including the header and the checksum.
-        int totalLogSize = logManagerProperties.getLogHeaderSize() + requestedSpaceForLog
-                + logManagerProperties.getLogChecksumSize();
+        int totalLogSize = logRecordHelper.getLogRecordSize(logType, logContentSize);
 
         // check for the total space requirement to be less than a log page.
         if (totalLogSize > logManagerProperties.getLogPageSize()) {
             throw new ACIDException(
                     " Maximum Log Content Size is "
-                            + (logManagerProperties.getLogPageSize() - logManagerProperties.getLogHeaderSize() - logManagerProperties
+                            + (logManagerProperties.getLogPageSize() - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
                                     .getLogChecksumSize()));
         }
 
-        // all constraints checked and we are goot to go and acquire a lsn.
-        long previousLogLocator = -1;
-        long myLogLocator; // the will be set to the location (a long value)
-        // where the log record needs to be placed.
+        // all constraints checked and we are good to go and acquire a lsn.
+        long previousLSN = -1;
 
-        /*
-         * The logs written by a transaction need to be linked to each other for
-         * a successful rollback/recovery. However there could be multiple
-         * threads operating concurrently that are part of a common transaction.
-         * These threads need to synchronize and record the lsn corresponding to
-         * the last log record written by (any thread of) the transaction.
-         */
-        synchronized (context) {
-            previousLogLocator = context.getLastLogLocator().getLsn();
-            myLogLocator = getLsn(totalLogSize, logType);
-            context.getLastLogLocator().setLsn(myLogLocator);
-            logLocator.setLsn(myLogLocator);
+        // the will be set to the location (a long value) where the log record
+        // needs to be placed.
+        long currentLSN;
+
+        // The logs written by a transaction need to be linked to each other for
+        // a successful rollback/recovery. However there could be multiple
+        // threads operating concurrently that are part of a common transaction.
+        // These threads need to synchronize and record the lsn corresponding to
+        // the last log record written by (any thread of) the transaction.
+        synchronized (txnCtx) {
+            previousLSN = txnCtx.getLastLogLocator().getLsn();
+            currentLSN = getLsn(totalLogSize, logType);
+            txnCtx.setLastLSN(currentLSN);
+            if (IS_DEBUG_MODE) {
+                System.out.println("--------------> LSN(" + currentLSN + ") is allocated");
+            }
+            logicalLogLocator.setLsn(currentLSN);
         }
 
         /*
@@ -548,112 +435,85 @@
          * performed correctly that is ownership is released.
          */
 
-        boolean decremented = false; // indicates if the transaction thread
-        // has release ownership of the
+        // indicates if the transaction thread has release ownership of the
         // page.
-        boolean addedFlushRequest = false; // indicates if the transaction
-        // thread has submitted a flush
-        // request.
+        boolean decremented = false;
 
-        int pageIndex = (int) getLogPageIndex(myLogLocator);
+        int pageIndex = (int) getLogPageIndex(currentLSN);
 
-        /*
-         * the lsn has been obtained for the log record. need to set the
-         * LogLocator instance accordingly.
-         */
-
+        // the lsn has been obtained for the log record. need to set the
+        // LogLocator instance accordingly.
         try {
+            logicalLogLocator.setBuffer(logPages[pageIndex]);
+            int pageOffset = getLogPageOffset(currentLSN);
+            logicalLogLocator.setMemoryOffset(pageOffset);
 
-            logLocator.setBuffer(logPages[pageIndex]);
-            int pageOffset = getLogPageOffset(myLogLocator);
-            logLocator.setMemoryOffset(pageOffset);
-
-            /*
-             * write the log header.
-             */
-            logRecordHelper.writeLogHeader(context, logLocator, resourceMgrId, pageId, logType, logActionType,
-                    requestedSpaceForLog, previousLogLocator);
+            // write the log header.
+            logRecordHelper.writeLogHeader(logicalLogLocator, logType, txnCtx, datasetId, PKHashValue, previousLSN,
+                    resourceId, resourceMgrId, logContentSize);
 
             // increment the offset so that the transaction can fill up the
             // content in the correct region of the allocated space.
-            logLocator.increaseMemoryOffset(logManagerProperties.getLogHeaderSize());
+            logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
 
-            // a COMMIT log record does not have any content
-            // and hence the logger (responsible for putting the log content) is
-            // not invoked.
-            if (requestedSpaceForLog != 0) {
-                logger.preLog(context, loggerArguments);
+            // a COMMIT log record does not have any content and hence
+            // the logger (responsible for putting the log content) is not
+            // invoked.
+            if (logContentSize != 0) {
+                logger.preLog(txnCtx, reusableLogContentObject);
             }
 
-            if (requestedSpaceForLog != 0) {
+            if (logContentSize != 0) {
                 // call the logger implementation and ask to fill in the log
                 // record content at the allocated space.
-                logger.log(context, logLocator, requestedSpaceForLog, loggerArguments);
-                logger.postLog(context, loggerArguments);
+                logger.log(txnCtx, logicalLogLocator, logContentSize, reusableLogContentObject);
+                logger.postLog(txnCtx, reusableLogContentObject);
+                if (IS_DEBUG_MODE) {
+                    logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset()
+                            - logRecordHelper.getLogHeaderSize(logType));
+                    System.out.println(logRecordHelper.getLogRecordForDisplay(logicalLogLocator));
+                    logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
+                }
             }
 
-            /*
-             * The log record has been written. For integrity checks, compute
-             * the checksum and put it at the end of the log record.
-             */
-            int startPosChecksum = logLocator.getMemoryOffset() - logManagerProperties.getLogHeaderSize();
-            int length = totalLogSize - logManagerProperties.getLogChecksumSize();
+            // The log record has been written. For integrity checks, compute
+            // the checksum and put it at the end of the log record.
+            int startPosChecksum = logicalLogLocator.getMemoryOffset() - logRecordHelper.getLogHeaderSize(logType);
+            int length = totalLogSize - logRecordHelper.getLogChecksumSize();
             long checksum = DataUtil.getChecksum(logPages[pageIndex], startPosChecksum, length);
-            logPages[pageIndex].writeLong(pageOffset + logManagerProperties.getLogHeaderSize() + requestedSpaceForLog,
+            logPages[pageIndex].writeLong(pageOffset + logRecordHelper.getLogHeaderSize(logType) + logContentSize,
                     checksum);
 
-            /*
-             * release the ownership as the log record has been placed in
-             * created space.
-             */
-            int pageDirtyCount = logPageOwnerCount[pageIndex].decrementAndGet();
+            if (IS_DEBUG_MODE) {
+                System.out.println("--------------> LSN(" + currentLSN + ") is written");
+            }
+
+            // release the ownership as the log record has been placed in
+            // created space.
+            logPageOwnerCount[pageIndex].decrementAndGet();
 
             // indicating that the transaction thread has released ownership
             decremented = true;
 
-            /*
-             * If the transaction thread happens to be the last owner of the log
-             * page the page must by marked as a candidate to be flushed.
-             */
-            if (pageDirtyCount == 0) {
-                logPageStatus[pageIndex].set(PageState.INACTIVE);
-                addFlushRequest(pageIndex);
-                addedFlushRequest = true;
-            }
-
-            /*
-             * If the log type is commit, a flush request is registered, if the
-             * log record has not reached the disk. It may be possible that this
-             * thread does not get CPU cycles and in-between the log record has
-             * been flushed to disk because the containing log page filled up.
-             */
-            if (logType == LogType.COMMIT) {
-                if (getLastFlushedLsn().get() < myLogLocator) {
-                    if (!addedFlushRequest) {
-                        addFlushRequest(pageIndex);
-                    }
-
-                    /*
-                     * the commit log record is still in log buffer. need to
-                     * wait until the containing log page is flushed. When the
-                     * log flusher thread does flush the page, it notifies all
-                     * waiting threads of the flush event.
-                     */
-                    synchronized (logPages[pageIndex]) {
-                        while (getLastFlushedLsn().get() < myLogLocator) {
-                            logPages[pageIndex].wait();
-                        }
-                    }
+            if (logType == LogType.ENTITY_COMMIT) {
+                map = activeTxnCountMaps.get(pageIndex);
+                if (map.containsKey(txnCtx)) {
+                    activeTxnCount = (Integer) map.get(txnCtx);
+                    activeTxnCount++;
+                    map.put(txnCtx, activeTxnCount);
+                } else {
+                    map.put(txnCtx, 1);
                 }
+                addFlushRequest(pageIndex, currentLSN, false);
+            } else if (logType == LogType.COMMIT) {
+                addFlushRequest(pageIndex, currentLSN, true);
             }
+
         } catch (Exception e) {
             e.printStackTrace();
-            throw new ACIDException(context, "Thread: " + Thread.currentThread().getName()
+            throw new ACIDException(txnCtx, "Thread: " + Thread.currentThread().getName()
                     + " logger encountered exception", e);
         } finally {
-            /*
-             * If an exception was encountered and we did not release ownership
-             */
             if (!decremented) {
                 logPageOwnerCount[pageIndex].decrementAndGet();
             }
@@ -688,97 +548,165 @@
     /*
      * Read a log that is residing on the disk.
      */
-    private LogicalLogLocator readDiskLog(PhysicalLogLocator physicalLogLocator) throws ACIDException {
-        LogicalLogLocator logicalLogLocator;
-        String filePath = LogUtil.getLogFilePath(logManagerProperties,
-                LogUtil.getFileId(this, physicalLogLocator.getLsn()));
-        long fileOffset = LogUtil.getFileOffset(this, physicalLogLocator.getLsn());
+    private void readDiskLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
+        String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, lsnValue));
+        long fileOffset = LogUtil.getFileOffset(this, lsnValue);
+
         ByteBuffer buffer = ByteBuffer.allocate(logManagerProperties.getLogPageSize());
         RandomAccessFile raf = null;
+        FileChannel fileChannel = null;
         try {
             raf = new RandomAccessFile(filePath, "r");
-            raf.seek(fileOffset);
-            FileChannel fileChannel = raf.getChannel();
+            fileChannel = raf.getChannel();
+            fileChannel.position(fileOffset);
             fileChannel.read(buffer);
             buffer.position(0);
-            buffer.limit(buffer.getInt(4));
+
+            byte logType = buffer.get(4);
+            int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
+            int logBodySize = buffer.getInt(logHeaderSize - 4);
+            int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
+            buffer.limit(logRecordSize);
             MemBasedBuffer memBuffer = new MemBasedBuffer(buffer.slice());
-            logicalLogLocator = new LogicalLogLocator(physicalLogLocator.getLsn(), memBuffer, 0, this);
-            if (!logRecordHelper.validateLogRecord(logManagerProperties, logicalLogLocator)) {
-                throw new ACIDException(" invalid log record at lsn " + physicalLogLocator.getLsn());
+            if (logicalLogLocator == null) {
+                logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
+            } else {
+                logicalLogLocator.setLsn(lsnValue);
+                logicalLogLocator.setBuffer(memBuffer);
+                logicalLogLocator.setMemoryOffset(0);
+            }
+            if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
+                throw new ACIDException(" invalid log record at lsn " + lsnValue);
             }
         } catch (Exception fnfe) {
-            throw new ACIDException(" unable to retrieve log record with lsn " + physicalLogLocator.getLsn()
-                    + " from the file system", fnfe);
+            fnfe.printStackTrace();
+            throw new ACIDException(" unable to retrieve log record with lsn " + lsnValue + " from the file system",
+                    fnfe);
         } finally {
             try {
-                if (raf != null) {
+                if (fileChannel != null) {
+                    fileChannel.close();
+                } else if (raf != null) {
                     raf.close();
                 }
             } catch (IOException ioe) {
                 ioe.printStackTrace();
-                throw new ACIDException(" exception in closing " + raf, ioe);
+                throw new ACIDException(" exception in closing a file: " + filePath, ioe);
             }
         }
-        return logicalLogLocator;
     }
 
     @Override
-    public LogicalLogLocator readLog(PhysicalLogLocator physicalLogLocator) throws ACIDException {
+    public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
         byte[] logRecord = null;
-        long lsnValue = physicalLogLocator.getLsn();
-        if (lsnValue > lsn.get()) {
-            throw new ACIDException(" invalid lsn " + physicalLogLocator);
-        }
 
-        LogicalLogLocator logLocator = null;
+        if (lsnValue >= lsn.get()) {
+            throw new ACIDException(" invalid lsn " + lsnValue);
+        }
 
         /* check if the log record in the log buffer or has reached the disk. */
         if (lsnValue > getLastFlushedLsn().get()) {
             int pageIndex = getLogPageIndex(lsnValue);
             int pageOffset = getLogPageOffset(lsnValue);
+
+            // TODO
+            // minimize memory allocation overhead. current code allocates the
+            // log page size per reading a log record.
+
             byte[] pageContent = new byte[logManagerProperties.getLogPageSize()];
+
             // take a lock on the log page so that the page is not flushed to
             // disk interim
             synchronized (logPages[pageIndex]) {
-                if (lsnValue > getLastFlushedLsn().get()) { // need to check
-                    // again
-                    // (this
-                    // thread may have got
-                    // de-scheduled and must
-                    // refresh!)
+
+                // need to check again (this thread may have got de-scheduled
+                // and must refresh!)
+                if (lsnValue > getLastFlushedLsn().get()) {
 
                     // get the log record length
                     logPages[pageIndex].getBytes(pageContent, 0, pageContent.length);
-                    int logRecordLength = DataUtil.byteArrayToInt(pageContent, pageOffset + 4);
-                    logRecord = new byte[logRecordLength];
+                    byte logType = pageContent[pageOffset + 4];
+                    int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
+                    int logBodySize = DataUtil.byteArrayToInt(pageContent, pageOffset + logHeaderSize - 4);
+                    int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
+                    logRecord = new byte[logRecordSize];
 
-                    /*
-                     * copy the log record content
-                     */
-                    System.arraycopy(pageContent, pageOffset, logRecord, 0, logRecordLength);
+                    // copy the log record content
+                    System.arraycopy(pageContent, pageOffset, logRecord, 0, logRecordSize);
                     MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
-                    logLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
+                    if (logicalLogLocator == null) {
+                        logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
+                    } else {
+                        logicalLogLocator.setLsn(lsnValue);
+                        logicalLogLocator.setBuffer(memBuffer);
+                        logicalLogLocator.setMemoryOffset(0);
+                    }
                     try {
                         // validate the log record by comparing checksums
-                        if (!logRecordHelper.validateLogRecord(logManagerProperties, logLocator)) {
-                            throw new ACIDException(" invalid log record at lsn " + physicalLogLocator);
+                        if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
+                            throw new ACIDException(" invalid log record at lsn " + lsnValue);
                         }
                     } catch (Exception e) {
-                        throw new ACIDException("exception encoutered in validating log record at lsn "
-                                + physicalLogLocator, e);
+                        throw new ACIDException("exception encoutered in validating log record at lsn " + lsnValue, e);
                     }
-                    return logLocator;
+                    return;
                 }
             }
         }
 
-        /*
-         * the log record is residing on the disk, read it from there.
-         */
-        return readDiskLog(physicalLogLocator);
+        // the log record is residing on the disk, read it from there.
+        readDiskLog(lsnValue, logicalLogLocator);
     }
 
+    public void renewLogFiles() throws ACIDException {
+        List<String> logFileNames = LogUtil.getLogFiles(logManagerProperties);
+        for (String name : logFileNames) {
+            File file = new File(LogUtil.getLogFilePath(logManagerProperties, Long.parseLong(name)));
+            if (!file.delete()) {
+                throw new ACIDException("Failed to delete a file: " + name);
+            }
+        }
+        closeLogPages();
+        initLSN();
+        openLogPages();
+        logPageFlusher.renew();
+    }
+
+    private PhysicalLogLocator initLSN() throws ACIDException {
+        PhysicalLogLocator nextPhysicalLsn = LogUtil.initializeLogAnchor(this);
+        startingLSN = nextPhysicalLsn.getLsn();
+        lastFlushedLSN.set(startingLSN - 1);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(" Starting lsn is : " + startingLSN);
+        }
+        lsn.set(startingLSN);
+        return nextPhysicalLsn;
+    }
+
+    private void closeLogPages() throws ACIDException {
+        for (int i = 0; i < numLogPages; i++) {
+            try {
+                logPages[i].close();
+            } catch (IOException e) {
+                throw new ACIDException(e);
+            }
+        }
+    }
+
+    private void openLogPages() throws ACIDException {
+        try {
+            String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, startingLSN));
+            for (int i = 0; i < numLogPages; i++) {
+                logPages[i].open(filePath,
+                        LogUtil.getFileOffset(this, startingLSN) + i * logManagerProperties.getLogPageSize(),
+                        logManagerProperties.getLogPageSize());
+            }
+        } catch (Exception e) {
+            throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
+        }
+    }
+
+    @Override
     public ILogRecordHelper getLogRecordHelper() {
         return logRecordHelper;
     }
@@ -789,11 +717,12 @@
      * logic to event based when log manager support is integrated with the
      * Buffer Manager.
      */
+    @Override
     public synchronized void flushLog(LogicalLogLocator logicalLogLocator) throws ACIDException {
         if (logicalLogLocator.getLsn() > lsn.get()) {
             throw new ACIDException(" invalid lsn " + logicalLogLocator.getLsn());
         }
-        while (lastFlushedLsn.get() < logicalLogLocator.getLsn());
+        while (lastFlushedLSN.get() < logicalLogLocator.getLsn());
     }
 
     /*
@@ -828,6 +757,7 @@
         return pageNo == 0 ? numLogPages - 1 : pageNo - 1;
     }
 
+    @Override
     public LogManagerProperties getLogManagerProperties() {
         return logManagerProperties;
     }
@@ -840,30 +770,40 @@
         return logPageOwnerCount[pageIndex];
     }
 
-    public ICommitResolver getCommitResolver() {
-        return commitResolver;
-    }
-
-    public CommitRequestStatistics getCommitRequestStatistics() {
-        return commitRequestStatistics;
-    }
-
     public IFileBasedBuffer[] getLogPages() {
         return logPages;
     }
 
-    public int getLastFlushedPage() {
-        return lastFlushedPage.get();
-    }
-
-    public void setLastFlushedPage(int lastFlushedPage) {
-        this.lastFlushedPage.set(lastFlushedPage);
-    }
-
     @Override
-    public TransactionProvider getTransactionProvider() {
+    public TransactionSubsystem getTransactionSubsystem() {
         return provider;
     }
+
+    public void decrementActiveTxnCountOnIndexes(int pageIndex) throws HyracksDataException {
+        TransactionContext ctx = null;
+        int count = 0;
+        int i = 0;
+
+        HashMap<TransactionContext, Integer> map = activeTxnCountMaps.get(pageIndex);
+        Set<Map.Entry<TransactionContext, Integer>> entrySet = map.entrySet();
+        if (entrySet != null) {
+            for (Map.Entry<TransactionContext, Integer> entry : entrySet) {
+                if (entry != null) {
+                    if (entry.getValue() != null) {
+                        count = entry.getValue();
+                    }
+                    if (count > 0) {
+                        ctx = entry.getKey();
+                        for (i = 0; i < count; i++) {
+                            ctx.decreaseActiveTransactionCountOnIndexes();
+                        }
+                    }
+                }
+            }
+        }
+
+        map.clear();
+    }
 }
 
 /*
@@ -875,36 +815,99 @@
 class LogPageFlushThread extends Thread {
 
     private LogManager logManager;
+    /*
+     * pendingFlushRequests is a map with key as Integer denoting the page
+     * index. When a (transaction) thread discovers the need to flush a page, it
+     * puts its Thread object into the corresponding value that is a
+     * LinkedBlockingQueue. The LogManager has a LogFlusher thread that scans
+     * this map in order of page index (and circling around). The flusher thread
+     * needs to flush pages in order and waits for a thread to deposit an object
+     * in the blocking queue corresponding to the next page in order. A request
+     * to flush a page is conveyed to the flush thread by simply depositing an
+     * object in to corresponding blocking queue. It is blocking in the sense
+     * that the flusher thread will continue to wait for an object to arrive in
+     * the queue. The object itself is ignored by the fliusher and just acts as
+     * a signal/event that a page needs to be flushed.
+     */
+    private final LinkedBlockingQueue<Object>[] flushRequestQueue;
+    private final Object[] flushRequests;
+    private int pageToFlush;
+    private final long groupCommitWaitPeriod;
+    private boolean isRenewRequest;
 
     public LogPageFlushThread(LogManager logManager) {
         this.logManager = logManager;
         setName("Flusher");
+        int numLogPages = logManager.getLogManagerProperties().getNumLogPages();
+        this.flushRequestQueue = new LinkedBlockingQueue[numLogPages];
+        this.flushRequests = new Object[numLogPages];
+        for (int i = 0; i < numLogPages; i++) {
+            flushRequestQueue[i] = new LinkedBlockingQueue<Object>(1);
+            flushRequests[i] = new Object();
+        }
+        this.pageToFlush = -1;
+        groupCommitWaitPeriod = logManager.getLogManagerProperties().getGroupCommitWaitPeriod();
+        isRenewRequest = false;
+    }
+
+    public void renew() {
+        isRenewRequest = true;
+        pageToFlush = -1;
+        this.interrupt();
+        isRenewRequest = false;
+    }
+
+    public void requestFlush(int pageIndex, long lsn, boolean isSynchronous) {
+        synchronized (logManager.getLogPage(pageIndex)) {
+            // return if flushedLSN >= lsn
+            if (logManager.getLastFlushedLsn().get() >= lsn) {
+                return;
+            }
+
+            // put a new request to the queue only if the request on the page is
+            // not in the queue.
+            flushRequestQueue[pageIndex].offer(flushRequests[pageIndex]);
+
+            // return if the request is asynchronous
+            if (!isSynchronous) {
+                return;
+            }
+
+            // wait until there is flush.
+            boolean isNotified = false;
+            while (!isNotified) {
+                try {
+                    logManager.getLogPage(pageIndex).wait();
+                    isNotified = true;
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+            }
+        }
     }
 
     @Override
     public void run() {
         while (true) {
             try {
-                int pageToFlush = logManager.getNextPageInSequence(logManager.getLastFlushedPage());
+                pageToFlush = logManager.getNextPageInSequence(pageToFlush);
 
-                /*
-                 * A wait call on the linkedBLockingQueue. The flusher thread is
-                 * notified when an object is added to the queue. Please note
-                 * that each page has an associated blocking queue.
-                 */
-                logManager.getPendingFlushRequests(pageToFlush).take();
+                // A wait call on the linkedBLockingQueue. The flusher thread is
+                // notified when an object is added to the queue. Please note
+                // that each page has an associated blocking queue.
+                try {
+                    flushRequestQueue[pageToFlush].take();
+                } catch (InterruptedException ie) {
+                    while (isRenewRequest) {
+                        sleep(1);
+                    }
+                    continue;
+                }
 
-                /*
-                 * The LogFlusher was waiting for a page to be marked as a
-                 * candidate for flushing. Now that has happened. The thread
-                 * shall proceed to take a lock on the log page
-                 */
-                synchronized (logManager.getLogPages()[pageToFlush]) {
+                synchronized (logManager.getLogPage(pageToFlush)) {
 
-                    /*
-                     * lock the internal state of the log manager and create a
-                     * log file if necessary.
-                     */
+                    // lock the internal state of the log manager and create a
+                    // log file if necessary.
                     int prevLogFileId = logManager.getLogFileId(logManager.getLastFlushedLsn().get());
                     int nextLogFileId = logManager.getLogFileId(logManager.getLastFlushedLsn().get()
                             + logManager.getLogManagerProperties().getLogPageSize());
@@ -916,197 +919,63 @@
                                 logManager.getLogManagerProperties().getLogPageSize());
                     }
 
-                    logManager.getLogPage(pageToFlush).flush(); // put the
-                    // content to
-                    // disk, the
-                    // thread still
-                    // has a lock on
-                    // the log page
+                    // #. sleep during the groupCommitWaitTime
+                    sleep(groupCommitWaitPeriod);
 
-                    /*
-                     * acquire lock on the log manager as we need to update the
-                     * internal bookkeeping data.
-                     */
+                    // #. set the logPageStatus to INACTIVE in order to prevent
+                    // other txns from writing on this page.
+                    logManager.getLogPageStatus(pageToFlush).set(PageState.INACTIVE);
 
-                    // increment the last flushed lsn.
-                    long lastFlushedLsn = logManager.incrementLastFlushedLsn(logManager.getLogManagerProperties()
-                            .getLogPageSize());
-
-                    /*
-                     * the log manager gains back ownership of the page. this is
-                     * reflected by incrementing the owner count of the page.
-                     * recall that when the page is begin flushed the owner
-                     * count is actually 0 Value of zero implicitly indicates
-                     * that the page is operated upon by the log flusher thread.
-                     */
-                    logManager.getLogPageOwnershipCount(pageToFlush).incrementAndGet();
-
-                    /*
-                     * get the number of log buffers that have been written so
-                     * far. A log buffer = number of log pages * size of a log
-                     * page
-                     */
-                    int numCycles = (int) lastFlushedLsn / logManager.getLogManagerProperties().getLogBufferSize();
-                    if (lastFlushedLsn % logManager.getLogManagerProperties().getLogBufferSize() == 0) {
-                        numCycles--;
+                    // #. need to wait until the logPageOwnerCount reaches 1
+                    // (LOG_WRITER)
+                    // meaning every one has finished writing logs on this page.
+                    while (logManager.getLogPageOwnershipCount(pageToFlush).get() != PageOwnershipStatus.LOG_WRITER) {
+                        sleep(0);
                     }
 
-                    /*
-                     * Map the log page to a new region in the log file.
-                     */
+                    // #. set the logPageOwnerCount to 0 (LOG_FLUSHER)
+                    // meaning it is flushing.
+                    logManager.getLogPageOwnershipCount(pageToFlush).set(PageOwnershipStatus.LOG_FLUSHER);
 
+                    // put the content to disk (the thread still has a lock on
+                    // the log page)
+                    logManager.getLogPage(pageToFlush).flush();
+
+                    // increment the last flushed lsn and lastFlushedPage
+                    logManager.incrementLastFlushedLsn(logManager.getLogManagerProperties().getLogPageSize());
+
+                    // decrement activeTxnCountOnIndexes
+                    logManager.decrementActiveTxnCountOnIndexes(pageToFlush);
+
+                    // reset the count to 1
+                    logManager.getLogPageOwnershipCount(pageToFlush).set(PageOwnershipStatus.LOG_WRITER);
+
+                    // Map the log page to a new region in the log file.
                     long nextWritePosition = logManager.getLogPages()[pageToFlush].getNextWritePosition()
                             + logManager.getLogManagerProperties().getLogBufferSize();
 
-                    /*
-                     * long nextPos = (numCycles + 1)
-                     * logManager.getLogManagerProperties() .getLogBufferSize()
-                     * + pageToFlush logManager.getLogManagerProperties()
-                     * .getLogPageSize();
-                     */
                     logManager.resetLogPage(nextWritePosition, pageToFlush);
 
                     // mark the page as ACTIVE
                     logManager.getLogPageStatus(pageToFlush).set(LogManager.PageState.ACTIVE);
 
+                    // #. checks the queue whether there is another flush
+                    // request on the same log buffer
+                    // If there is another request, then simply remove it.
+                    if (flushRequestQueue[pageToFlush].peek() != null) {
+                        flushRequestQueue[pageToFlush].take();
+                    }
+
                     // notify all waiting (transaction) threads.
-                    // Transaction thread may be waiting for the page to be
-                    // available or may have a commit log record on the page
-                    // that got flushed.
-                    logManager.getLogPages()[pageToFlush].notifyAll();
-                    logManager.setLastFlushedPage(pageToFlush);
+                    logManager.getLogPage(pageToFlush).notifyAll();
                 }
             } catch (IOException ioe) {
                 ioe.printStackTrace();
                 throw new Error(" exception in flushing log page", ioe);
             } catch (InterruptedException e) {
                 e.printStackTrace();
-                break; // must break from the loop as the exception indicates
-                // some thing horrendous has happened elsewhere
+                break;
             }
         }
     }
-}
-
-/*
- * TODO: By default the commit policy is to commit at each request and not have
- * a group commit. The following code needs to change to support group commit.
- * The code for group commit has not been tested thoroughly and is under
- * development.
- */
-class BasicCommitResolver implements ICommitResolver {
-
-    public boolean shouldCommitPage(int pageIndex, LogManager logManager,
-            CommitRequestStatistics commitRequestStatistics) {
-        return true;
-    }
-
-    public void init(LogManager logManager) {
-    }
-}
-
-class GroupCommitResolver implements ICommitResolver {
-
-    public boolean shouldCommitPage(int pageIndex, LogManager logManager,
-            CommitRequestStatistics commitRequestStatistics) {
-        long maxCommitWait = logManager.getLogManagerProperties().getGroupCommitWaitPeriod();
-        long timestamp = commitRequestStatistics.getPageLevelLastCommitRequestTimestamp(pageIndex);
-        if (timestamp == -1) {
-            if (maxCommitWait == 0) {
-                return true;
-            } else {
-                timestamp = System.currentTimeMillis();
-            }
-        }
-        long currenTime = System.currentTimeMillis();
-        if (currenTime - timestamp > maxCommitWait) {
-            return true;
-        }
-        return false;
-    }
-
-    public void init(LogManager logManager) {
-        GroupCommitHandlerThread groupCommitHandler = new GroupCommitHandlerThread(logManager);
-        groupCommitHandler.setDaemon(true);
-        groupCommitHandler.start();
-    }
-
-    class GroupCommitHandlerThread extends Thread {
-
-        private LogManager logManager;
-
-        public GroupCommitHandlerThread(LogManager logManager) {
-            this.logManager = logManager;
-            setName("Group Commit Handler");
-        }
-
-        @Override
-        public void run() {
-            int pageIndex = -1;
-            while (true) {
-                pageIndex = logManager.getNextPageInSequence(pageIndex);
-                long lastCommitRequeestTimestamp = logManager.getCommitRequestStatistics()
-                        .getPageLevelLastCommitRequestTimestamp(pageIndex);
-                if (lastCommitRequeestTimestamp != -1
-                        && System.currentTimeMillis() - lastCommitRequeestTimestamp > logManager
-                                .getLogManagerProperties().getGroupCommitWaitPeriod()) {
-                    int dirtyCount = logManager.getLogPageOwnershipCount(pageIndex).decrementAndGet();
-                    if (dirtyCount == 0) {
-                        try {
-                            logManager.getLogPageStatus(pageIndex).set(LogManager.PageState.INACTIVE);
-                            logManager.getPendingFlushRequests(pageIndex).put(Thread.currentThread());
-                        } catch (InterruptedException e) {
-                            e.printStackTrace();
-                            break;
-                        }
-                        logManager.getCommitRequestStatistics().committedPage(pageIndex);
-                    }
-                }
-            }
-        }
-    }
-
-}
-
-interface ICommitResolver {
-    public boolean shouldCommitPage(int pageIndex, LogManager logManager,
-            CommitRequestStatistics commitRequestStatistics);
-
-    public void init(LogManager logManager);
-}
-
-/**
- * Represents a collection of all commit requests by transactions for each log
- * page. The requests are accumulated until the commit policy triggers a flush
- * of the corresponding log page. Upon a flush of a page, all commit requests
- * for the page are cleared.
- */
-class CommitRequestStatistics {
-
-    AtomicInteger[] pageLevelCommitRequestCount;
-    AtomicLong[] pageLevelLastCommitRequestTimestamp;
-
-    public CommitRequestStatistics(int numPages) {
-        pageLevelCommitRequestCount = new AtomicInteger[numPages];
-        pageLevelLastCommitRequestTimestamp = new AtomicLong[numPages];
-        for (int i = 0; i < numPages; i++) {
-            pageLevelCommitRequestCount[i] = new AtomicInteger(0);
-            pageLevelLastCommitRequestTimestamp[i] = new AtomicLong(-1L);
-        }
-    }
-
-    public void registerCommitRequest(int pageIndex) {
-        pageLevelCommitRequestCount[pageIndex].incrementAndGet();
-        pageLevelLastCommitRequestTimestamp[pageIndex].set(System.currentTimeMillis());
-    }
-
-    public long getPageLevelLastCommitRequestTimestamp(int pageIndex) {
-        return pageLevelLastCommitRequestTimestamp[pageIndex].get();
-    }
-
-    public void committedPage(int pageIndex) {
-        pageLevelCommitRequestCount[pageIndex].set(0);
-        pageLevelLastCommitRequestTimestamp[pageIndex].set(-1L);
-    }
-
-}
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
index 14b45b6..0211f69 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManagerProperties.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2009-2010 by The Regents of the University of California
+ * 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
@@ -17,111 +17,87 @@
 import java.io.Serializable;
 import java.util.Properties;
 
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
-
 public class LogManagerProperties implements Serializable {
 
-    /**
-     * generated SerialVersionUID
-     */
     private static final long serialVersionUID = 2084227360840799662L;
 
-    private String logFilePrefix = "asterix_transaction_log"; // log files
-
-    // follow the
-    // naming
-    // convention
-    // <logFilePrefix>_<number>
-    // where number
-    // starts from 0
-
-    private String logDir = "asterix_logs"; // the path where the LogManager
-                                            // will create log files
-
-    private int logPageSize = 128 * 1024; // 128 KB
-    private int numLogPages = 8; // number of log pages in the log buffer.
-    private long logPartitionSize = logPageSize * 250; // maximum size of each
-                                                       // log file
-    private long groupCommitWaitPeriod = 0; // time in milliseconds for which a
-    // commit record will wait before
-    // the housing page is marked for
-    // flushing.
-    private int logBufferSize = logPageSize * numLogPages;
-
-    private final int logHeaderSize = 43; /*
-                                           * ( magic number(4) + (length(4) +
-                                           * type(1) + actionType(1) +
-                                           * timestamp(8) + transacitonId(8) +
-                                           * resourceMgrId(1) + pageId(8) +
-                                           * prevLSN(8)
-                                           */
-    private int logTailSize = 8; /* checksum(8) */
-    public int logMagicNumber = 123456789;
-
-    public static final String LOG_PARTITION_SIZE_KEY = "log_partition_size";
-    public static final String LOG_DIR_KEY = "log_dir";
+    public static final int LOG_MAGIC_NUMBER = 123456789;
+    public static final String LOG_DIR_SUFFIX_KEY = ".txnLogDir";
     public static final String LOG_PAGE_SIZE_KEY = "log_page_size";
+    public static final String LOG_PARTITION_SIZE_KEY = "log_partition_size";
     public static final String NUM_LOG_PAGES_KEY = "num_log_pages";
     public static final String LOG_FILE_PREFIX_KEY = "log_file_prefix";
-    public static final String GROUP_COMMIT_WAIT_PERIOD = "group_commit_wait_period";
+    public static final String GROUP_COMMIT_WAIT_PERIOD_KEY = "group_commit_wait_period";
 
-    public LogManagerProperties(Properties properties) {
-        if (properties.get(LOG_PAGE_SIZE_KEY) != null) {
-            logPageSize = Integer.parseInt(properties.getProperty(LOG_PAGE_SIZE_KEY));
-        }
-        if (properties.get(NUM_LOG_PAGES_KEY) != null) {
-            numLogPages = Integer.parseInt(properties.getProperty(NUM_LOG_PAGES_KEY));
-        }
+    private static final int DEFAULT_LOG_PAGE_SIZE = 128 * 1024; //128KB
+    private static final int DEFAULT_NUM_LOG_PAGES = 8;
+    private static final long DEFAULT_LOG_PARTITION_SIZE = (long) 1024 * 1024 * 1024 * 2; //2GB 
+    private static final long DEFAULT_GROUP_COMMIT_WAIT_PERIOD = 1; // time in millisec.
+    private static final String DEFAULT_LOG_FILE_PREFIX = "asterix_transaction_log";
+    private static final String DEFAULT_LOG_DIRECTORY = "asterix_logs/";
 
-        if (properties.get(LOG_PARTITION_SIZE_KEY) != null) {
-            logPartitionSize = Long.parseLong(properties.getProperty(LOG_PARTITION_SIZE_KEY));
-        }
+    // follow the naming convention <logFilePrefix>_<number> where number starts from 0
+    private final String logFilePrefix;
+    private final String logDir;
+    public String logDirKey;
 
-        groupCommitWaitPeriod = Long.parseLong(properties.getProperty("group_commit_wait_period", ""
-                + groupCommitWaitPeriod));
-        logFilePrefix = properties.getProperty(LOG_FILE_PREFIX_KEY, logFilePrefix);
-        logDir = properties
-                .getProperty(LOG_DIR_KEY, TransactionManagementConstants.LogManagerConstants.DEFAULT_LOG_DIR);
+    // number of log pages in the log buffer
+    private final int logPageSize;
+    // number of log pages in the log buffer.
+    private final int numLogPages;
+    // time in milliseconds
+    private final long groupCommitWaitPeriod;
+    // logBufferSize = logPageSize * numLogPages;
+    private final int logBufferSize;
+    // maximum size of each log file
+    private final long logPartitionSize;
+
+    public LogManagerProperties(Properties properties, String nodeId) {
+        this.logDirKey = new String(nodeId + LOG_DIR_SUFFIX_KEY);
+        this.logPageSize = Integer.parseInt(properties.getProperty(LOG_PAGE_SIZE_KEY, "" + DEFAULT_LOG_PAGE_SIZE));
+        this.numLogPages = Integer.parseInt(properties.getProperty(NUM_LOG_PAGES_KEY, "" + DEFAULT_NUM_LOG_PAGES));
+        long logPartitionSize = Long.parseLong(properties.getProperty(LOG_PARTITION_SIZE_KEY, ""
+                + DEFAULT_LOG_PARTITION_SIZE));
+        this.logDir = properties.getProperty(logDirKey, DEFAULT_LOG_DIRECTORY + nodeId);
+        this.logFilePrefix = properties.getProperty(LOG_FILE_PREFIX_KEY, DEFAULT_LOG_FILE_PREFIX);
+        this.groupCommitWaitPeriod = Long.parseLong(properties.getProperty(GROUP_COMMIT_WAIT_PERIOD_KEY, ""
+                + DEFAULT_GROUP_COMMIT_WAIT_PERIOD));
+
+        this.logBufferSize = logPageSize * numLogPages;
+        //make sure that the log partition size is the multiple of log buffer size.
+        this.logPartitionSize = (logPartitionSize / logBufferSize) * logBufferSize;
     }
 
     public long getLogPartitionSize() {
         return logPartitionSize;
     }
 
-    public void setLogPartitionSize(long logPartitionSize) {
-        this.logPartitionSize = logPartitionSize;
-    }
-
     public String getLogFilePrefix() {
         return logFilePrefix;
     }
 
-    public void setLogDir(String logDir) {
-        this.logDir = logDir;
-    }
-
     public String getLogDir() {
         return logDir;
     }
 
-    public int getLogHeaderSize() {
-        return logHeaderSize;
-    }
-
-    public int getLogChecksumSize() {
-        return logTailSize;
-    }
-
-    public int getTotalLogRecordLength(int logContentSize) {
-        return logContentSize + logHeaderSize + logTailSize;
-    }
-
     public int getLogPageSize() {
         return logPageSize;
     }
 
-    public void setLogPageSize(int logPageSize) {
-        this.logPageSize = logPageSize;
+    public int getNumLogPages() {
+        return numLogPages;
+    }
+
+    public int getLogBufferSize() {
+        return logBufferSize;
+    }
+
+    public long getGroupCommitWaitPeriod() {
+        return groupCommitWaitPeriod;
+    }
+
+    public String getLogDirKey() {
+        return logDirKey;
     }
 
     public String toString() {
@@ -134,29 +110,4 @@
         builder.append("group_commit_wait_period : " + groupCommitWaitPeriod + FileUtil.lineSeparator);
         return builder.toString();
     }
-
-    public void setNumLogPages(int numLogPages) {
-        this.numLogPages = numLogPages;
-    }
-
-    public int getNumLogPages() {
-        return numLogPages;
-    }
-
-    public void setLogBufferSize(int logBufferSize) {
-        this.logBufferSize = logBufferSize;
-    }
-
-    public int getLogBufferSize() {
-        return logBufferSize;
-    }
-
-    public long getGroupCommitWaitPeriod() {
-        return groupCommitWaitPeriod;
-    }
-
-    public void setGroupCommitWaitPeriod(long groupCommitWaitPeriod) {
-        this.groupCommitWaitPeriod = groupCommitWaitPeriod;
-    }
-
 }
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..6b882ef 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
@@ -21,18 +21,45 @@
  * for writing/reading of log header and checksum as well as validating log
  * record by checksum comparison. Every ILogManager implementation has an
  * associated ILogRecordHelper implementation.
+ * == LogRecordFormat ==
+ * [Header]
+ * --------------------------- Header part1(17) : Both COMMIT and UPDATE log type have part1 fields
+ * LogMagicNumber(4)
+ * LogType(1)
+ * JobId(4)
+ * DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * PKHashValue(4)
+ * --------------------------- Header part2(21) : Only UPDATE log type has part2 fields
+ * PrevLSN(8) //only for UPDATE
+ * ResourceId(8) //stored in .metadata of the corresponding index in NC node
+ * ResourceMgrId(1)
+ * LogRecordSize(4)
+ * --------------------------- COMMIT doesn't have Body fields.
+ * [Body] The Body size is given through the parameter reusableLogContentObjectLength
+ * TupleFieldCount(4)
+ * NewOp(1)
+ * NewValueLength(4)
+ * NewValue(NewValueLength)
+ * OldOp(1)
+ * OldValueLength(4)
+ * OldValue(OldValueLength)
+ * --------------------------- Both COMMIT and UPDATE have tail fields.
+ * [Tail]
+ * Checksum(8)
  */
 public class LogRecordHelper implements ILogRecordHelper {
 
-    private final int BEGIN_MAGIC_NO_POS = 0;
-    private final int BEGING_LENGTH_POS = 4;
-    private final int BEGIN_TYPE_POS = 8;
-    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 LOG_CHECKSUM_SIZE = 8;
+
+    private final int MAGIC_NO_POS = 0;
+    private final int LOG_TYPE_POS = 4;
+    private final int JOB_ID_POS = 5;
+    private final int DATASET_ID_POS = 9;
+    private final int PK_HASH_VALUE_POS = 13;
+    private final int PREV_LSN_POS = 17;
+    private final int RESOURCE_ID_POS = 25;
+    private final int RESOURCE_MGR_ID_POS = 33;
+    private final int LOG_RECORD_SIZE_POS = 34;
 
     private ILogManager logManager;
 
@@ -40,59 +67,38 @@
         this.logManager = logManager;
     }
 
+    @Override
     public byte getLogType(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + BEGIN_TYPE_POS);
+        return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS);
     }
 
-    public byte getLogActionType(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + BEGIN_ACTION_TYPE_POS);
+    @Override
+    public int getJobId(LogicalLogLocator logicalLogLocator) {
+        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS);
     }
 
-    public int getLogLength(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).readInt(logicalLogLocator.getMemoryOffset() + BEGING_LENGTH_POS);
+    @Override
+    public int getDatasetId(LogicalLogLocator logicalLogLocator) {
+        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS);
     }
 
-    public long getLogTimestamp(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_TIMESTAMP_POS);
+    @Override
+    public int getPKHashValue(LogicalLogLocator logicalLogLocator) {
+        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS);
     }
 
-    public long getLogChecksum(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
-                + getLogLength(logicalLogLocator) - 8);
-    }
-
-    public long getLogTransactionId(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS);
-    }
-
-    public byte getResourceMgrId(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).getByte(logicalLogLocator.getMemoryOffset() + BEGIN_RESOURCE_MGR_ID_POS);
-    }
-
-    public long getPageId(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset() + BEGIN_PAGE_ID_POS);
-    }
-
-    public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getMemoryOffset() + logManager.getLogManagerProperties().getLogHeaderSize();
-    }
-
-    public int getLogContentEndPos(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getMemoryOffset() + getLogLength(logicalLogLocator)
-                - logManager.getLogManagerProperties().getLogChecksumSize();
-    }
-
-    public PhysicalLogLocator getPreviousLsnByTransaction(LogicalLogLocator logicalLogLocator) {
-        long prevLsnValue = (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
-                + BEGIN_PREV_LSN_POS);
+    @Override
+    public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator) {
+        long prevLsnValue = (logicalLogLocator.getBuffer())
+                .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
         PhysicalLogLocator previousLogLocator = new PhysicalLogLocator(prevLsnValue, logManager);
         return previousLogLocator;
     }
 
-    public boolean getPreviousLsnByTransaction(PhysicalLogLocator physicalLogLocator,
-            LogicalLogLocator logicalLogLocator) {
-        long prevLsnValue = (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
-                + BEGIN_PREV_LSN_POS);
+    @Override
+    public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator) {
+        long prevLsnValue = (logicalLogLocator.getBuffer())
+                .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
         if (prevLsnValue == -1) {
             return false;
         }
@@ -100,6 +106,41 @@
         return true;
     }
 
+    @Override
+    public long getResourceId(LogicalLogLocator logicalLogLocator) {
+        return logicalLogLocator.getBuffer().readLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS);
+    }
+
+    @Override
+    public byte getResourceMgrId(LogicalLogLocator logicalLogLocater) {
+        return logicalLogLocater.getBuffer().getByte(logicalLogLocater.getMemoryOffset() + RESOURCE_MGR_ID_POS);
+    }
+
+    @Override
+    public int getLogContentSize(LogicalLogLocator logicalLogLocater) {
+        return logicalLogLocater.getBuffer().readInt(logicalLogLocater.getMemoryOffset() + LOG_RECORD_SIZE_POS);
+    }
+
+    @Override
+    public long getLogChecksum(LogicalLogLocator logicalLogLocator) {
+        return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
+                + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
+                - LOG_CHECKSUM_SIZE);
+    }
+
+    @Override
+    public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator) {
+        return logicalLogLocator.getMemoryOffset() + getLogHeaderSize(getLogType(logicalLogLocator));
+    }
+
+    @Override
+    public int getLogContentEndPos(LogicalLogLocator logicalLogLocator) {
+        return logicalLogLocator.getMemoryOffset()
+                + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
+                - LOG_CHECKSUM_SIZE;
+    }
+
+    @Override
     public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator) {
         StringBuilder builder = new StringBuilder();
         byte logType = new Byte(getLogType(logicalLogLocator));
@@ -111,67 +152,101 @@
             case LogType.UPDATE:
                 logTypeDisplay = "UPDATE";
                 break;
+            case LogType.ENTITY_COMMIT:
+                logTypeDisplay = "ENTITY_COMMIT";
+                break;
         }
-        builder.append(" Log Type :" + logTypeDisplay);
-        builder.append(" Log Length :" + getLogLength(logicalLogLocator));
-        builder.append(" Log Timestamp:" + getLogTimestamp(logicalLogLocator));
-        builder.append(" Log Transaction Id:" + getLogTransactionId(logicalLogLocator));
-        builder.append(" Log Resource Mgr Id:" + getResourceMgrId(logicalLogLocator));
-        builder.append(" Page Id:" + getPageId(logicalLogLocator));
-        builder.append(" Log Checksum:" + getLogChecksum(logicalLogLocator));
-        builder.append(" Log Previous lsn: " + getPreviousLsnByTransaction(logicalLogLocator));
-        return new String(builder);
-    }
-
-    public void writeLogHeader(TransactionContext context, LogicalLogLocator logicalLogLocator, byte resourceMgrId,
-            long pageId, byte logType, byte logActionType, int logContentSize, long prevLogicalLogLocator) {
-        /* magic no */
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + BEGIN_MAGIC_NO_POS,
-                logManager.getLogManagerProperties().logMagicNumber);
-
-        /* length */
-        int length = logManager.getLogManagerProperties().getTotalLogRecordLength(logContentSize);
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + BEGING_LENGTH_POS, length);
-
-        /* log type */
-        (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_TYPE_POS, logType);
-
-        /* log action type */
-        (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_ACTION_TYPE_POS, logActionType);
-
-        /* timestamp */
-        long timestamp = System.currentTimeMillis();
-        (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TIMESTAMP_POS, timestamp);
-
-        /* transaction id */
-        (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_TRANSACTION_ID_POS,
-                context.getTransactionID());
-
-        /* resource Mgr id */
-        (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + BEGIN_RESOURCE_MGR_ID_POS,
-                resourceMgrId);
-
-        /* page id */
-        (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_PAGE_ID_POS, pageId);
-
-        /* previous LSN's File Id by the transaction */
-        (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + BEGIN_PREV_LSN_POS,
-                prevLogicalLogLocator);
-    }
-
-    public void writeLogTail(LogicalLogLocator logicalLogLocator, ILogManager logManager) {
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset(),
-                logManager.getLogManagerProperties().logMagicNumber);
+        builder.append(" LSN : ").append(logicalLogLocator.getLsn());
+        builder.append(" Log Type : ").append(logTypeDisplay);
+        builder.append(" Job Id : ").append(getJobId(logicalLogLocator));
+        builder.append(" Dataset Id : ").append(getDatasetId(logicalLogLocator));
+        builder.append(" PK Hash Value : ").append(getPKHashValue(logicalLogLocator));
+        if (logType == LogType.UPDATE) {
+            builder.append(" PrevLSN : ").append(getPrevLSN(logicalLogLocator).getLsn());
+            builder.append(" Resource Id : ").append(getResourceId(logicalLogLocator));
+            builder.append(" ResourceMgr Id : ").append(getResourceMgrId(logicalLogLocator));
+            builder.append(" Log Record Size : ").append(
+                    getLogRecordSize(logType, getLogContentSize(logicalLogLocator)));
+        }
+        return builder.toString();
     }
 
     @Override
-    public boolean validateLogRecord(LogManagerProperties logManagerProperties, LogicalLogLocator logicalLogLocator) {
-        int logLength = this.getLogLength(logicalLogLocator);
+    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, TransactionContext context,
+            int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
+            int logRecordSize) {
+
+        /* magic no */
+        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + MAGIC_NO_POS,
+                logManager.getLogManagerProperties().LOG_MAGIC_NUMBER);
+
+        /* log type */
+        (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS, logType);
+
+        /* jobId */
+        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS, context.getJobId()
+                .getId());
+
+        /* datasetId */
+        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS, datasetId);
+
+        /* PK hash value */
+        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS, PKHashValue);
+
+        if (logType == LogType.UPDATE) {
+            /* prevLSN */
+            (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS,
+                    prevLogicalLogLocator);
+
+            /* resourceId */
+            (logicalLogLocator.getBuffer())
+                    .writeLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS, resourceId);
+
+            /* resourceMgr id */
+            (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + RESOURCE_MGR_ID_POS,
+                    resourceMgrId);
+
+            /* log record size */
+            (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + LOG_RECORD_SIZE_POS,
+                    logRecordSize);
+
+        }
+    }
+
+    @Override
+    public boolean validateLogRecord(LogicalLogLocator logicalLogLocator) {
+        int logLength = this.getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator));
         long expectedChecksum = DataUtil.getChecksum(logicalLogLocator.getBuffer(),
-                logicalLogLocator.getMemoryOffset(), logLength - logManagerProperties.getLogChecksumSize());
-        long actualChecksum = logicalLogLocator.getBuffer().readLong(
-                logicalLogLocator.getMemoryOffset() + logLength - logManagerProperties.getLogChecksumSize());
+                logicalLogLocator.getMemoryOffset(), logLength - LOG_CHECKSUM_SIZE);
+        long actualChecksum = getLogChecksum(logicalLogLocator);
         return expectedChecksum == actualChecksum;
     }
 
+    /**
+     * @param logType
+     * @param logBodySize
+     * @return
+     */
+    @Override
+    public int getLogRecordSize(byte logType, int logBodySize) {
+        if (logType == LogType.UPDATE) {
+            return 46 + logBodySize;
+        } else {
+            return 25;
+        }
+    }
+
+    @Override
+    public int getLogHeaderSize(byte logType) {
+        if (logType == LogType.UPDATE) {
+            return 38;
+        } else {
+            return 17;
+        }
+    }
+
+    @Override
+    public int getLogChecksumSize() {
+        return LOG_CHECKSUM_SIZE;
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
index 72b393d..96c6dba 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
@@ -18,8 +18,6 @@
 
     public static final byte UPDATE = 0;
     public static final byte COMMIT = 1;
-    public static final byte CLR = 2;
-    public static final byte BGN_CHPKT = 3;
-    public static final byte END_CHPKT = 4;
+    public static final byte ENTITY_COMMIT = 2;
 
 }
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
deleted file mode 100644
index 3b923dd..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLogger.java
+++ /dev/null
@@ -1,200 +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.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleWriter;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOp;
-
-/**
- * Represents a utility class for generating log records corresponding to
- * operations on a ITreeIndex implementation. A TreeLogger instance is thread
- * safe and can be shared across multiple threads that may belong to same or
- * different transactions.
- */
-class TransactionState {
-
-    private final Map<Long, TxnThreadState> transactionThreads = new HashMap<Long, TxnThreadState>();
-
-    public synchronized TxnThreadState getTransactionThreadState(long threadId) {
-        return transactionThreads.get(threadId);
-    }
-
-    public synchronized void putTransactionThreadState(long threadId, TxnThreadState txnThreadState) {
-        this.transactionThreads.put(threadId, txnThreadState);
-    }
-
-    public synchronized void remove(long threadId) {
-        transactionThreads.remove(threadId);
-    }
-}
-
-/**
- * Represents the state of a transaction thread. The state contains information
- * that includes the tuple being operated, the operation and the location of the
- * log record corresponding to the operation.
- */
-class TxnThreadState {
-
-    private ITupleReference tuple;
-    private IndexOp indexOperation;
-    private LogicalLogLocator logicalLogLocator;
-
-    public TxnThreadState(LogicalLogLocator logicalLogLocator, IndexOp indexOperation, ITupleReference tupleReference) {
-        this.tuple = tupleReference;
-        this.indexOperation = indexOperation;
-        this.logicalLogLocator = logicalLogLocator;
-    }
-
-    public synchronized ITupleReference getTuple() {
-        return tuple;
-    }
-
-    public synchronized void setTuple(ITupleReference tuple) {
-        this.tuple = tuple;
-    }
-
-    public synchronized IndexOp getIndexOperation() {
-        return indexOperation;
-    }
-
-    public synchronized void setIndexOperation(IndexOp indexOperation) {
-        this.indexOperation = indexOperation;
-    }
-
-    public synchronized LogicalLogLocator getLogicalLogLocator() {
-        return logicalLogLocator;
-    }
-
-    public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
-        this.logicalLogLocator = logicalLogLocator;
-    }
-
-}
-
-public class TreeLogger implements ILogger, ICloseable {
-
-    private static final byte resourceMgrId = TreeResourceManager.ID;
-    private final Map<Object, Object> arguments = new ConcurrentHashMap<Object, Object>();
-
-    public static final String TREE_INDEX = "TREE_INDEX";
-    public static final String TUPLE_REFERENCE = "TUPLE_REFERENCE";
-    public static final String TUPLE_WRITER = "TUPLE_WRITER";
-    public static final String INDEX_OPERATION = "INDEX_OPERATION";
-    public static final String RESOURCE_ID = "RESOURCE_ID";
-
-    private final ITreeIndex treeIndex;
-    private final ITreeIndexTupleWriter treeIndexTupleWriter;
-    private final byte[] resourceIdBytes;
-    private final byte[] resourceIdLengthBytes;
-
-    public class BTreeOperationCodes {
-        public static final byte INSERT = 0;
-        public static final byte DELETE = 1;
-    }
-
-    public TreeLogger(byte[] resourceIdBytes, ITreeIndex treeIndex) {
-        this.resourceIdBytes = resourceIdBytes;
-        this.treeIndex = treeIndex;
-        treeIndexTupleWriter = treeIndex.getLeafFrameFactory().getTupleWriterFactory().createTupleWriter();
-        this.resourceIdLengthBytes = DataUtil.intToByteArray(resourceIdBytes.length);
-    }
-
-    public synchronized void close(TransactionContext context) {
-        TransactionState txnState = (TransactionState) arguments.get(context.getTransactionID());
-        txnState.remove(Thread.currentThread().getId());
-        arguments.remove(context.getTransactionID());
-    }
-
-    public void generateLogRecord(TransactionProvider provider, TransactionContext context, IndexOp operation,
-            ITupleReference tuple) throws ACIDException {
-        context.addCloseableResource(this); // the close method would be called
-        // on this TreeLogger instance at
-        // the time of transaction
-        // commit/abort.
-        if (operation != IndexOp.INSERT && operation != IndexOp.DELETE) {
-            throw new ACIDException("Loging for Operation " + operation + " not supported");
-
-        }
-
-        TxnThreadState txnThreadState = null;
-        TransactionState txnState;
-        txnState = (TransactionState) arguments.get(context.getTransactionID());
-        if (txnState == null) {
-            synchronized (context) { // threads belonging to different
-                // transaction do not need to
-                // synchronize amongst them.
-                if (txnState == null) {
-                    txnState = new TransactionState();
-                    arguments.put(context.getTransactionID(), txnState);
-                }
-            }
-        }
-
-        txnThreadState = txnState.getTransactionThreadState(Thread.currentThread().getId());
-        if (txnThreadState == null) {
-            LogicalLogLocator logicalLogLocator = LogUtil.getDummyLogicalLogLocator(provider.getLogManager());
-            txnThreadState = new TxnThreadState(logicalLogLocator, operation, tuple);
-            txnState.putTransactionThreadState(Thread.currentThread().getId(), txnThreadState);
-        }
-        txnThreadState.setIndexOperation(operation);
-        txnThreadState.setTuple(tuple);
-        int tupleSize = treeIndexTupleWriter.bytesRequired(tuple);
-        // Below 4 is for the int representing the length of resource id and 1
-        // is for
-        // the byte representing the operation
-        int logContentLength = 4 + resourceIdBytes.length + 1 + tupleSize;
-        provider.getLogManager().log(txnThreadState.getLogicalLogLocator(), context, resourceMgrId, 0L, LogType.UPDATE,
-                LogActionType.REDO_UNDO, logContentLength, (ILogger) this, arguments);
-    }
-
-    @Override
-    public void log(TransactionContext context, LogicalLogLocator logicalLogLocator, int logRecordSize,
-            Map<Object, Object> loggerArguments) throws ACIDException {
-        TransactionState txnState = (TransactionState) loggerArguments.get(context.getTransactionID());
-        TxnThreadState state = (TxnThreadState) txnState.getTransactionThreadState(Thread.currentThread().getId());
-        int count = 0;
-        byte[] logBuffer = logicalLogLocator.getBuffer().getArray();
-        System.arraycopy(resourceIdLengthBytes, 0, logBuffer, logicalLogLocator.getMemoryOffset(), 4);
-        count += 4; // count is incremented by 4 because we wrote the length
-                    // that is an int and hence 4 bytes
-        System.arraycopy(resourceIdBytes, 0, logBuffer, logicalLogLocator.getMemoryOffset() + count,
-                resourceIdBytes.length);
-        count += resourceIdBytes.length;
-        logBuffer[logicalLogLocator.getMemoryOffset() + count] = (byte) state.getIndexOperation().ordinal();
-        count += 1; // count is incremented by 1 to account for the byte
-                    // written.
-        treeIndexTupleWriter.writeTuple(state.getTuple(), logicalLogLocator.getBuffer().getArray(),
-                logicalLogLocator.getMemoryOffset() + count);
-    }
-
-    @Override
-    public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
-    }
-
-    @Override
-    public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java
deleted file mode 100644
index 34e1466..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeLoggerRepository.java
+++ /dev/null
@@ -1,44 +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.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-
-public class TreeLoggerRepository {
-
-    private final Map<ByteBuffer, TreeLogger> loggers = new HashMap<ByteBuffer, TreeLogger>();
-    private final TransactionProvider provider;
-
-    public TreeLoggerRepository(TransactionProvider provider) {
-        this.provider = provider;
-    }
-
-    public synchronized TreeLogger getTreeLogger(byte[] resourceIdBytes) {
-        ByteBuffer resourceId = ByteBuffer.wrap(resourceIdBytes);
-        TreeLogger logger = loggers.get(resourceId);
-        if (logger == null) {
-            ITreeIndex treeIndex = (ITreeIndex) provider.getTransactionalResourceRepository().getTransactionalResource(
-                    resourceIdBytes);
-            logger = new TreeLogger(resourceIdBytes, treeIndex);
-            loggers.put(resourceId, logger);
-        }
-        return logger;
-    }
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
deleted file mode 100644
index 7c6d05a..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/TreeResourceManager.java
+++ /dev/null
@@ -1,80 +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.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexAccessor;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexTupleReference;
-
-public class TreeResourceManager implements IResourceManager {
-
-    public static final byte ID = (byte) 1;
-
-    private final TransactionProvider provider;
-
-    public TreeResourceManager(TransactionProvider provider) {
-        this.provider = provider;
-    }
-
-    public byte getResourceManagerId() {
-        return ID;
-    }
-
-    public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
-
-        int logContentBeginPos = logRecordHelper.getLogContentBeginPos(logLocator);
-        byte[] logBufferContent = logLocator.getBuffer().getArray();
-        // read the length of resource id byte array
-        int resourceIdLength = DataUtil.byteArrayToInt(logBufferContent, logContentBeginPos);
-        byte[] resourceIdBytes = new byte[resourceIdLength];
-
-        // copy the resource if bytes
-        System.arraycopy(logBufferContent, logContentBeginPos + 4, resourceIdBytes, 0, resourceIdLength);
-
-        // look up the repository to obtain the resource object
-        ITreeIndex treeIndex = (ITreeIndex) provider.getTransactionalResourceRepository().getTransactionalResource(
-                resourceIdBytes);
-        int operationOffset = logContentBeginPos + 4 + resourceIdLength;
-        int tupleBeginPos = operationOffset + 1;
-
-        ITreeIndexTupleReference tupleReference = treeIndex.getLeafFrameFactory().getTupleWriterFactory()
-                .createTupleWriter().createTupleReference();
-        // TODO: remove this call.
-        tupleReference.setFieldCount(tupleReference.getFieldCount());
-        tupleReference.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), tupleBeginPos);
-        byte operation = logBufferContent[operationOffset];
-        IIndexAccessor treeIndexAccessor = treeIndex.createAccessor();
-        try {
-            switch (operation) {
-                case TreeLogger.BTreeOperationCodes.INSERT:
-                    treeIndexAccessor.delete(tupleReference);
-                    break;
-                case TreeLogger.BTreeOperationCodes.DELETE:
-                    treeIndexAccessor.insert(tupleReference);
-                    break;
-            }
-        } catch (Exception e) {
-            throw new ACIDException(" could not rollback ", e);
-        }
-    }
-
-    public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException {
-        throw new UnsupportedOperationException(" Redo logic will be implemented as part of crash recovery feature");
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
new file mode 100644
index 0000000..f4bb73a
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
@@ -0,0 +1,54 @@
+package edu.uci.ics.asterix.transaction.management.service.recovery;
+
+import java.io.Serializable;
+
+public class CheckpointObject implements Serializable, Comparable<CheckpointObject> {
+
+    private static final long serialVersionUID = 1L;
+ 
+    private final long checkpointLSN;
+    private final long minMCTFirstLSN;
+    private final int maxJobId;
+    private final long timeStamp;
+
+    public CheckpointObject(long checkpointLSN, long minMCTFirstLSN, int maxJobId, long timeStamp) {
+        this.checkpointLSN = checkpointLSN;
+        this.minMCTFirstLSN = minMCTFirstLSN;
+        this.maxJobId = maxJobId;
+        this.timeStamp = timeStamp;
+    }
+    
+    public long getCheckpointLSN() {
+        return checkpointLSN;
+    }
+
+    public long getMinMCTFirstLSN() {
+        return minMCTFirstLSN;
+    }
+
+    public int getMaxJobId() {
+        return maxJobId;
+    }
+
+    public long getTimeStamp() {
+        return timeStamp;
+    }
+
+    @Override
+    public int compareTo(CheckpointObject checkpointObject) {
+        long compareTimeStamp = checkpointObject.getTimeStamp(); 
+        
+        //decending order
+        long diff = compareTimeStamp - this.timeStamp;
+        if (diff > 0) {
+            return 1;
+        } else if (diff == 0){
+            return 0;
+        } else {
+            return -1;
+        }
+ 
+        //ascending order
+        //return this.timeStamp - compareTimeStamp;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
new file mode 100644
index 0000000..3eb87bc
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
@@ -0,0 +1,66 @@
+package edu.uci.ics.asterix.transaction.management.service.recovery;
+
+import java.util.List;
+
+import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+
+public class CheckpointThread extends Thread {
+
+    private static final long LSN_THRESHOLD = 64 * 1024 * 1024;
+    private long checkpointTermInSecs = 120; //seconds.
+
+    private long lastMinMCTFirstLSN = 0;
+
+    private final IRecoveryManager recoveryMgr;
+    private final IIndexLifecycleManager indexLifecycleManager;
+
+    public CheckpointThread(IRecoveryManager recoveryMgr, IIndexLifecycleManager indexLifecycleManager,
+            long checkpointTermInSecs) {
+        this.recoveryMgr = recoveryMgr;
+        this.indexLifecycleManager = indexLifecycleManager;
+        if (this.checkpointTermInSecs < checkpointTermInSecs) {
+            this.checkpointTermInSecs = checkpointTermInSecs;
+        }
+    }
+
+    @Override
+    public void run() {
+        long currentMinMCTFirstLSN = 0;
+        while (true) {
+            try {
+                sleep(checkpointTermInSecs * 1000);
+            } catch (InterruptedException e) {
+                //ignore
+            }
+
+            currentMinMCTFirstLSN = getMinMCTFirstLSN();
+            if (currentMinMCTFirstLSN - lastMinMCTFirstLSN > LSN_THRESHOLD) {
+                try {
+                    recoveryMgr.checkpoint(false);
+                    lastMinMCTFirstLSN = currentMinMCTFirstLSN;
+                } catch (ACIDException e) {
+                    throw new Error("failed to checkpoint", e);
+                }
+            }
+        }
+    }
+
+    private long getMinMCTFirstLSN() {
+        List<IIndex> openIndexList = indexLifecycleManager.getOpenIndexes();
+        long minMCTFirstLSN = Long.MAX_VALUE;
+        long firstLSN;
+        if (openIndexList.size() > 0) {
+            for (IIndex index : openIndexList) {
+                firstLSN = ((IndexOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
+            }
+        } else {
+            minMCTFirstLSN = -1;
+        }
+        return minMCTFirstLSN;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java
new file mode 100644
index 0000000..3ae7da8
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IAsterixAppRuntimeContextProvider.java
@@ -0,0 +1,48 @@
+package edu.uci.ics.asterix.transaction.management.service.recovery;
+
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.io.IIOManager;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
+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.ILSMOperationTrackerFactory;
+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 interface IAsterixAppRuntimeContextProvider {
+
+    public IBufferCache getBufferCache();
+
+    public IFileMapProvider getFileMapManager();
+
+    public TransactionSubsystem getTransactionSubsystem();
+
+    public IIndexLifecycleManager getIndexLifecycleManager();
+
+    public ILSMMergePolicy getLSMMergePolicy();
+
+    public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory();
+    
+    public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory();
+    
+    public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory();
+    
+    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider();
+    
+    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider();
+    
+    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider();
+    
+    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider();
+    
+    public ILSMIOOperationScheduler getLSMIOScheduler();
+
+    public ILocalResourceRepository getLocalResourceRepository();
+
+    public ResourceIdFactory getResourceIdFactory();
+    
+    public IIOManager getIOManager();
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IRecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IRecoveryManager.java
index 2e0fd47..38802a2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IRecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/IRecoveryManager.java
@@ -28,6 +28,7 @@
 public interface IRecoveryManager {
 
     public enum SystemState {
+        NEW_UNIVERSE,
         RECOVERING,
         HEALTHY,
         CORRUPTED
@@ -57,7 +58,7 @@
      *         recovery.
      * @throws ACIDException
      */
-    public SystemState startRecovery(boolean synchronous) throws IOException, ACIDException;
+    public void startRecovery(boolean synchronous) throws IOException, ACIDException;
 
     /**
      * Rolls back a transaction.
@@ -67,4 +68,6 @@
      * @throws ACIDException
      */
     public void rollbackTransaction(TransactionContext txnContext) throws ACIDException;
+
+    public void checkpoint(boolean isSharpCheckpoint) throws ACIDException;
 }
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..c27ae20 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
@@ -14,36 +14,66 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.recovery;
 
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
 import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.FilenameFilter;
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
-import edu.uci.ics.asterix.transaction.management.service.logging.FileUtil;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
+import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.service.logging.IBuffer;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogCursor;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogFilter;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogRecordHelper;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogRecordHelper;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 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.logging.PhysicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.IResourceManager.ResourceType;
 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.asterix.transaction.management.service.transaction.TransactionManager;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
+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.ITreeIndexMetaDataFrame;
+import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeImmutableComponent;
+import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
+import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
+import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
+import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
+import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
+import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 
 /**
  * This is the Recovery Manager and is responsible for rolling back a
@@ -53,208 +83,737 @@
  */
 public class RecoveryManager implements IRecoveryManager {
 
+    public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
-    private TransactionProvider transactionProvider;
+    private final TransactionSubsystem txnSubsystem;
 
     /**
      * A file at a known location that contains the LSN of the last log record
      * traversed doing a successful checkpoint.
      */
-    private static final String checkpoint_record_file = "last_checkpoint_lsn";
+    private static final String CHECKPOINT_FILENAME_PREFIX = "checkpoint_";
     private SystemState state;
-    private Map<Long, TransactionTableEntry> transactionTable;
-    private Map<Long, List<PhysicalLogLocator>> dirtyPagesTable;
 
-    public RecoveryManager(TransactionProvider TransactionProvider) throws ACIDException {
-        this.transactionProvider = TransactionProvider;
-        try {
-            FileUtil.createFileIfNotExists(checkpoint_record_file);
-        } catch (IOException ioe) {
-            throw new ACIDException(" unable to create checkpoint record file " + checkpoint_record_file, ioe);
-        }
-    }
-
-    public SystemState getSystemState() throws ACIDException {
-        return state;
-    }
-
-    private PhysicalLogLocator getBeginRecoveryLSN() throws ACIDException {
-        return new PhysicalLogLocator(0, transactionProvider.getLogManager());
+    public RecoveryManager(TransactionSubsystem TransactionProvider) throws ACIDException {
+        this.txnSubsystem = TransactionProvider;
     }
 
     /**
-     * TODO:This method is currently not implemented completely.
+     * returns system state which could be one of the three states: HEALTHY, RECOVERING, CORRUPTED.
+     * This state information could be used in a case where more than one thread is running
+     * in the bootstrap process to provide higher availability. In other words, while the system
+     * is recovered, another thread may start a new transaction with understanding the side effect
+     * of the operation, or the system can be recovered concurrently. This kind of concurrency is
+     * not supported, yet.
      */
-    public SystemState startRecovery(boolean synchronous) throws IOException, ACIDException {
-        ILogManager logManager = transactionProvider.getLogManager();
-        state = SystemState.RECOVERING;
-        transactionTable = new HashMap<Long, TransactionTableEntry>();
-        dirtyPagesTable = new HashMap<Long, List<PhysicalLogLocator>>();
+    public SystemState getSystemState() throws ACIDException {
 
-        PhysicalLogLocator beginLSN = getBeginRecoveryLSN();
-        ILogCursor cursor = logManager.readLog(beginLSN, new ILogFilter() {
+        //#. read checkpoint file
+        CheckpointObject checkpointObject = null;
+        try {
+            checkpointObject = readCheckpoint();
+        } catch (FileNotFoundException e) {
+            //This is initial bootstrap. 
+            //Otherwise, the checkpoint file is deleted unfortunately. What we can do in this case?
+            state = SystemState.NEW_UNIVERSE;
+            return state;
+        }
+
+        //#. if minMCTFirstLSN is equal to -1 && 
+        //   checkpointLSN in the checkpoint file is equal to the lastLSN in the log file,
+        //   then return healthy state. Otherwise, return corrupted.
+        LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
+        if (checkpointObject.getMinMCTFirstLSN() == -1
+                && checkpointObject.getCheckpointLSN() == logMgr.getCurrentLsn().get()) {
+            state = SystemState.HEALTHY;
+            return state;
+        } else {
+            state = SystemState.CORRUPTED;
+            return state;
+        }
+    }
+
+    public void startRecovery(boolean synchronous) throws IOException, ACIDException {
+
+        int updateLogCount = 0;
+        int commitLogCount = 0;
+        int redoCount = 0;
+
+        state = SystemState.RECOVERING;
+
+        ILogManager logManager = txnSubsystem.getLogManager();
+        ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("[RecoveryMgr] starting recovery ...");
+        }
+
+        //winnerTxnTable is used to add pairs, <committed TxnId, the most recent commit LSN of the TxnId>
+        Map<TxnId, Long> winnerTxnTable = new HashMap<TxnId, Long>();
+        TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
+        byte logType;
+
+        //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
+        CheckpointObject checkpointObject = readCheckpoint();
+        long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLSN();
+        if (lowWaterMarkLSN == -1) {
+            lowWaterMarkLSN = 0;
+        }
+        int maxJobId = checkpointObject.getMaxJobId();
+        int currentJobId;
+
+        //-------------------------------------------------------------------------
+        //  [ analysis phase ]
+        //  - collect all committed LSN 
+        //  - if there are duplicate commits for the same TxnId, 
+        //    keep only the mostRecentCommitLSN among the duplicates.
+        //-------------------------------------------------------------------------
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("[RecoveryMgr] in analysis phase");
+        }
+
+        //#. set log cursor to the lowWaterMarkLSN
+        ILogCursor logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager),
+                new ILogFilter() {
+                    public boolean accept(IBuffer logs, long startOffset, int endOffset) {
+                        return true;
+                    }
+                });
+        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
+
+        //#. collect all committed txn's pairs,<TxnId, LSN>
+        while (logCursor.next(currentLogLocator)) {
+
+            if (LogManager.IS_DEBUG_MODE) {
+                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+            }
+
+            logType = logRecordHelper.getLogType(currentLogLocator);
+
+            //update max jobId
+            currentJobId = logRecordHelper.getJobId(currentLogLocator);
+            if (currentJobId > maxJobId) {
+                maxJobId = currentJobId;
+            }
+
+            TxnId commitTxnId = null;
+            switch (logType) {
+                case LogType.UPDATE:
+                    if (IS_DEBUG_MODE) {
+                        updateLogCount++;
+                    }
+                    break;
+
+                case LogType.COMMIT:
+                case LogType.ENTITY_COMMIT:
+                    commitTxnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
+                            logRecordHelper.getDatasetId(currentLogLocator),
+                            logRecordHelper.getPKHashValue(currentLogLocator));
+                    winnerTxnTable.put(commitTxnId, currentLogLocator.getLsn());
+                    if (IS_DEBUG_MODE) {
+                        commitLogCount++;
+                    }
+                    break;
+
+                default:
+                    throw new ACIDException("Unsupported LogType: " + logType);
+            }
+        }
+
+        //-------------------------------------------------------------------------
+        //  [ redo phase ]
+        //  - redo if
+        //    1) The TxnId is committed --> gurantee durability
+        //      &&  
+        //    2) the currentLSN > maxDiskLastLSN of the index --> guarantee idempotance
+        //-------------------------------------------------------------------------
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("[RecoveryMgr] in redo phase");
+        }
+        //#. set log cursor to the lowWaterMarkLSN again.
+        logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager), new ILogFilter() {
             public boolean accept(IBuffer logs, long startOffset, int endOffset) {
                 return true;
             }
         });
-        LogicalLogLocator memLSN = new LogicalLogLocator(beginLSN.getLsn(), null, -1, logManager);
-        boolean logValidity = true;
-        LogRecordHelper parser = new LogRecordHelper(logManager);
-        try {
-            while (logValidity) {
-                logValidity = cursor.next(memLSN);
-                if (!logValidity) {
-                    if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("reached end of log !");
+        currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
+
+        long resourceId;
+        byte resourceMgrId;
+        long maxDiskLastLSN;
+        long currentLSN = -1;
+        int resourceType;
+        ILSMIndex index = null;
+        LocalResource localResource = null;
+        ILocalResourceMetadata localResourceMetadata = null;
+        Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
+        List<ILSMComponent> immutableDiskIndexList = null;
+        TxnId jobLevelTxnId = new TxnId(-1, -1, -1);
+        boolean foundWinnerTxn;
+
+        //#. get indexLifeCycleManager 
+        IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
+        IIndexLifecycleManager indexLifecycleManager = appRuntimeContext.getIndexLifecycleManager();
+        ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
+
+        //#. redo
+        while (logCursor.next(currentLogLocator)) {
+            foundWinnerTxn = false;
+            if (LogManager.IS_DEBUG_MODE) {
+                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+            }
+
+            logType = logRecordHelper.getLogType(currentLogLocator);
+
+            switch (logType) {
+                case LogType.UPDATE:
+                    tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
+                            logRecordHelper.getDatasetId(currentLogLocator),
+                            logRecordHelper.getPKHashValue(currentLogLocator));
+                    jobLevelTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator), -1, -1);
+                    if (winnerTxnTable.containsKey(tempKeyTxnId)) {
+                        currentLSN = winnerTxnTable.get(tempKeyTxnId);
+                        foundWinnerTxn = true;
+                    } else if (winnerTxnTable.containsKey(jobLevelTxnId)) {
+                        currentLSN = winnerTxnTable.get(jobLevelTxnId);
+                        foundWinnerTxn = true;
+                    }
+
+                    if (foundWinnerTxn) {
+                        resourceId = logRecordHelper.getResourceId(currentLogLocator);
+                        localResource = localResourceRepository.getResourceById(resourceId);
+
+                        //get index instance from IndexLifeCycleManager
+                        //if index is not registered into IndexLifeCycleManager,
+                        //create the index using LocalMetadata stored in LocalResourceRepository
+                        index = (ILSMIndex) indexLifecycleManager.getIndex(resourceId);
+                        if (index == null) {
+
+                            /*******************************************************************
+                             * [Notice]
+                             * -> Issue
+                             * Delete index may cause a problem during redo.
+                             * The index operation to be redone couldn't be redone because the corresponding index
+                             * may not exist in NC due to the possible index drop DDL operation.
+                             * -> Approach
+                             * Avoid the problem during redo.
+                             * More specifically, the problem will be detected when the localResource of
+                             * the corresponding index is retrieved, which will end up with 'null' return from
+                             * localResourceRepository. If null is returned, then just go and process the next
+                             * log record.
+                             *******************************************************************/
+                            if (localResource == null) {
+                                continue;
+                            }
+                            /*******************************************************************/
+
+                            //#. create index instance and register to indexLifeCycleManager
+                            localResourceMetadata = (ILocalResourceMetadata) localResource.getResourceObject();
+                            index = localResourceMetadata.createIndexInstance(appRuntimeContext,
+                                    localResource.getResourceName(), localResource.getPartition());
+                            indexLifecycleManager.register(resourceId, index);
+                            indexLifecycleManager.open(resourceId);
+
+                            //#. get maxDiskLastLSN
+                            resourceType = localResource.getResourceType();
+                            immutableDiskIndexList = index.getImmutableComponents();
+
+                            maxDiskLastLSN = -1;
+                            switch (resourceType) {
+
+                                case ResourceType.LSM_BTREE:
+                                    for (ILSMComponent c : immutableDiskIndexList) {
+                                        BTree btree = ((LSMBTreeImmutableComponent) c).getBTree();
+                                        maxDiskLastLSN = Math.max(getTreeIndexLSN(btree), maxDiskLastLSN);
+                                    }
+                                    break;
+
+                                case ResourceType.LSM_RTREE:
+                                    for (ILSMComponent c : immutableDiskIndexList) {
+                                        RTree rtree = ((LSMRTreeImmutableComponent) c).getRTree();
+                                        maxDiskLastLSN = Math.max(getTreeIndexLSN(rtree), maxDiskLastLSN);
+                                    }
+                                    break;
+
+                                case ResourceType.LSM_INVERTED_INDEX:
+                                    for (ILSMComponent c : immutableDiskIndexList) {
+                                        BTree delKeyBtree = ((LSMInvertedIndexImmutableComponent) c)
+                                                .getDeletedKeysBTree();
+                                        maxDiskLastLSN = Math.max(getTreeIndexLSN(delKeyBtree), maxDiskLastLSN);
+                                    }
+                                    break;
+
+                                default:
+                                    throw new ACIDException("Unsupported resouce type");
+                            }
+
+                            //#. set resourceId and maxDiskLastLSN to the map
+                            resourceId2MaxLSNMap.put(resourceId, maxDiskLastLSN);
+                        } else {
+                            maxDiskLastLSN = resourceId2MaxLSNMap.get(resourceId);
+                        }
+
+                        if (currentLSN > maxDiskLastLSN) {
+                            resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
+
+                            // look up the repository to get the resource manager
+                            // register resourceMgr if it is not registered. 
+                            IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
+                                    .getTransactionalResourceMgr(resourceMgrId);
+                            if (resourceMgr == null) {
+                                resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
+                                txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
+                                        resourceMgrId, resourceMgr);
+                            }
+
+                            //redo finally.
+                            resourceMgr.redo(logRecordHelper, currentLogLocator);
+                            if (IS_DEBUG_MODE) {
+                                redoCount++;
+                            }
+                        }
                     }
                     break;
+
+                case LogType.COMMIT:
+                case LogType.ENTITY_COMMIT:
+                    //do nothing
+                    break;
+
+                default:
+                    throw new ACIDException("Unsupported LogType: " + logType);
+            }
+        }
+
+        //close all indexes
+        Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
+        for (long r : resourceIdList) {
+            indexLifecycleManager.close(r);
+        }
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("[RecoveryMgr] recovery is over");
+        }
+        if (IS_DEBUG_MODE) {
+            System.out.println("[RecoveryMgr] Count: Update/Commit/Redo = " + updateLogCount + "/" + commitLogCount
+                    + "/" + redoCount);
+        }
+    }
+
+    //TODO
+    //This function came from the AbstractLSMIOOperationCallback class. 
+    //We'd better factor out this function into a component of reading/writing the local metadata of indexes.
+    private long getTreeIndexLSN(ITreeIndex treeIndex) throws HyracksDataException {
+        int fileId = treeIndex.getFileId();
+        IBufferCache bufferCache = treeIndex.getBufferCache();
+        ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
+        int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
+        ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
+        metadataPage.acquireReadLatch();
+        try {
+            metadataFrame.setPage(metadataPage);
+            return metadataFrame.getLSN();
+        } finally {
+            metadataPage.releaseReadLatch();
+            bufferCache.unpin(metadataPage);
+        }
+    }
+
+    @Override
+    public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException {
+
+        LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
+        TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
+        String logDir = logMgr.getLogManagerProperties().getLogDir();
+
+        //#. get the filename of the previous checkpoint files which are about to be deleted 
+        //   right after the new checkpoint file is written.
+        File[] prevCheckpointFiles = getPreviousCheckpointFiles();
+
+        IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
+                .getIndexLifecycleManager();
+        List<IIndex> openIndexList = indexLifecycleManager.getOpenIndexes();
+        List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
+
+        //#. flush all in-memory components if it is the sharp checkpoint
+        if (isSharpCheckpoint) {
+            for (IIndex index : openIndexList) {
+                ILSMIndex lsmIndex = (ILSMIndex) index;
+                ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(NoOpOperationCallback.INSTANCE,
+                        NoOpOperationCallback.INSTANCE);
+                IndexOperationTracker indexOpTracker = (IndexOperationTracker) lsmIndex.getOperationTracker();
+                BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
+                        indexOpTracker.getIOOperationCallback());
+                callbackList.add(cb);
+                try {
+                    indexAccessor.scheduleFlush(cb);
+                } catch (HyracksDataException e) {
+                    throw new ACIDException(e);
                 }
-                byte resourceMgrId = parser.getResourceMgrId(memLSN);
-                IResourceManager resourceMgr = transactionProvider.getTransactionalResourceRepository()
-                        .getTransactionalResourceMgr(resourceMgrId);
-                if (resourceMgr == null) {
-                    throw new ACIDException("unknown resource mgr with id " + resourceMgrId);
-                } else {
-                    byte actionType = parser.getLogActionType(memLSN);
-                    switch (actionType) {
-                        case LogActionType.REDO:
-                            resourceMgr.redo(parser, memLSN);
-                            break;
-                        case LogActionType.UNDO: /* skip these records */
-                            break;
-                        default: /* do nothing */
+            }
+
+            for (BlockingIOOperationCallbackWrapper cb : callbackList) {
+                try {
+                    cb.waitForIO();
+                } catch (InterruptedException e) {
+                    throw new ACIDException(e);
+                }
+            }
+        }
+
+        //#. create and store the checkpointObject into the new checkpoint file
+        long minMCTFirstLSN = Long.MAX_VALUE;
+        long firstLSN;
+        if (openIndexList.size() > 0) {
+            for (IIndex index : openIndexList) {
+                firstLSN = ((IndexOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
+            }
+        } else {
+            minMCTFirstLSN = -1;
+        }
+
+        CheckpointObject checkpointObject = new CheckpointObject(logMgr.getCurrentLsn().get(), minMCTFirstLSN,
+                txnMgr.getMaxJobId(), System.currentTimeMillis());
+
+        FileOutputStream fos = null;
+        ObjectOutputStream oosToFos = null;
+        try {
+            String fileName = getFileName(logDir, Long.toString(checkpointObject.getTimeStamp()));
+            fos = new FileOutputStream(fileName);
+            oosToFos = new ObjectOutputStream(fos);
+            oosToFos.writeObject(checkpointObject);
+            oosToFos.flush();
+        } catch (IOException e) {
+            throw new ACIDException("Failed to checkpoint", e);
+        } finally {
+            if (oosToFos != null) {
+                try {
+                    oosToFos.close();
+                } catch (IOException e) {
+                    throw new ACIDException("Failed to checkpoint", e);
+                }
+            }
+            if (oosToFos == null && fos != null) {
+                try {
+                    fos.close();
+                } catch (IOException e) {
+                    throw new ACIDException("Failed to checkpoint", e);
+                }
+            }
+        }
+
+        //#. delete the previous checkpoint files
+        if (prevCheckpointFiles != null) {
+            for (File file : prevCheckpointFiles) {
+                file.delete();
+            }
+        }
+
+        if (isSharpCheckpoint) {
+            logMgr.renewLogFiles();
+        }
+    }
+
+    private CheckpointObject readCheckpoint() throws ACIDException, FileNotFoundException {
+
+        CheckpointObject checkpointObject = null;
+
+        //#. read all checkpointObjects from the existing checkpoint files
+        File[] prevCheckpointFiles = getPreviousCheckpointFiles();
+
+        if (prevCheckpointFiles == null || prevCheckpointFiles.length == 0) {
+            throw new FileNotFoundException("Checkpoint file is not found");
+        }
+
+        List<CheckpointObject> checkpointObjectList = new ArrayList<CheckpointObject>();
+
+        for (File file : prevCheckpointFiles) {
+            FileInputStream fis = null;
+            ObjectInputStream oisFromFis = null;
+
+            try {
+                fis = new FileInputStream(file);
+                oisFromFis = new ObjectInputStream(fis);
+                checkpointObject = (CheckpointObject) oisFromFis.readObject();
+                checkpointObjectList.add(checkpointObject);
+            } catch (Exception e) {
+                throw new ACIDException("Failed to read a checkpoint file", e);
+            } finally {
+                if (oisFromFis != null) {
+                    try {
+                        oisFromFis.close();
+                    } catch (IOException e) {
+                        throw new ACIDException("Failed to read a checkpoint file", e);
                     }
                 }
-                writeCheckpointRecord(memLSN.getLsn());
+                if (oisFromFis == null && fis != null) {
+                    try {
+                        fis.close();
+                    } catch (IOException e) {
+                        throw new ACIDException("Failed to read a checkpoint file", e);
+                    }
+                }
             }
-            state = SystemState.HEALTHY;
-        } catch (Exception e) {
-            state = SystemState.CORRUPTED;
-            throw new ACIDException(" could not recover , corrputed log !", e);
         }
-        return state;
+
+        //#. sort checkpointObjects in descending order by timeStamp to find out the most recent one.
+        Collections.sort(checkpointObjectList);
+
+        //#. return the most recent one (the first one in sorted list)
+        return checkpointObjectList.get(0);
     }
 
-    private void writeCheckpointRecord(long lsn) throws ACIDException {
-        try {
-            FileWriter writer = new FileWriter(new File(checkpoint_record_file));
-            BufferedWriter buffWriter = new BufferedWriter(writer);
-            buffWriter.write("" + lsn);
-            buffWriter.flush();
-        } catch (IOException ioe) {
-            throw new ACIDException(" unable to create check point record", ioe);
-        }
+    private File[] getPreviousCheckpointFiles() {
+        String logDir = txnSubsystem.getLogManager().getLogManagerProperties().getLogDir();
+
+        File parentDir = new File(logDir);
+
+        FilenameFilter filter = new FilenameFilter() {
+            @Override
+            public boolean accept(File dir, String name) {
+                if (name.contains(CHECKPOINT_FILENAME_PREFIX)) {
+                    return true;
+                } else {
+                    return false;
+                }
+            }
+        };
+
+        File[] prevCheckpointFiles = parentDir.listFiles(filter);
+
+        return prevCheckpointFiles;
     }
 
-    /*
-     * Currently this method is not used, but will be used as part of crash
-     * recovery logic.
-     */
-    private long getLastCheckpointRecordLSN() throws Exception {
-        FileReader reader;
-        BufferedReader buffReader;
-        String content = null;
-        reader = new FileReader(new File(checkpoint_record_file));
-        buffReader = new BufferedReader(reader);
-        content = buffReader.readLine();
-        if (content != null) {
-            return Long.parseLong(content);
+    private String getFileName(String baseDir, String suffix) {
+
+        if (!baseDir.endsWith(System.getProperty("file.separator"))) {
+            baseDir += System.getProperty("file.separator");
         }
-        return -1;
+
+        return baseDir + CHECKPOINT_FILENAME_PREFIX + suffix;
     }
 
     /**
-     * Rollback a transaction (non-Javadoc)
+     * Rollback a transaction
      * 
-     * @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 {
-        ILogManager logManager = transactionProvider.getLogManager();
-        ILogRecordHelper parser = logManager.getLogRecordHelper();
+        ILogManager logManager = txnSubsystem.getLogManager();
+        ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
+        Map<TxnId, List<Long>> loserTxnTable = new HashMap<TxnId, List<Long>>();
+        TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
 
-        // Obtain the last log record written by the transaction
-        PhysicalLogLocator lsn = txnContext.getLastLogLocator();
+        int updateLogCount = 0;
+        int commitLogCount = 0;
+
+        // Obtain the first log record written by the Job
+        PhysicalLogLocator firstLSNLogLocator = txnContext.getFirstLogLocator();
+        PhysicalLogLocator lastLSNLogLocator = txnContext.getLastLogLocator();
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" rollbacking transaction log records at lsn " + lsn.getLsn());
+            LOGGER.info(" rollbacking transaction log records from " + firstLSNLogLocator.getLsn() + " to "
+                    + lastLSNLogLocator.getLsn());
         }
 
         // check if the transaction actually wrote some logs.
-        if (lsn.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
+        if (firstLSNLogLocator.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;
         }
 
-        // a dummy logLocator instance that is re-used during rollback
-        LogicalLogLocator logLocator = LogUtil.getDummyLogicalLogLocator(logManager);
+        // While reading log records from firstLSN to lastLSN, collect uncommitted txn's LSNs 
+        ILogCursor logCursor;
+        try {
+            logCursor = logManager.readLog(firstLSNLogLocator, new ILogFilter() {
+                @Override
+                public boolean accept(IBuffer buffer, long startOffset, int length) {
+                    return true;
+                }
+            });
+        } catch (IOException e) {
+            throw new ACIDException("Failed to create LogCursor with LSN:" + firstLSNLogLocator.getLsn(), e);
+        }
 
-        while (true) {
+        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
+        boolean valid;
+        byte logType;
+        List<Long> undoLSNSet = null;
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(" collecting loser transaction's LSNs from " + firstLSNLogLocator.getLsn() + " to "
+                    + +lastLSNLogLocator.getLsn());
+        }
+
+        while (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
             try {
-                // read the log record at the given position
-                logLocator = logManager.readLog(lsn);
-            } catch (Exception e) {
-                e.printStackTrace();
-                state = SystemState.CORRUPTED;
-                throw new ACIDException(" could not read log at lsn :" + lsn, e);
+                valid = logCursor.next(currentLogLocator);
+            } catch (IOException e) {
+                throw new ACIDException("Failed to read log at LSN:" + currentLogLocator.getLsn(), e);
+            }
+            if (!valid) {
+                if (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
+                    throw new ACIDException("LastLSN mismatch: " + lastLSNLogLocator.getLsn() + " vs "
+                            + currentLogLocator.getLsn() + " during Rollback a transaction( " + txnContext.getJobId()
+                            + ")");
+                } else {
+                    break;//End of Log File
+                }
             }
 
-            byte logType = parser.getLogType(logLocator);
-            if (LOGGER.isLoggable(Level.FINE)) {
-                LOGGER.fine(" reading LSN value inside rollback transaction method " + txnContext.getLastLogLocator()
-                        + " txn id " + parser.getLogTransactionId(logLocator) + " log type  " + logType);
+            if (IS_DEBUG_MODE) {
+                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
             }
 
+            tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
+                    logRecordHelper.getDatasetId(currentLogLocator), logRecordHelper.getPKHashValue(currentLogLocator));
+            logType = logRecordHelper.getLogType(currentLogLocator);
+
             switch (logType) {
                 case LogType.UPDATE:
-
-                    // extract the resource manager id from the log record.
-                    byte resourceMgrId = parser.getResourceMgrId(logLocator);
-                    if (LOGGER.isLoggable(Level.FINE)) {
-                        LOGGER.fine(parser.getLogRecordForDisplay(logLocator));
+                    undoLSNSet = loserTxnTable.get(tempKeyTxnId);
+                    if (undoLSNSet == null) {
+                        TxnId txnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
+                                logRecordHelper.getDatasetId(currentLogLocator),
+                                logRecordHelper.getPKHashValue(currentLogLocator));
+                        undoLSNSet = new LinkedList<Long>();
+                        loserTxnTable.put(txnId, undoLSNSet);
                     }
-
-                    // look up the repository to get the resource manager
-                    IResourceManager resourceMgr = transactionProvider.getTransactionalResourceRepository()
-                            .getTransactionalResourceMgr(resourceMgrId);
-                    if (resourceMgr == null) {
-                        throw new ACIDException(txnContext, " unknown resource manager " + resourceMgrId);
-                    } else {
-                        byte actionType = parser.getLogActionType(logLocator);
-                        switch (actionType) {
-                            case LogActionType.REDO: // no need to do anything
-                                break;
-                            case LogActionType.UNDO: // undo the log record
-                                resourceMgr.undo(parser, logLocator);
-                                break;
-                            case LogActionType.REDO_UNDO: // undo log record
-                                resourceMgr.undo(parser, logLocator);
-                                break;
-                            default:
-                        }
+                    undoLSNSet.add(currentLogLocator.getLsn());
+                    if (IS_DEBUG_MODE) {
+                        updateLogCount++;
+                        System.out.println("" + Thread.currentThread().getId() + "======> update["
+                                + currentLogLocator.getLsn() + "]:" + tempKeyTxnId);
                     }
-                case LogType.CLR: // skip the CLRs as they are not undone
                     break;
+
                 case LogType.COMMIT:
-                    throw new ACIDException(txnContext, " cannot rollback commmitted transaction");
+                case LogType.ENTITY_COMMIT:
+                    undoLSNSet = loserTxnTable.get(tempKeyTxnId);
+                    if (undoLSNSet != null) {
+                        loserTxnTable.remove(tempKeyTxnId);
+                    }
+                    if (IS_DEBUG_MODE) {
+                        commitLogCount++;
+                        System.out.println("" + Thread.currentThread().getId() + "======> commit["
+                                + currentLogLocator.getLsn() + "]" + tempKeyTxnId);
+                    }
+                    break;
 
-            }
-
-            // follow the previous LSN pointer to get the previous log record
-            // written by the transaction
-            // If the return value is true, the logLocator, it indicates that
-            // the logLocator object has been
-            // appropriately set to the location of the next log record to be
-            // processed as part of the roll back
-            boolean moreLogs = parser.getPreviousLsnByTransaction(lsn, logLocator);
-            if (!moreLogs) {
-                // no more logs to process
-                break;
+                default:
+                    throw new ACIDException("Unsupported LogType: " + logType);
             }
         }
 
+        //undo loserTxn's effect
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(" undoing loser transaction's effect");
+        }
+
+        TxnId txnId = null;
+        Iterator<Entry<TxnId, List<Long>>> iter = loserTxnTable.entrySet().iterator();
+        byte resourceMgrId;
+        int undoCount = 0;
+        while (iter.hasNext()) {
+            //TODO 
+            //Sort the lsns in order to undo in one pass. 
+
+            Map.Entry<TxnId, List<Long>> loserTxn = (Map.Entry<TxnId, List<Long>>) iter.next();
+            txnId = loserTxn.getKey();
+
+            undoLSNSet = loserTxn.getValue();
+
+            for (long undoLSN : undoLSNSet) {
+                // here, all the log records are UPDATE type. So, we don't need to check the type again.
+
+                //read the corresponding log record to be undone.
+                logManager.readLog(undoLSN, currentLogLocator);
+
+                if (IS_DEBUG_MODE) {
+                    System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+                }
+
+                // extract the resource manager id from the log record.
+                resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
+                if (LOGGER.isLoggable(Level.FINE)) {
+                    LOGGER.fine(logRecordHelper.getLogRecordForDisplay(currentLogLocator));
+                }
+
+                // look up the repository to get the resource manager
+                IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
+                        .getTransactionalResourceMgr(resourceMgrId);
+
+                // register resourceMgr if it is not registered. 
+                if (resourceMgr == null) {
+                    resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
+                    txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
+                            resourceMgrId, resourceMgr);
+                }
+                resourceMgr.undo(logRecordHelper, currentLogLocator);
+
+                if (IS_DEBUG_MODE) {
+                    undoCount++;
+                }
+            }
+        }
+
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info(" undone loser transaction's effect");
+        }
+        if (IS_DEBUG_MODE) {
+            System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + commitLogCount + "/"
+                    + undoCount);
+        }
+    }
+}
+
+class TxnId {
+    public int jobId;
+    public int datasetId;
+    public int pkHashVal;
+
+    public TxnId(int jobId, int datasetId, int pkHashVal) {
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.pkHashVal = pkHashVal;
     }
 
+    public void setTxnId(int jobId, int datasetId, int pkHashVal) {
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.pkHashVal = pkHashVal;
+    }
+
+    public void setTxnId(TxnId txnId) {
+        this.jobId = txnId.jobId;
+        this.datasetId = txnId.datasetId;
+        this.pkHashVal = txnId.pkHashVal;
+    }
+
+    @Override
+    public String toString() {
+        return "[" + jobId + "," + datasetId + "," + pkHashVal + "]";
+    }
+
+    @Override
+    public int hashCode() {
+        return pkHashVal;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof TxnId)) {
+            return false;
+        }
+        TxnId txnId = (TxnId) o;
+
+        return (txnId.pkHashVal == pkHashVal && txnId.datasetId == datasetId && txnId.jobId == jobId);
+    }
 }
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/FieldsHashValueGenerator.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/FieldsHashValueGenerator.java
new file mode 100644
index 0000000..d4f15d9
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/FieldsHashValueGenerator.java
@@ -0,0 +1,36 @@
+/*
+ * 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.transaction;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class FieldsHashValueGenerator {
+    public static int computeFieldsHashValue(ITupleReference tuple, int[] fieldIndexes,
+            IBinaryHashFunction[] fieldHashFunctions) {
+        int h = 0;
+        for (int i = 0; i < fieldIndexes.length; i++) {
+            int primaryKeyFieldIdx = fieldIndexes[i];
+            int fh = fieldHashFunctions[i].hash(tuple.getFieldData(primaryKeyFieldIdx),
+                    tuple.getFieldStart(primaryKeyFieldIdx), tuple.getFieldLength(primaryKeyFieldIdx));
+            h = h * 31 + fh;
+            if (h < 0) {
+                h = h*(-1);
+            }
+        }
+        return h;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
index 200527f..f7715e8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/IResourceManager.java
@@ -22,6 +22,12 @@
  * Provides APIs for undo or redo of an operation on a resource.
  */
 public interface IResourceManager {
+    
+    public class ResourceType {
+        public static final byte LSM_BTREE = 1;
+        public static final byte LSM_RTREE = 2;
+        public static final byte LSM_INVERTED_INDEX = 3;
+    }
 
     /**
      * Returns the unique identifier for the resource manager.
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..3f55ac9 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,46 +40,50 @@
      * 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.
      * 
      * @param txnContext
      *            the transaction context associated with the transaction
+     * @param datasetId TODO
+     * @param PKHashVal TODO
      * @throws ACIDException
-     * @see transactionContext
+     * @see transactionContextimport edu.uci.ics.hyracks.api.job.JobId;
      * @see ACIDException
      */
-    public void commitTransaction(TransactionContext txnContext) throws ACIDException;
+    public void commitTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal) throws ACIDException;
 
     /**
      * Aborts a transaction.
      * 
      * @param txnContext
      *            the transaction context associated with the transaction
+     * @param datasetId TODO
+     * @param PKHashVal TODO
      * @throws ACIDException
      * @see transactionContext
      * @see ACIDException
      */
-    public void abortTransaction(TransactionContext txnContext) throws ACIDException;
+    public void abortTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal) throws ACIDException;
 
     /**
      * Indicates end of all activity for a transaction. In other words, all
@@ -88,36 +92,23 @@
      * 
      * @param txnContext
      *            the transaction context associated with the transaction
+     * @param datasetId TODO
+     * @param PKHashVal TODO
      * @param success
      *            indicates the success or failure. The transaction is committed
      *            or aborted accordingly.
      * @throws ACIDException
      */
-    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;
+    public void completedTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success) 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.
      * 
-     * @see TransactionProvider
+     * @see TransactionSubsystem
      * @return TransactionProvider
      */
-    public TransactionProvider getTransactionProvider();
+    public TransactionSubsystem getTransactionProvider();
 
 }
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/ITransactionSubsystemProvider.java
similarity index 70%
copy from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionIDFactory.java
copy to asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/ITransactionSubsystemProvider.java
index e5475f4..857d8ae 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/ITransactionSubsystemProvider.java
@@ -12,17 +12,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
-import java.util.concurrent.atomic.AtomicLong;
+import java.io.Serializable;
 
-/**
- * Represents a factory to generate unique transaction IDs.
- */
-public class TransactionIDFactory {
-    private static final AtomicLong ID = new AtomicLong();
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 
-    public static long generateTransactionId() {
-        return ID.incrementAndGet();
-    }
-}
\ No newline at end of file
+public interface ITransactionSubsystemProvider extends Serializable{
+    public TransactionSubsystem getTransactionSubsystem(IHyracksTaskContext ctx);
+}
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 71%
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..cac4f8e 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,19 @@
  */
 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());
+    }
+    
+    public static void initJobId(int id) {
+        Id.set(id);
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/MutableResourceId.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/MutableResourceId.java
new file mode 100644
index 0000000..5552930
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/MutableResourceId.java
@@ -0,0 +1,30 @@
+package edu.uci.ics.asterix.transaction.management.service.transaction;
+
+public class MutableResourceId{
+    long id;
+
+    public MutableResourceId(long id) {
+        this.id = id;
+    }
+
+    public void setId(long id) {
+        this.id = id;
+    }
+
+    public long getId() {
+        return id;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int)id;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if ((o == null) || !(o instanceof MutableResourceId)) {
+            return false;
+        }
+        return ((MutableResourceId) o).id == this.id;
+    }
+}
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..5b01edf 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
@@ -15,14 +15,23 @@
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
 import java.io.Serializable;
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.AbstractOperationCallback;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.IndexOperationTracker;
 import edu.uci.ics.asterix.transaction.management.resource.ICloseable;
 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.ITransactionManager.TransactionState;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 
 /**
  * Represents a holder object that contains all information related to a
@@ -36,7 +45,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,
@@ -44,14 +53,62 @@
     }
 
     private static final long serialVersionUID = -6105616785783310111L;
-    private TransactionProvider transactionProvider;
-    private long transactionID;
-    private LogicalLogLocator lastLogLocator;
+    private TransactionSubsystem transactionSubsystem;
+    private LogicalLogLocator firstLogLocator;//firstLSN of the Job
+    private LogicalLogLocator lastLogLocator;//lastLSN of the Job
     private TransactionState txnState;
     private long startWaitTime;
     private int status;
     private Set<ICloseable> resources = new HashSet<ICloseable>();
     private TransactionType transactionType = TransactionType.READ;
+    private JobId jobId;
+
+    // List of indexes on which operations were performed on behalf of this transaction.
+    private final List<ILSMIndex> indexes = new ArrayList<ILSMIndex>();
+
+    // List of operation callbacks corresponding to the operand indexes. In particular, needed to track
+    // the number of active operations contributed by this transaction.
+    private final List<AbstractOperationCallback> callbacks = new ArrayList<AbstractOperationCallback>();
+
+    public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
+        this.jobId = jobId;
+        this.transactionSubsystem = transactionSubsystem;
+        init();
+    }
+
+    private void init() throws ACIDException {
+        firstLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
+        lastLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
+        txnState = TransactionState.ACTIVE;
+        startWaitTime = INVALID_TIME;
+        status = ACTIVE_STATUS;
+    }
+
+    public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback) {
+        synchronized (indexes) {
+            indexes.add(index);
+            callbacks.add(callback);
+        }
+    }
+
+    public void updateLastLSNForIndexes(long lastLSN) {
+        synchronized (indexes) {
+            for (ILSMIndex index : indexes) {
+                ((IndexOperationTracker) index.getOperationTracker()).updateLastLSN(lastLSN);
+            }
+        }
+    }
+
+    public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException {
+        synchronized (indexes) {
+            for (int i = 0; i < indexes.size(); i++) {
+                ILSMIndex index = indexes.get(i);
+                IModificationOperationCallback modificationCallback = (IModificationOperationCallback) callbacks.get(i);
+                ((IndexOperationTracker) index.getOperationTracker()).completeOperation(LSMOperationType.MODIFICATION,
+                        null, modificationCallback);
+            }
+        }
+    }
 
     public void setTransactionType(TransactionType transactionType) {
         this.transactionType = transactionType;
@@ -65,29 +122,23 @@
         resources.add(resource);
     }
 
-    public TransactionContext(long transactionId, TransactionProvider transactionProvider) throws ACIDException {
-        this.transactionID = transactionId;
-        this.transactionProvider = transactionProvider;
-        init();
-    }
-
-    private void init() throws ACIDException {
-        lastLogLocator = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
-        txnState = TransactionState.ACTIVE;
-        startWaitTime = INVALID_TIME;
-        status = ACTIVE_STATUS;
+    public LogicalLogLocator getFirstLogLocator() {
+        return firstLogLocator;
     }
 
     public LogicalLogLocator getLastLogLocator() {
         return lastLogLocator;
     }
 
-    public void setLastLSN(LogicalLogLocator lastLogLocator) {
-        this.lastLogLocator = lastLogLocator;
+    public void setLastLSN(long lsn) {
+        if (firstLogLocator.getLsn() == -1) {
+            firstLogLocator.setLsn(lsn);
+        }
+        lastLogLocator.setLsn(lsn);
     }
 
-    public long getTransactionID() {
-        return transactionID;
+    public JobId getJobId() {
+        return jobId;
     }
 
     public void setStartWaitTime(long time) {
@@ -119,5 +170,14 @@
             closeable.close(this);
         }
     }
+    
+    @Override
+    public int hashCode() {
+        return jobId.getId();
+    }
 
+    @Override
+    public boolean equals(Object o) {
+        return (o == this);
+    }
 }
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..3d25e54 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
@@ -27,7 +27,7 @@
 
     public static class LogManagerConstants {
         public static final String LOG_CONF_DIR = "log_conf";
-        public static final String LOG_CONF_FILE = "log.conf";
+        public static final String LOG_CONF_FILE = "log.properties";
         public static final String ASTERIX_CONF_DIR = "src/main/resources";
         public static final String ASTERIX_CONF_FILE = "test.properties";
         public static final String DEFAULT_LOG_DIR = "asterix_logs";
@@ -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..d3294d0 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
@@ -16,12 +16,13 @@
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * An implementation of the @see ITransactionManager interface that provides
@@ -29,15 +30,17 @@
  */
 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 final TransactionSubsystem transactionProvider;
+    private Map<JobId, TransactionContext> transactionContextRepository = new HashMap<JobId, TransactionContext>();
+    private AtomicInteger maxJobId = new AtomicInteger(0);
 
-    public TransactionManager(TransactionProvider provider) {
+    public TransactionManager(TransactionSubsystem provider) {
         this.transactionProvider = provider;
     }
 
     @Override
-    public void abortTransaction(TransactionContext txnContext) throws ACIDException {
+    public void abortTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+            throws ACIDException {
         synchronized (txnContext) {
             if (txnContext.getTxnState().equals(TransactionState.ABORTED)) {
                 return;
@@ -50,88 +53,107 @@
                 if (LOGGER.isLoggable(Level.SEVERE)) {
                     LOGGER.severe(msg);
                 }
+                ae.printStackTrace();
                 throw new Error(msg);
             } 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 {
+        setMaxJobId(jobId.getId());
+        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 {
+        setMaxJobId(jobId.getId());
         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;
         }
     }
 
     @Override
-    public void commitTransaction(TransactionContext txnContext) throws ACIDException {
+    public void commitTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal)
+            throws ACIDException {
         synchronized (txnContext) {
             if ((txnContext.getTxnState().equals(TransactionState.COMMITTED))) {
                 return;
             }
 
+            //There is either job-level commit or entity-level commit.
+            //The job-level commit will have -1 value both for datasetId and PKHashVal.
+
+            //for entity-level commit
+            if (PKHashVal != -1) {
+                transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext, true);
+                /*****************************
+                try {
+                    //decrease the transaction reference count on index
+                    txnContext.decreaseActiveTransactionCountOnIndexes();
+                } catch (HyracksDataException e) {
+                    throw new ACIDException("failed to complete index operation", e);
+                }
+                *****************************/
+                return;
+            }
+
+            //for job-level commit
             try {
-                if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) { // conditionally
-                    // write
-                    // commit
-                    // log
-                    // record
-                    transactionProvider.getLogManager().log(txnContext.getLastLogLocator(), txnContext, (byte) (-1), 0,
-                            LogType.COMMIT, LogActionType.NO_OP, 0, null, null);
+                if (txnContext.getTransactionType().equals(TransactionContext.TransactionType.READ_WRITE)) {
+                    transactionProvider.getLogManager().log(LogType.COMMIT, txnContext, -1, -1, -1, (byte) 0, 0, null,
+                            null, txnContext.getLastLogLocator());
                 }
             } 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 {
+    public void completedTransaction(TransactionContext txnContext, DatasetId datasetId, int PKHashVal, boolean success)
+            throws ACIDException {
         if (!success) {
-            abortTransaction(txnContext);
+            abortTransaction(txnContext, datasetId, PKHashVal);
         } else {
-            commitTransaction(txnContext);
+            commitTransaction(txnContext, datasetId, PKHashVal);
         }
     }
 
     @Override
-    public TransactionProvider getTransactionProvider() {
+    public TransactionSubsystem getTransactionProvider() {
         return transactionProvider;
     }
-
+    
+    public void setMaxJobId(int jobId) {
+        maxJobId.set(Math.max(maxJobId.get(), jobId));
+    }
+    
+    public int getMaxJobId() {
+        return maxJobId.get();
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
similarity index 62%
rename from asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java
rename to asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index ef843f4..c83b7bc 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -15,12 +15,14 @@
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceRepository;
+import edu.uci.ics.asterix.transaction.management.resource.TransactionalResourceManagerRepository;
 import edu.uci.ics.asterix.transaction.management.service.locking.ILockManager;
 import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogManager;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLoggerRepository;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.TreeLoggerRepository;
+import edu.uci.ics.asterix.transaction.management.service.recovery.CheckpointThread;
+import edu.uci.ics.asterix.transaction.management.service.recovery.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
 import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
 
@@ -28,23 +30,33 @@
  * Provider for all the sub-systems (transaction/lock/log/recovery) managers.
  * Users of transaction sub-systems must obtain them from the provider.
  */
-public class TransactionProvider {
+public class TransactionSubsystem {
     private final String id;
     private final ILogManager logManager;
     private final ILockManager lockManager;
     private final ITransactionManager transactionManager;
     private final IRecoveryManager recoveryManager;
-    private final TransactionalResourceRepository resourceRepository;
-    private final TreeLoggerRepository loggerRepository;
+    private final TransactionalResourceManagerRepository resourceRepository;
+    private final IndexLoggerRepository loggerRepository;
+    private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
+    private final CheckpointThread checkpointThread;
 
-    public TransactionProvider(String id) throws ACIDException {
+    public TransactionSubsystem(String id, IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider)
+            throws ACIDException {
         this.id = id;
         this.transactionManager = new TransactionManager(this);
         this.logManager = new LogManager(this);
         this.lockManager = new LockManager(this);
         this.recoveryManager = new RecoveryManager(this);
-        this.loggerRepository = new TreeLoggerRepository(this);
-        this.resourceRepository = new TransactionalResourceRepository();
+        this.loggerRepository = new IndexLoggerRepository(this);
+        this.resourceRepository = new TransactionalResourceManagerRepository();
+        this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
+        if (asterixAppRuntimeContextProvider != null) {
+	        this.checkpointThread = new CheckpointThread(recoveryManager,
+	                asterixAppRuntimeContextProvider.getIndexLifecycleManager(), 0);
+        } else {
+        	this.checkpointThread = null;
+        }
     }
 
     public ILogManager getLogManager() {
@@ -62,15 +74,19 @@
     public IRecoveryManager getRecoveryManager() {
         return recoveryManager;
     }
-    
-    public TransactionalResourceRepository getTransactionalResourceRepository() {
+
+    public TransactionalResourceManagerRepository getTransactionalResourceRepository() {
         return resourceRepository;
     }
-    
-    public TreeLoggerRepository getTreeLoggerRepository() {
+
+    public IndexLoggerRepository getTreeLoggerRepository() {
         return loggerRepository;
     }
 
+    public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider() {
+        return asterixAppRuntimeContextProvider;
+    }
+
     public String getId() {
         return id;
     }
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
index 3e95d6a..c9d01a0 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
@@ -14,12 +14,12 @@
  */
 package edu.uci.ics.asterix.transaction.management.logging;
 
-import java.util.Map;
 import java.util.Random;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.service.logging.IBuffer;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 
@@ -29,7 +29,7 @@
     private static long count = 0;
 
     public void log(TransactionContext context, LogicalLogLocator wMemLSN, int length,
-            Map<Object, Object> loggerArguments) throws ACIDException {
+            ReusableLogContentObject reusableLogContentObject) throws ACIDException {
 
         byte[] logContent = getRandomBytes(length);
         try {
@@ -66,12 +66,12 @@
         return averageContentCreationTime;
     }
 
-    public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
 
-    public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
index 3a288a2..54ec036 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
@@ -18,20 +18,20 @@
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class RecoverySimulator {
 
     private static IRecoveryManager recoveryManager;
 
-    public static IRecoveryManager.SystemState startRecovery() throws IOException, ACIDException {
-        return recoveryManager.startRecovery(true);
+    public static void startRecovery() throws IOException, ACIDException {
+        recoveryManager.startRecovery(true);
     }
 
     public static void main(String args[]) throws IOException, ACIDException {
         String id = "nc1";
         try {
-            TransactionProvider factory = new TransactionProvider(id);
+            TransactionSubsystem factory = new TransactionSubsystem(id, null);
             IRecoveryManager recoveryManager = factory.getRecoveryManager();
             recoveryManager.startRecovery(true);
         } catch (ACIDException acide) {
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..dcd644d 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,16 +27,18 @@
 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.IResourceManager.ResourceType;
+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;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class TransactionWorkloadSimulator {
 
     public static ILogManager logManager;
     public static ILockManager lockManager;
-    TransactionProvider provider;
+    TransactionSubsystem provider;
 
     public static WorkloadProperties workload;
     Transaction[] transactions;
@@ -48,7 +49,7 @@
     }
 
     public void beginWorkload() throws ACIDException {
-        provider = new TransactionProvider("nc1");
+        provider = new TransactionSubsystem("nc1", null);
         logManager = provider.getLogManager();
         lockManager = provider.getLockManager();
         provider.getTransactionalResourceRepository().registerTransactionalResourceManager(DummyResourceMgr.id,
@@ -67,7 +68,7 @@
         }
 
         for (int i = 0; i < workload.numActiveThreads; i++) {
-            provider.getTransactionManager().commitTransaction(transactions[i].getContext());
+            provider.getTransactionManager().commitTransaction(transactions[i].getContext(), new DatasetId(-1), -1);
         }
 
         long endTime = System.nanoTime();
@@ -97,9 +98,9 @@
 class SingleTransactionContextFactory {
     private static TransactionContext context;
 
-    public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
+    public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
         if (context == null) {
-            context = new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+            context = new TransactionContext(JobIdFactory.generateJobId(), provider);
         }
         return context;
     }
@@ -107,8 +108,8 @@
 
 class MultipleTransactionContextFactory {
 
-    public static TransactionContext getContext(TransactionProvider provider) throws ACIDException {
-        return new TransactionContext(TransactionIDFactory.generateTransactionId(), provider);
+    public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
+        return new TransactionContext(JobIdFactory.generateJobId(), provider);
     }
 }
 
@@ -121,12 +122,14 @@
     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 TransactionSubsystem transactionProvider;
     private ILogManager logManager;
+    private DatasetId tempDatasetId = new DatasetId(-1);
 
-    public Transaction(TransactionProvider provider, String name, boolean singleTransaction) throws ACIDException {
+    public Transaction(TransactionSubsystem provider, String name, boolean singleTransaction) throws ACIDException {
         this.name = name;
         this.transactionProvider = provider;
         if (singleTransaction) {
@@ -157,12 +160,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,21 +177,18 @@
                 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;
-                }
-                TransactionWorkloadSimulator.logManager.log(memLSN, context, ResourceMgrInfo.BTreeResourceMgrId,
-                        pageId, logType, logActionType, logSize, logger, null);
+                tempDatasetId.setId(resourceID);
+                TransactionWorkloadSimulator.lockManager.lock(tempDatasetId, -1, lockMode, context);
+                TransactionWorkloadSimulator.logManager.log(logType, context, resourceID,
+                        -1, resourceID, ResourceType.LSM_BTREE, logSize, null, logger, memLSN);
                 retry = false;
                 Thread.currentThread().sleep(TransactionWorkloadSimulator.workload.thinkTime);
                 logCount.incrementAndGet();
                 logByteCount.addAndGet(logSize
-                        + TransactionWorkloadSimulator.logManager.getLogManagerProperties().getLogHeaderSize()
-                        + TransactionWorkloadSimulator.logManager.getLogManagerProperties().getLogChecksumSize());
+                        + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogHeaderSize(logType)
+                        + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogChecksumSize());
                 myLogCount++;
             }
         } catch (ACIDException acide) {
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
index 16c5d6b..54bb0b8 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
@@ -14,11 +14,11 @@
  */
 package edu.uci.ics.asterix.transaction.management.test;
 
-import java.util.Map;
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.logging.IResource;
 import edu.uci.ics.asterix.transaction.management.service.logging.ILogger;
+import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger.ReusableLogContentObject;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogicalLogLocator;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 
@@ -39,14 +39,14 @@
     }
 
     @Override
-    public void preLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+    public void preLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
 
     @Override
-    public void log(TransactionContext context, final LogicalLogLocator memLSN, int logRecordSize,
-            Map<Object, Object> loggerArguments) throws ACIDException {
+    public void log(TransactionContext context, final LogicalLogLocator memLSN, int logContentSize,
+            ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         byte[] buffer = memLSN.getBuffer().getArray();
         byte[] content = logRecordContent.getBytes();
         for (int i = 0; i < resource.getId().length; i++) {
@@ -58,7 +58,7 @@
     }
 
     @Override
-    public void postLog(TransactionContext context, Map<Object, Object> loggerArguments) throws ACIDException {
+    public void postLog(TransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
         // TODO Auto-generated method stub
 
     }
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
index 899dbde..ad1238a 100644
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
+++ b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
@@ -28,13 +28,13 @@
 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.logging.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class LogRecordReader {
 
     ILogManager logManager;
 
-    public LogRecordReader(TransactionProvider factory) throws ACIDException {
+    public LogRecordReader(TransactionSubsystem factory) throws ACIDException {
         logManager = factory.getLogManager();
     }
 
@@ -51,12 +51,12 @@
                 return true;
             }
         });
-        LogicalLogLocator memLSN = LogUtil.getDummyLogicalLogLocator(logManager);
+        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
         int logCount = 0;
         while (true) {
-            boolean logValidity = logCursor.next(memLSN);
+            boolean logValidity = logCursor.next(currentLogLocator);
             if (logValidity) {
-                System.out.println(++logCount + parser.getLogRecordForDisplay(memLSN));
+                System.out.println(++logCount + parser.getLogRecordForDisplay(currentLogLocator));
             } else {
                 break;
             }
@@ -64,7 +64,8 @@
     }
 
     public void readLogRecord(long lsnValue) throws IOException, ACIDException {
-        LogicalLogLocator memLSN = logManager.readLog(new PhysicalLogLocator(lsnValue, logManager));
+        LogicalLogLocator memLSN = null;
+        logManager.readLog(lsnValue, memLSN);
         System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(memLSN));
     }
 
@@ -72,16 +73,9 @@
      * @param args
      */
     public static void main(String[] args) throws ACIDException, Exception {
-        long lsnValue = 10747454;
-        String id = "nc1";
-        String logDir = "/home/raman/research/work/hyracks-branches/svn/trunk/hyracks/asterix_logs/";
-        Properties props = new Properties();
-        props.setProperty(LogManagerProperties.LOG_DIR_KEY, logDir + "/" + id);
-        LogManagerProperties logProps = new LogManagerProperties(props);
-        LogManager logManager = new LogManager(null, logProps);
+        LogManager logManager = new LogManager(null, "nc1");
         LogRecordReader logReader = new LogRecordReader(logManager);
         logReader.readLogs(0);
-        //   logReader.readLogRecord(1703620);
     }
 
 }
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..092f2ed 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
@@ -19,21 +19,22 @@
 
 import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
 import edu.uci.ics.asterix.transaction.management.logging.IResource;
-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.ILogger;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
 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.recovery.IRecoveryManager;
 import edu.uci.ics.asterix.transaction.management.service.recovery.IRecoveryManager.SystemState;
+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.IResourceManager.ResourceType;
 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.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;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 
 public class TransactionSimulator {
 
@@ -45,11 +46,11 @@
     private ILogger logger;
     private IResource resource;
     private LogicalLogLocator memLSN;
-    private TransactionProvider transactionProvider;
+    private TransactionSubsystem transactionProvider;
 
     public TransactionSimulator(IResource resource, IResourceManager resourceMgr) throws ACIDException {
         String id = "nc1";
-        transactionProvider = new TransactionProvider(id);
+        transactionProvider = new TransactionSubsystem(id, null);
         transactionManager = transactionProvider.getTransactionManager();
         logManager = transactionProvider.getLogManager();
         lockManager = transactionProvider.getLockManager();
@@ -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)
@@ -77,15 +78,13 @@
             case INCREMENT:
                 finalValue = currentValue + 1;
                 int logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
-                logManager.log(memLSN, txnContext, FileResourceManager.id, 0, LogType.UPDATE, LogActionType.REDO_UNDO,
-                        logRecordLength, logger, null);
+                logManager.log(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
                 ((FileResource) resource).increment();
                 break;
             case DECREMENT:
                 finalValue = currentValue - 1;
                 logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
-                logManager.log(memLSN, txnContext, FileResourceManager.id, 0, LogType.UPDATE, LogActionType.REDO_UNDO,
-                        logRecordLength, logger, null);
+                logManager.log(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
                 ((FileResource) resource).decrement();
                 break;
         }
@@ -93,13 +92,12 @@
     }
 
     public void commitTransaction(TransactionContext context) throws ACIDException {
-        transactionManager.commitTransaction(context);
+        transactionManager.commitTransaction(context, new DatasetId(-1), -1);
     }
 
-    public SystemState recover() throws ACIDException, IOException {
-        SystemState state = recoveryManager.startRecovery(true);
+    public void recover() throws ACIDException, IOException {
+        recoveryManager.startRecovery(true);
         ((FileResource) resource).sync();
-        return state;
     }
 
     /**
@@ -126,8 +124,7 @@
         }
 
         int finalExpectedValue = existingValue + schedule.getDeltaChange();
-        SystemState state = txnSimulator.recover();
-        System.out.println(" State is " + state);
+        txnSimulator.recover();
         boolean isCorrect = ((FileResource) resource).checkIfValueInSync(finalExpectedValue);
         System.out.println(" Did recovery happen correctly " + isCorrect);
     }
diff --git a/pom.xml b/pom.xml
index a8e7220..b1b4161 100644
--- a/pom.xml
+++ b/pom.xml
@@ -79,7 +79,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-test-framework</module>