Merge branch 'master' into eugenia/black_cherry_stable
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
index ba940d6..693a19e 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -39,6 +39,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -387,6 +388,13 @@
     public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg) {
         throw new UnsupportedOperationException();
     }
+    
+    @Override
+	public ILogicalOperator visitExternalDataAccessByRIDOperator(
+			ExternalDataAccessByRIDOperator op, ILogicalOperator arg)
+			throws AlgebricksException {
+    	throw new UnsupportedOperationException();
+	}
 
     @Override
     public ILogicalOperator visitUnnestOperator(UnnestOperator op, ILogicalOperator arg) throws AlgebricksException {
@@ -438,4 +446,5 @@
     public Map<LogicalVariable, LogicalVariable> getVariableMapping() {
         return outVarMapping;
     }
+
 }
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
index 269526b..6dd11bd 100644
--- 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
@@ -19,11 +19,11 @@
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILogManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
 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;
@@ -35,16 +35,18 @@
 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 ILogManager logMgr;
     private final JobId jobId;
-    private final DatasetId datasetId;
+    private final int datasetId;
     private final int[] primaryKeyFields;
     private final boolean isWriteTransaction;
-    private final long[] longHashes; 
+    private final long[] longHashes;
+    private final LogRecord logRecord;
 
     private ITransactionContext transactionContext;
     private RecordDescriptor inputRecordDesc;
@@ -54,23 +56,24 @@
     public CommitRuntime(IHyracksTaskContext ctx, JobId jobId, int datasetId, int[] primaryKeyFields,
             boolean isWriteTransaction) {
         this.hyracksTaskCtx = ctx;
-        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext()
-                .getApplicationObject();
+        IAsterixAppRuntimeContext runtimeCtx = (IAsterixAppRuntimeContext) ctx.getJobletContext()
+                .getApplicationContext().getApplicationObject();
         this.transactionManager = runtimeCtx.getTransactionSubsystem().getTransactionManager();
+        this.logMgr = runtimeCtx.getTransactionSubsystem().getLogManager();
         this.jobId = jobId;
-        this.datasetId = new DatasetId(datasetId);
+        this.datasetId = datasetId;
         this.primaryKeyFields = primaryKeyFields;
         this.frameTupleReference = new FrameTupleReference();
         this.isWriteTransaction = isWriteTransaction;
-        this.longHashes= new long[2];
+        this.longHashes = new long[2];
+        this.logRecord = new LogRecord();
     }
 
     @Override
     public void open() throws HyracksDataException {
         try {
             transactionContext = transactionManager.getTransactionContext(jobId);
-            transactionContext.setTransactionType(isWriteTransaction ? TransactionType.READ_WRITE
-                    : TransactionType.READ);
+            transactionContext.setWriteTxn(isWriteTransaction);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
@@ -84,17 +87,15 @@
         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);
-            }
+            logRecord.formEntityCommitLogRecord(transactionContext, datasetId, pkHash, frameTupleReference,
+                    primaryKeyFields);
+            logMgr.log(logRecord);
         }
     }
-    
+
     private int computePrimaryKeyHashValue(ITupleReference tuple, int[] primaryKeyFields) {
         MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
-        return Math.abs((int) longHashes[0]); 
+        return Math.abs((int) longHashes[0]);
     }
 
     @Override
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
new file mode 100644
index 0000000..447555e
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
@@ -0,0 +1,110 @@
+package edu.uci.ics.asterix.algebra.operators.physical;
+
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.types.ARecordType;
+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.core.algebra.base.IHyracksJobBuilder;
+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.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+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.metadata.IDataSourcePropertiesProvider;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractScanPOperator;
+import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
+
+public class ExternalDataAccessByRIDPOperator extends AbstractScanPOperator{
+
+    private AqlSourceId datasetId;
+    private Dataset dataset;
+    private ARecordType recordType;
+    private Index secondaryIndex;
+	public ExternalDataAccessByRIDPOperator(AqlSourceId datasetId, Dataset dataset, ARecordType recordType,Index secondaryIndex)
+    {
+    	this.datasetId = datasetId;
+    	this.dataset = dataset;
+    	this.recordType = recordType;
+    	this.secondaryIndex = secondaryIndex;
+    }
+    
+	public Dataset getDataset() {
+		return dataset;
+	}
+
+	public void setDataset(Dataset dataset) {
+		this.dataset = dataset;
+	}
+
+	public ARecordType getRecordType() {
+		return recordType;
+	}
+
+	public void setRecordType(ARecordType recordType) {
+		this.recordType = recordType;
+	}
+	
+	public AqlSourceId getDatasetId() {
+		return datasetId;
+	}
+
+	public void setDatasetId(AqlSourceId datasetId) {
+		this.datasetId = datasetId;
+	}
+	
+	@Override
+	public PhysicalOperatorTag getOperatorTag() {
+		return PhysicalOperatorTag.EXTERNAL_ACCESS_BY_RID;
+	}
+
+	@Override
+	public void computeDeliveredProperties(ILogicalOperator op,
+			IOptimizationContext context) throws AlgebricksException {
+		AqlDataSource ds = new AqlDataSource(datasetId, dataset, recordType);
+        IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
+        AbstractScanOperator as = (AbstractScanOperator) op;
+        deliveredProperties = dspp.computePropertiesVector(as.getVariables());
+	}
+
+	@Override
+	public void contributeRuntimeOperator(IHyracksJobBuilder builder,
+			JobGenContext context, ILogicalOperator op,
+			IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
+			IOperatorSchema outerPlanSchema) throws AlgebricksException {
+		ExternalDataAccessByRIDOperator edabro = (ExternalDataAccessByRIDOperator) op;
+        ILogicalExpression expr = edabro.getExpressionRef().getValue();
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            throw new IllegalStateException();
+        }
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+        if (!funcIdent.equals(AsterixBuiltinFunctions.EXTERNAL_ACCESS_BY_RID)) {
+            return;
+        }
+        AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalAccessByRID = metadataProvider.buildExternalDataAccesByRIDRuntime(
+        		builder.getJobSpec(), dataset,secondaryIndex);
+        builder.contributeHyracksOperator(edabro, externalAccessByRID.first);
+        builder.contributeAlgebricksPartitionConstraint(externalAccessByRID.first, externalAccessByRID.second);
+        ILogicalOperator srcExchange = edabro.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(srcExchange, 0, edabro, 0);
+	}
+	
+	@Override
+	public boolean isMicroOperator() {
+		return false;
+	}
+
+}
\ No newline at end of file
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 23f1d56..56fc0e7 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
@@ -20,6 +20,7 @@
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
@@ -38,6 +39,7 @@
 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;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -217,18 +219,18 @@
             if (!isPartitioned) {
                 dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
                         new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate());
             } else {
                 dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
                         new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate());
             }
             LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
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 ab0fd79..95d29e3 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
@@ -22,8 +22,12 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
+import edu.uci.ics.asterix.algebra.operators.physical.ExternalDataAccessByRIDPOperator;
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
@@ -34,6 +38,7 @@
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 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.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -52,6 +57,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
@@ -60,290 +66,384 @@
  * Static helper functions for rewriting plans using indexes.
  */
 public class AccessMethodUtils {
-    public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target)
-            throws IOException {
-        ARecordType recordType = (ARecordType) itemType;
-        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-        for (String partitioningKey : partitioningKeys) {
-            target.add(recordType.getFieldType(partitioningKey));
-        }
-        target.add(itemType);
-    }
+	public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target)
+			throws IOException {
+		ARecordType recordType = (ARecordType) itemType;
+		List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+		for (String partitioningKey : partitioningKeys) {
+			target.add(recordType.getFieldType(partitioningKey));
+		}
+		target.add(itemType);
+	}
 
-    public static ConstantExpression createStringConstant(String str) {
-        return new ConstantExpression(new AsterixConstantValue(new AString(str)));
-    }
+	public static void appendExternalRecTypes(Dataset dataset, IAType itemType, List<Object> target){
+		target.add(itemType);
+	}
 
-    public static ConstantExpression createInt32Constant(int i) {
-        return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
-    }
+	public static void appendExternalRecPrimaryKey(Dataset dataset, List<Object> target){
+		target.add(BuiltinType.ASTRING);
+		target.add(BuiltinType.AINT64);
+		if(DatasetUtils.getExternalRIDSize(dataset) == 3)
+		{
+			target.add(BuiltinType.AINT32);
+		}
+	}
 
-    public static ConstantExpression createBooleanConstant(boolean b) {
-        if (b) {
-            return new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE));
-        } else {
-            return new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
-        }
-    }
+	public static ConstantExpression createStringConstant(String str) {
+		return new ConstantExpression(new AsterixConstantValue(new AString(str)));
+	}
 
-    public static String getStringConstant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
-        return ((AString) obj).getStringValue();
-    }
+	public static ConstantExpression createInt32Constant(int i) {
+		return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
+	}
 
-    public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
-        return ((AInt32) obj).getIntegerValue();
-    }
+	public static ConstantExpression createBooleanConstant(boolean b) {
+		if (b) {
+			return new ConstantExpression(new AsterixConstantValue(ABoolean.TRUE));
+		} else {
+			return new ConstantExpression(new AsterixConstantValue(ABoolean.FALSE));
+		}
+	}
 
-    public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
-        return ((ABoolean) obj).getBoolean();
-    }
+	public static String getStringConstant(Mutable<ILogicalExpression> expr) {
+		IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+		return ((AString) obj).getStringValue();
+	}
 
-    public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
-            AccessMethodAnalysisContext analysisCtx) {
-        IAlgebricksConstantValue constFilterVal = null;
-        LogicalVariable fieldVar = null;
-        ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
-        ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
-        // One of the args must be a constant, and the other arg must be a variable.
-        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
-                && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-            ConstantExpression constExpr = (ConstantExpression) arg1;
-            constFilterVal = constExpr.getValue();
-            VariableReferenceExpression varExpr = (VariableReferenceExpression) arg2;
-            fieldVar = varExpr.getVariableReference();
-        } else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
-                && arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-            ConstantExpression constExpr = (ConstantExpression) arg2;
-            constFilterVal = constExpr.getValue();
-            VariableReferenceExpression varExpr = (VariableReferenceExpression) arg1;
-            fieldVar = varExpr.getVariableReference();
-        } else {
-            return false;
-        }
-        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
-        return true;
-    }
+	public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
+		IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+		return ((AInt32) obj).getIntegerValue();
+	}
 
-    public static boolean analyzeFuncExprArgsForTwoVars(AbstractFunctionCallExpression funcExpr,
-            AccessMethodAnalysisContext analysisCtx) {
-        LogicalVariable fieldVar1 = null;
-        LogicalVariable fieldVar2 = null;
-        ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
-        ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
-        if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
-                && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-            fieldVar1 = ((VariableReferenceExpression) arg1).getVariableReference();
-            fieldVar2 = ((VariableReferenceExpression) arg2).getVariableReference();
-        } else {
-            return false;
-        }
-        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
-                new LogicalVariable[] { fieldVar1, fieldVar2 }, null));
-        return true;
-    }
+	public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
+		IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+		return ((ABoolean) obj).getBoolean();
+	}
 
-    public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
-        switch (index.getIndexType()) {
-            case BTREE:
-            case SINGLE_PARTITION_WORD_INVIX:
-            case SINGLE_PARTITION_NGRAM_INVIX: 
-            case LENGTH_PARTITIONED_WORD_INVIX:
-            case LENGTH_PARTITIONED_NGRAM_INVIX: {
-                return index.getKeyFieldNames().size();
-            }
-            case RTREE: {
-                Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames().get(0),
-                        recordType);
-                IAType keyType = keyPairType.first;
-                int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
-                return numDimensions * 2;
-            }
-            default: {
-                throw new AlgebricksException("Unknown index kind: " + index.getIndexType());
-            }
-        }
-    }
+	public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
+			AccessMethodAnalysisContext analysisCtx) {
+		IAlgebricksConstantValue constFilterVal = null;
+		LogicalVariable fieldVar = null;
+		ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+		ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+		// One of the args must be a constant, and the other arg must be a variable.
+		if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
+				&& arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+			ConstantExpression constExpr = (ConstantExpression) arg1;
+			constFilterVal = constExpr.getValue();
+			VariableReferenceExpression varExpr = (VariableReferenceExpression) arg2;
+			fieldVar = varExpr.getVariableReference();
+		} else if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
+				&& arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+			ConstantExpression constExpr = (ConstantExpression) arg2;
+			constFilterVal = constExpr.getValue();
+			VariableReferenceExpression varExpr = (VariableReferenceExpression) arg1;
+			fieldVar = varExpr.getVariableReference();
+		} else {
+			return false;
+		}
+		analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
+		return true;
+	}
 
-    /**
-     * Appends the types of the fields produced by the given secondary index to dest.
-     */
-    public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, Index index,
-            boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
-        if (!primaryKeysOnly) {
-            switch (index.getIndexType()) {
-                case BTREE:
-                case SINGLE_PARTITION_WORD_INVIX:
-                case SINGLE_PARTITION_NGRAM_INVIX: {
-                    for (String sk : index.getKeyFieldNames()) {
-                        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(sk, recordType);
-                        dest.add(keyPairType.first);
-                    }
-                    break;
-                }
-                case RTREE: {
-                    Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames()
-                            .get(0), recordType);
-                    IAType keyType = keyPairType.first;
-                    IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
-                    int numKeys = getNumSecondaryKeys(index, recordType);
-                    for (int i = 0; i < numKeys; i++) {
-                        dest.add(nestedKeyType);
-                    }
-                    break;
-                }
-            }
-        }
-        // Primary keys.
-        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
-        for (String partitioningKey : partitioningKeys) {
-            try {
-                dest.add(recordType.getFieldType(partitioningKey));
-            } catch (IOException e) {
-                throw new AlgebricksException(e);
-            }
-        }
-    }
+	public static boolean analyzeFuncExprArgsForTwoVars(AbstractFunctionCallExpression funcExpr,
+			AccessMethodAnalysisContext analysisCtx) {
+		LogicalVariable fieldVar1 = null;
+		LogicalVariable fieldVar2 = null;
+		ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+		ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+		if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
+				&& arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+			fieldVar1 = ((VariableReferenceExpression) arg1).getVariableReference();
+			fieldVar2 = ((VariableReferenceExpression) arg2).getVariableReference();
+		} else {
+			return false;
+		}
+		analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
+				new LogicalVariable[] { fieldVar1, fieldVar2 }, null));
+		return true;
+	}
 
-    public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType, Index index,
-            boolean primaryKeysOnly, IOptimizationContext context, List<LogicalVariable> dest)
-            throws AlgebricksException {
-        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-        int numSecondaryKeys = getNumSecondaryKeys(index, recordType);
-        int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
-        for (int i = 0; i < numVars; i++) {
-            dest.add(context.newVar());
-        }
-    }
+	public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
+		switch (index.getIndexType()) {
+		case BTREE:
+		case SINGLE_PARTITION_WORD_INVIX:
+		case SINGLE_PARTITION_NGRAM_INVIX: 
+		case LENGTH_PARTITIONED_WORD_INVIX:
+		case LENGTH_PARTITIONED_NGRAM_INVIX: {
+			return index.getKeyFieldNames().size();
+		}
+		case RTREE: {
+			Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames().get(0),
+					recordType);
+			IAType keyType = keyPairType.first;
+			int numDimensions = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
+			return numDimensions * 2;
+		}
+		default: {
+			throw new AlgebricksException("Unknown index kind: " + index.getIndexType());
+		}
+		}
+	}
 
-    public static List<LogicalVariable> getPrimaryKeyVarsFromUnnestMap(Dataset dataset, ILogicalOperator unnestMapOp) {
-        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-        List<LogicalVariable> primaryKeyVars = new ArrayList<LogicalVariable>();
-        List<LogicalVariable> sourceVars = ((UnnestMapOperator) unnestMapOp).getVariables();
-        // Assumes the primary keys are located at the end.
-        int start = sourceVars.size() - numPrimaryKeys;
-        int stop = sourceVars.size();
-        for (int i = start; i < stop; i++) {
-            primaryKeyVars.add(sourceVars.get(i));
-        }
-        return primaryKeyVars;
-    }
+	/**
+	 * Appends the types of the fields produced by the given secondary index to dest.
+	 */
+	public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, Index index,
+			boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
+		if (!primaryKeysOnly) {
+			switch (index.getIndexType()) {
+			case BTREE:
+			case SINGLE_PARTITION_WORD_INVIX:
+			case SINGLE_PARTITION_NGRAM_INVIX: {
+				for (String sk : index.getKeyFieldNames()) {
+					Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(sk, recordType);
+					dest.add(keyPairType.first);
+				}
+				break;
+			}
+			case RTREE: {
+				Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames()
+						.get(0), recordType);
+				IAType keyType = keyPairType.first;
+				IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+				int numKeys = getNumSecondaryKeys(index, recordType);
+				for (int i = 0; i < numKeys; i++) {
+					dest.add(nestedKeyType);
+				}
+				break;
+			}
+			}
+		}
+		// Primary keys.
+		if(dataset.getDatasetType() == DatasetType.EXTERNAL)
+		{
+			//add primary keys
+			appendExternalRecPrimaryKey(dataset, dest);
+		}
+		else
+		{
+			List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+			for (String partitioningKey : partitioningKeys) {
+				try {
+					dest.add(recordType.getFieldType(partitioningKey));
+				} catch (IOException e) {
+					throw new AlgebricksException(e);
+				}
+			}
+		}
+	}
 
-    /**
-     * Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query then this method returns
-     * the a ConstantExpression from the first constant value in the optimizable function expression.
-     * If we are optimizing a join, then this method returns the VariableReferenceExpression that should feed the secondary index probe.
-     */
-    public static ILogicalExpression createSearchKeyExpr(IOptimizableFuncExpr optFuncExpr,
-            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree) {
-        if (probeSubTree == null) {
-            // We are optimizing a selection query. Search key is a constant.
-            return new ConstantExpression(optFuncExpr.getConstantVal(0));
-        } else {
-            // We are optimizing a join query. Determine which variable feeds the secondary index. 
-            if (optFuncExpr.getOperatorSubTree(0) == null || optFuncExpr.getOperatorSubTree(0) == probeSubTree) {
-                return new VariableReferenceExpression(optFuncExpr.getLogicalVar(0));
-            } else {
-                return new VariableReferenceExpression(optFuncExpr.getLogicalVar(1));
-            }
-        }
-    }
+	public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType, Index index,
+			boolean primaryKeysOnly, IOptimizationContext context, List<LogicalVariable> dest)
+					throws AlgebricksException {
+		int numPrimaryKeys=0;
+		if(dataset.getDatasetType() == DatasetType.EXTERNAL)
+		{
+			numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
+		}
+		else
+		{
+			numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+		}
+		int numSecondaryKeys = getNumSecondaryKeys(index, recordType);
+		int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
+		for (int i = 0; i < numVars; i++) {
+			dest.add(context.newVar());
+		}
+	}
 
-    /**
-     * Returns the first expr optimizable by this index.
-     */
-    public static IOptimizableFuncExpr chooseFirstOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
-        List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
-        int firstExprIndex = indexExprs.get(0);
-        return analysisCtx.matchedFuncExprs.get(firstExprIndex);
-    }
+	public static List<LogicalVariable> getPrimaryKeyVarsFromUnnestMap(Dataset dataset, ILogicalOperator unnestMapOp) {
+		int numPrimaryKeys;
+		if(dataset.getDatasetType() == DatasetType.EXTERNAL)
+		{
+			numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
+		}
+		else
+		{
+			numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+		}
+		List<LogicalVariable> primaryKeyVars = new ArrayList<LogicalVariable>();
+		List<LogicalVariable> sourceVars = ((UnnestMapOperator) unnestMapOp).getVariables();
+		// Assumes the primary keys are located at the end.
+		int start = sourceVars.size() - numPrimaryKeys;
+		int stop = sourceVars.size();
+		for (int i = start; i < stop; i++) {
+			primaryKeyVars.add(sourceVars.get(i));
+		}
+		return primaryKeyVars;
+	}
 
-    public static UnnestMapOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType, Index index,
-            ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams, IOptimizationContext context,
-            boolean outputPrimaryKeysOnly, boolean retainInput) throws AlgebricksException {
-        // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
-        ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
-        jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
-        // Variables and types coming out of the secondary-index search. 
-        List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<LogicalVariable>();
-        List<Object> secondaryIndexOutputTypes = new ArrayList<Object>();
-        // Append output variables/types generated by the secondary-index search (not forwarded from input).
-        appendSecondaryIndexOutputVars(dataset, recordType, index, outputPrimaryKeysOnly, context,
-                secondaryIndexUnnestVars);
-        appendSecondaryIndexTypes(dataset, recordType, index, outputPrimaryKeysOnly, secondaryIndexOutputTypes);
-        // An index search is expressed as an unnest over an index-search function.
-        IFunctionInfo secondaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-        UnnestingFunctionCallExpression secondaryIndexSearchFunc = new UnnestingFunctionCallExpression(
-                secondaryIndexSearch, secondaryIndexFuncArgs);
-        secondaryIndexSearchFunc.setReturnsUniqueValues(true);
-        // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
-        // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
-        UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
-                new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes, retainInput);
-        secondaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
-        context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
-        secondaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
-        return secondaryIndexUnnestOp;
-    }
+	/**
+	 * Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query then this method returns
+	 * the a ConstantExpression from the first constant value in the optimizable function expression.
+	 * If we are optimizing a join, then this method returns the VariableReferenceExpression that should feed the secondary index probe.
+	 */
+	public static ILogicalExpression createSearchKeyExpr(IOptimizableFuncExpr optFuncExpr,
+			OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree) {
+		if (probeSubTree == null) {
+			// We are optimizing a selection query. Search key is a constant.
+			return new ConstantExpression(optFuncExpr.getConstantVal(0));
+		} else {
+			// We are optimizing a join query. Determine which variable feeds the secondary index. 
+			if (optFuncExpr.getOperatorSubTree(0) == null || optFuncExpr.getOperatorSubTree(0) == probeSubTree) {
+				return new VariableReferenceExpression(optFuncExpr.getLogicalVar(0));
+			} else {
+				return new VariableReferenceExpression(optFuncExpr.getLogicalVar(1));
+			}
+		}
+	}
 
-    public static UnnestMapOperator createPrimaryIndexUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset,
-            ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys,
-            boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {
-        List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(dataset, inputOp);
-        // Optionally add a sort on the primary-index keys before searching the primary index.
-        OrderOperator order = null;
-        if (sortPrimaryKeys) {
-            order = new OrderOperator();
-            for (LogicalVariable pkVar : primaryKeyVars) {
-                Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(pkVar));
-                order.getOrderExpressions().add(
-                        new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
-            }
-            // The secondary-index search feeds into the sort.
-            order.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
-            order.setExecutionMode(ExecutionMode.LOCAL);
-            context.computeAndSetTypeEnvironmentForOperator(order);
-        }
-        // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments. 
-        List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
-        BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
-                dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
-        // Set low/high inclusive to true for a point lookup.
-        jobGenParams.setLowKeyInclusive(true);
-        jobGenParams.setHighKeyInclusive(true);
-        jobGenParams.setLowKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
-        jobGenParams.setHighKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
-        jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
-        // Variables and types coming out of the primary-index search.
-        List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<LogicalVariable>();
-        List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
-        // Append output variables/types generated by the primary-index search (not forwarded from input).
-        primaryIndexUnnestVars.addAll(dataSourceScan.getVariables());
-        try {
-            appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
-        } catch (IOException e) {
-            throw new AlgebricksException(e);
-        }
-        // An index search is expressed as an unnest over an index-search function.
-        IFunctionInfo primaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
-        AbstractFunctionCallExpression primaryIndexSearchFunc = new ScalarFunctionCallExpression(primaryIndexSearch,
-                primaryIndexFuncArgs);
-        // This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
-        // which index to use, which variables contain the index-search keys, what is the original dataset, etc.
-        UnnestMapOperator primaryIndexUnnestOp = new UnnestMapOperator(primaryIndexUnnestVars,
-                new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
-        // Fed by the order operator or the secondaryIndexUnnestOp.
-        if (sortPrimaryKeys) {
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
-        } else {
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
-        }
-        context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestOp);
-        primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
-        return primaryIndexUnnestOp;
-    }
+	/**
+	 * Returns the first expr optimizable by this index.
+	 */
+	public static IOptimizableFuncExpr chooseFirstOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
+		List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+		int firstExprIndex = indexExprs.get(0);
+		return analysisCtx.matchedFuncExprs.get(firstExprIndex);
+	}
+
+	public static void writeVarList(List<LogicalVariable> varList, List<Mutable<ILogicalExpression>> funcArgs) {
+		Mutable<ILogicalExpression> numKeysRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
+				new AsterixConstantValue(new AInt32(varList.size()))));
+		funcArgs.add(numKeysRef);
+		for (LogicalVariable keyVar : varList) {
+			Mutable<ILogicalExpression> keyVarRef = new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(keyVar));
+			funcArgs.add(keyVarRef);
+		}
+	}
+
+	public static ExternalDataAccessByRIDOperator createExternalDataAccessByRIDUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset,
+			ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context, Index secondaryIndex) throws AlgebricksException {
+		List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(dataset, inputOp);
+
+		// add a sort on the RID fields before fetching external data.
+		OrderOperator order = new OrderOperator();
+		for (LogicalVariable pkVar : primaryKeyVars) {
+			Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
+					new VariableReferenceExpression(pkVar));
+			order.getOrderExpressions().add(
+					new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
+		}
+		// The secondary-index search feeds into the sort.
+		order.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+		order.setExecutionMode(ExecutionMode.LOCAL);
+		context.computeAndSetTypeEnvironmentForOperator(order);
+
+		List<Mutable<ILogicalExpression>> externalRIDAccessFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+		AccessMethodUtils.writeVarList(primaryKeyVars,externalRIDAccessFuncArgs);
+
+		// Variables and types coming out of the external access.
+		List<LogicalVariable> externalAccessByRIDVars = new ArrayList<LogicalVariable>();
+		List<Object> externalAccessOutputTypes = new ArrayList<Object>();
+		// Append output variables/types generated by the data scan (not forwarded from input).
+		externalAccessByRIDVars.addAll(dataSourceScan.getVariables());
+		appendExternalRecTypes(dataset, recordType, externalAccessOutputTypes);
+
+		IFunctionInfo externalAccessByRID = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EXTERNAL_ACCESS_BY_RID);
+		AbstractFunctionCallExpression externalAccessFunc = new ScalarFunctionCallExpression(externalAccessByRID,
+				externalRIDAccessFuncArgs);
+
+		ExternalDataAccessByRIDOperator externalAccessByRIDOp = new ExternalDataAccessByRIDOperator(externalAccessByRIDVars,
+				new MutableObject<ILogicalExpression>(externalAccessFunc), externalAccessOutputTypes);
+		// Fed by the order operator or the secondaryIndexUnnestOp.
+		externalAccessByRIDOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+
+		context.computeAndSetTypeEnvironmentForOperator(externalAccessByRIDOp);
+		externalAccessByRIDOp.setExecutionMode(ExecutionMode.PARTITIONED);
+
+		//set the physical operator
+		AqlSourceId dataSourceId = new AqlSourceId(dataset.getDataverseName(),
+				dataset.getDatasetName());
+		externalAccessByRIDOp.setPhysicalOperator(new ExternalDataAccessByRIDPOperator(dataSourceId,  dataset, recordType, secondaryIndex));
+		return externalAccessByRIDOp;
+	}
+
+	public static UnnestMapOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType, Index index,
+			ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams, IOptimizationContext context,
+			boolean outputPrimaryKeysOnly, boolean retainInput) throws AlgebricksException {
+		// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments.
+		ArrayList<Mutable<ILogicalExpression>> secondaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+		jobGenParams.writeToFuncArgs(secondaryIndexFuncArgs);
+		// Variables and types coming out of the secondary-index search. 
+		List<LogicalVariable> secondaryIndexUnnestVars = new ArrayList<LogicalVariable>();
+		List<Object> secondaryIndexOutputTypes = new ArrayList<Object>();
+		// Append output variables/types generated by the secondary-index search (not forwarded from input).
+		appendSecondaryIndexOutputVars(dataset, recordType, index, outputPrimaryKeysOnly, context,
+				secondaryIndexUnnestVars);
+		appendSecondaryIndexTypes(dataset, recordType, index, outputPrimaryKeysOnly, secondaryIndexOutputTypes);
+		// An index search is expressed as an unnest over an index-search function.
+		IFunctionInfo secondaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
+		UnnestingFunctionCallExpression secondaryIndexSearchFunc = new UnnestingFunctionCallExpression(
+				secondaryIndexSearch, secondaryIndexFuncArgs);
+		secondaryIndexSearchFunc.setReturnsUniqueValues(true);
+		// This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
+		// which index to use, which variables contain the index-search keys, what is the original dataset, etc.
+		UnnestMapOperator secondaryIndexUnnestOp = new UnnestMapOperator(secondaryIndexUnnestVars,
+				new MutableObject<ILogicalExpression>(secondaryIndexSearchFunc), secondaryIndexOutputTypes, retainInput);
+		secondaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+		context.computeAndSetTypeEnvironmentForOperator(secondaryIndexUnnestOp);
+		secondaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
+		return secondaryIndexUnnestOp;
+	}
+
+	public static UnnestMapOperator createPrimaryIndexUnnestMap(DataSourceScanOperator dataSourceScan, Dataset dataset,
+			ARecordType recordType, ILogicalOperator inputOp, IOptimizationContext context, boolean sortPrimaryKeys,
+			boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {
+		List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(dataset, inputOp);
+		// Optionally add a sort on the primary-index keys before searching the primary index.
+		OrderOperator order = null;
+		if (sortPrimaryKeys) {
+			order = new OrderOperator();
+			for (LogicalVariable pkVar : primaryKeyVars) {
+				Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
+						new VariableReferenceExpression(pkVar));
+				order.getOrderExpressions().add(
+						new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
+			}
+			// The secondary-index search feeds into the sort.
+			order.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+			order.setExecutionMode(ExecutionMode.LOCAL);
+			context.computeAndSetTypeEnvironmentForOperator(order);
+		}
+		// The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments. 
+		List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
+		BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
+				dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
+		// Set low/high inclusive to true for a point lookup.
+		jobGenParams.setLowKeyInclusive(true);
+		jobGenParams.setHighKeyInclusive(true);
+		jobGenParams.setLowKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
+		jobGenParams.setHighKeyVarList(primaryKeyVars, 0, primaryKeyVars.size());
+		jobGenParams.writeToFuncArgs(primaryIndexFuncArgs);
+		// Variables and types coming out of the primary-index search.
+		List<LogicalVariable> primaryIndexUnnestVars = new ArrayList<LogicalVariable>();
+		List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
+		// Append output variables/types generated by the primary-index search (not forwarded from input).
+		primaryIndexUnnestVars.addAll(dataSourceScan.getVariables());
+		try {
+			appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
+		} catch (IOException e) {
+			throw new AlgebricksException(e);
+		}
+		// An index search is expressed as an unnest over an index-search function.
+		IFunctionInfo primaryIndexSearch = FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.INDEX_SEARCH);
+		AbstractFunctionCallExpression primaryIndexSearchFunc = new ScalarFunctionCallExpression(primaryIndexSearch,
+				primaryIndexFuncArgs);
+		// This is the operator that jobgen will be looking for. It contains an unnest function that has all necessary arguments to determine
+		// which index to use, which variables contain the index-search keys, what is the original dataset, etc.
+		UnnestMapOperator primaryIndexUnnestOp = new UnnestMapOperator(primaryIndexUnnestVars,
+				new MutableObject<ILogicalExpression>(primaryIndexSearchFunc), primaryIndexOutputTypes, retainInput);
+		// Fed by the order operator or the secondaryIndexUnnestOp.
+		if (sortPrimaryKeys) {
+			primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(order));
+		} else {
+			primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
+		}
+		context.computeAndSetTypeEnvironmentForOperator(primaryIndexUnnestOp);
+		primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
+		return primaryIndexUnnestOp;
+	}
 
 }
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 ddcf768..ce2a1f7 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
@@ -27,6 +27,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
@@ -51,6 +52,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
@@ -416,10 +418,16 @@
         // Generate the rest of the upstream plan which feeds the search results into the primary index.        
         UnnestMapOperator primaryIndexUnnestOp;
         boolean isPrimaryIndex = chosenIndex.getIndexName().equals(dataset.getDatasetName());
-        if (!isPrimaryIndex) {
+        if(dataset.getDatasetType() == DatasetType.EXTERNAL)
+    	{
+        	ExternalDataAccessByRIDOperator externalDataAccessOp = AccessMethodUtils.createExternalDataAccessByRIDUnnestMap(dataSourceScan, dataset, 
+    				recordType, secondaryIndexUnnestOp, context, chosenIndex);
+    		indexSubTree.dataSourceScanRef.setValue(externalDataAccessOp);
+    		return externalDataAccessOp;
+    	}
+        else 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
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
index 28aee7a..ea2adad 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableOperatorSubTree.java
@@ -114,7 +114,8 @@
         if (dataset == null) {
             throw new AlgebricksException("No metadata for dataset " + datasetName);
         }
-        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED
+        		&& dataset.getDatasetType() != DatasetType.EXTERNAL	) {
             return false;
         }
         // Get the record type for that dataset.
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
index b8125aa..8af4ac1 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -21,6 +21,7 @@
 import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
@@ -44,6 +45,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
@@ -196,10 +198,19 @@
         UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
                 chosenIndex, assignSearchKeys, jobGenParams, context, false, retainInput);
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
-        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
-                recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
+        if(dataset.getDatasetType() == DatasetType.EXTERNAL)
+		{
+        	ExternalDataAccessByRIDOperator externalDataAccessOp = AccessMethodUtils.createExternalDataAccessByRIDUnnestMap(dataSourceScan, dataset, 
+    				recordType, secondaryIndexUnnestOp, context, chosenIndex);
+			return externalDataAccessOp;
+		}
+		else
+		{
+			UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+					recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
 
-        return primaryIndexUnnestOp;
+			return primaryIndexUnnestOp;
+		}
     }
 
     @Override
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index e56ed92..18a81e5 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.asterix.api.common;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
@@ -45,7 +46,7 @@
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.SynchronousScheduler;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
 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;
@@ -107,7 +108,8 @@
                 storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages(),
                 storageProperties.getBufferCacheMaxOpenFiles(), ncApplicationContext.getThreadFactory());
 
-        lsmIOScheduler = SynchronousScheduler.INSTANCE;
+        AsynchronousScheduler.INSTANCE.init(ncApplicationContext.getThreadFactory());
+        lsmIOScheduler = AsynchronousScheduler.INSTANCE;
         mergePolicy = new ConstantMergePolicy(storageProperties.getLSMIndexMergeThreshold(), this);
 
         ILocalResourceRepositoryFactory persistentLocalResourceRepositoryFactory = new PersistentLocalResourceRepositoryFactory(
@@ -212,8 +214,8 @@
     }
 
     @Override
-    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
-        return indexLifecycleManager.getVirtualBufferCache(datasetID);
+    public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
+        return indexLifecycleManager.getVirtualBufferCaches(datasetID);
     }
 
     @Override
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index 712d993..ee9dfae 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.api.common;
 
+import java.util.List;
+
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -88,32 +90,28 @@
     }
 
     @Override
-    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
-        return asterixAppRuntimeContext.getVirtualBufferCache(datasetID);
+    public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
+        return asterixAppRuntimeContext.getVirtualBufferCaches(datasetID);
     }
 
     @Override
     public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
 
     @Override
     public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider(boolean isPrimary) {
-        if (isPrimary) {
-            return AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER;
-        } else {
-            return AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER;
-        }
+        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
 
     @Override
     public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER;
+        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
 
     @Override
     public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER;
+        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
 
     @Override
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 8aba12a..9c3a853 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
@@ -128,6 +128,7 @@
         ADDED_PENDINGOP_RECORD_TO_METADATA
     }
 
+    public static final boolean IS_DEBUG_MODE = false;//true
     private final List<Statement> aqlStatements;
     private final PrintWriter out;
     private final SessionConfig sessionConfig;
@@ -413,7 +414,8 @@
                 case EXTERNAL: {
                     String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
                     Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
-                    datasetDetails = new ExternalDatasetDetails(adapter, properties);
+                    String ngName = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
+                    datasetDetails = new ExternalDatasetDetails(adapter, properties,ngName);
                     break;
                 }
                 case FEED: {
@@ -589,6 +591,18 @@
             //#. create the index artifact in NC.
             runJob(hcc, spec, true);
 
+            //if external data and optimization is turned on, load file names
+            if(ds.getDatasetType() == DatasetType.EXTERNAL && AqlMetadataProvider.isOptimizeExternalIndexes())
+            {
+            	//load the file names into external files index
+            	mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+            	bActiveTxn = true;
+                metadataProvider.setMetadataTxnContext(mdTxnCtx);
+                IndexOperations.addExternalDatasetFilesToMetadata(metadataProvider, ds);
+                MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+                bActiveTxn = false;
+            }
+            
             mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
             bActiveTxn = true;
             metadataProvider.setMetadataTxnContext(mdTxnCtx);
@@ -653,6 +667,8 @@
                     throw new IllegalStateException("System is inconsistent state: pending index(" + dataverseName
                             + "." + datasetName + "." + indexName + ") couldn't be removed from the metadata", e);
                 }
+                
+                //if external dataset, remove external files from metadata
             }
             throw e;
         } finally {
@@ -729,7 +745,7 @@
             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) {
+                if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED || dsType == DatasetType.EXTERNAL) {
 
                     List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
                             datasetName);
@@ -842,7 +858,7 @@
                 }
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED || ds.getDatasetType() == DatasetType.EXTERNAL) {
 
                 //#. prepare jobs to drop the datatset and the indexes in NC
                 List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -943,7 +959,7 @@
                         + dataverseName);
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
+            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED || ds.getDatasetType() == DatasetType.EXTERNAL) {
                 indexName = stmtIndexDrop.getIndexName().getValue();
                 Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 if (index == null) {
@@ -981,8 +997,8 @@
                 //#. 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.");
+                //throw new AlgebricksException(datasetName
+                //        + " is an external dataset. Indexes are not maintained for external datasets.");
             }
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
 
@@ -1506,6 +1522,9 @@
 
     private void abort(Exception rootE, Exception parentE, MetadataTransactionContext mdTxnCtx) {
         try {
+            if (IS_DEBUG_MODE) {
+                rootE.printStackTrace();
+            }
             MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
         } catch (Exception e2) {
             parentE.addSuppressed(e2);
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 8086b0d..3e41a77 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
@@ -123,12 +123,12 @@
                         datasetName);
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
-                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
@@ -175,13 +175,12 @@
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
         TreeIndexCreateOperatorDescriptor indexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields,
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
                                 .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
@@ -260,15 +259,15 @@
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad;
         if (!loadStmt.alreadySorted()) {
             btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                    AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, splitsAndConstraint.first, typeTraits,
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
                     comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
                     true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -286,15 +285,15 @@
             spec.connect(new OneToOneConnectorDescriptor(spec), sorter, 0, btreeBulkLoad, 0);
         } else {
             btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                    AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, splitsAndConstraint.first, typeTraits,
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, splitsAndConstraint.first, typeTraits,
                     comparatorFactories, blooFilterKeyFields, fieldPermutation, GlobalConfig.DEFAULT_BTREE_FILL_FACTOR,
                     true, numElementsHint, true, new LSMBTreeDataflowHelperFactory(
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             storageProperties.getBloomFilterFalsePositiveRate()), 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 7bd6c69..de4d075 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
@@ -14,14 +14,20 @@
  */
 package edu.uci.ics.asterix.file;
 
+import java.util.ArrayList;
+
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledIndexDropStatement;
@@ -53,6 +59,17 @@
                 metadataProvider, physicalOptimizationConfig);
         return secondaryIndexCreator.buildLoadingJobSpec();
     }
+    
+    public static void addExternalDatasetFilesToMetadata(AqlMetadataProvider metadataProvider, 
+			Dataset dataset) throws AlgebricksException, MetadataException{
+			//get the file list
+			ArrayList<ExternalFile> files = metadataProvider.getExternalDatasetFiles(dataset);
+			//add files to metadata
+			for(int i=0; i < files.size(); i++)
+			{
+				MetadataManager.INSTANCE.addExternalFile(metadataProvider.getMetadataTxnContext(), files.get(i));
+			}
+	}
 
     public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
             AqlMetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
@@ -66,12 +83,12 @@
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
         AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
-                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset
+                                .getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
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 052368a..baf16de 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
@@ -17,18 +17,31 @@
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
+
+import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
+import edu.uci.ics.asterix.external.util.ExternalIndexHashPartitionComputerFactory;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 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.service.transaction.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 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.jobgen.impl.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 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.BTreeSearchOperatorDescriptor;
@@ -47,26 +60,24 @@
         super(physOptConf, propertiesProvider);
     }
 
-    @Override
-    public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
-        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
-        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
-        ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
-                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
-                dataset.getDatasetId());
-        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
-                localResourceMetadata, LocalResource.LSMBTreeResource);
-
+	@Override
+	public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
+		JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+		AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+		//prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+		ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
+				secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
+				dataset.getDatasetId());
+		ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+				localResourceMetadata, LocalResource.LSMBTreeResource);
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
                 secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
-                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
+                        dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset
+                                .getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -76,52 +87,117 @@
         return spec;
     }
 
-    @Override
-    public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
-        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
-        // Create dummy key provider for feeding the primary index scan. 
-        AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
+	@Override
+	public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException{
+		if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+			JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+			Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> RIDScanOpAndConstraints;
+			AlgebricksMetaOperatorDescriptor asterixAssignOp;
+			try
+			{
+				//create external indexing scan operator
+				RIDScanOpAndConstraints = createExternalIndexingOp(spec);
 
-        // Create primary index scan op.
-        BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
+				//create assign operator
+				asterixAssignOp = createExternalAssignOp(spec);
+				AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
+						RIDScanOpAndConstraints.second);
+			}
+			catch(Exception e)
+			{
+				throw new AsterixException("Failed to create external index scanning and loading job");
+			}
 
-        // Assign op.
-        AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
+			// 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);
+			}
 
-        // 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);
-        }
-
-        // Sort by secondary keys.
-        ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
-
-        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
-        // Create secondary BTree bulk load op.
+			// Sort by secondary keys.
+			ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc,RIDScanOpAndConstraints.second);
+			AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+			// Create secondary BTree bulk load op.
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
                 spec,
                 numSecondaryKeys,
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+                                LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+			IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
+
+			//select partitioning keys (always the first 2 after secondary keys)
+			int[] keys = new int[2];
+			keys[0] = numSecondaryKeys;
+			keys[1] = numSecondaryKeys + 1;
+
+			IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
+					new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
+
+			spec.connect(new OneToOneConnectorDescriptor(spec), RIDScanOpAndConstraints.first, 0, asterixAssignOp, 0);
+			if (anySecondaryKeyIsNullable) {
+				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+				spec.connect(hashConn, selectOp, 0, sortOp, 0);
+			} else {
+				spec.connect(hashConn, asterixAssignOp, 0, sortOp, 0);
+			}
+			spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
+			spec.addRoot(secondaryBulkLoadOp);
+			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+			return spec;
+		}
+		else
+		{
+			JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+
+			// Create dummy key provider for feeding the primary index scan. 
+			AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
+
+			// Create primary index scan op.
+			BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
+
+			// Assign op.
+			AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
+
+			// 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);
+			}
+
+			// Sort by secondary keys.
+			ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
+
+			AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+			// Create secondary BTree bulk load op.
+        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+                spec,
+                numSecondaryKeys,
+                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+                                LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
                                 .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
-        // Connect the operators.
-        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
-        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
-        if (anySecondaryKeyIsNullable) {
-            spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-            spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
-        } else {
-            spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
-        }
-        spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
-        spec.addRoot(secondaryBulkLoadOp);
-        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-        return spec;
-    }
+			// Connect the operators.
+			spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+			spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+			if (anySecondaryKeyIsNullable) {
+				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+				spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+			} else {
+				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+			}
+			spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
+			spec.addRoot(secondaryBulkLoadOp);
+			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+			return spec;
+		}
+	}
 }
+
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 6717b4d..5da336f 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
@@ -19,28 +19,40 @@
 import java.io.IOException;
 import java.util.List;
 
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
+import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
+import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
+import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.JobId;
 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.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.IsNullDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.NotDescriptor;
+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.PrimaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
+import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
@@ -63,6 +75,7 @@
 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.job.IJobletEventListenerFactory;
 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.marshalling.IntegerSerializerDeserializer;
@@ -70,12 +83,15 @@
 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.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
+import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 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.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,11 +109,11 @@
     protected ISerializerDeserializer payloadSerde;
     protected IFileSplitProvider primaryFileSplitProvider;
     protected AlgebricksPartitionConstraint primaryPartitionConstraint;
+    protected List<String> secondaryKeyFields;
     protected IFileSplitProvider secondaryFileSplitProvider;
     protected AlgebricksPartitionConstraint secondaryPartitionConstraint;
     protected String secondaryIndexName;
     protected boolean anySecondaryKeyIsNullable = false;
-
     protected long numElementsHint;
     protected IBinaryComparatorFactory[] primaryComparatorFactories;
     protected int[] primaryBloomFilterKeyFields;
@@ -106,7 +122,6 @@
     protected int[] secondaryBloomFilterKeyFields;
     protected RecordDescriptor secondaryRecDesc;
     protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
-
     protected IAsterixPropertiesProvider propertiesProvider;
 
     // Prevent public construction. Should be created via createIndexCreator().
@@ -150,37 +165,68 @@
     public abstract JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException;
 
     protected void init(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadataProvider)
-            throws AsterixException, AlgebricksException {
-        this.metadataProvider = metadataProvider;
-        dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
-                : createIndexStmt.getDataverseName();
-        datasetName = createIndexStmt.getDatasetName();
-        secondaryIndexName = createIndexStmt.getIndexName();
-        dataset = metadataProvider.findDataset(dataverseName, datasetName);
-        if (dataset == null) {
-            throw new AsterixException("Unknown dataset " + datasetName);
+                        throws AsterixException, AlgebricksException {
+                this.metadataProvider = metadataProvider;
+                dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
+                                : createIndexStmt.getDataverseName();
+                datasetName = createIndexStmt.getDatasetName();
+                secondaryIndexName = createIndexStmt.getIndexName();
+                dataset = metadataProvider.findDataset(dataverseName, datasetName);
+                if (dataset == null) {
+                        throw new AsterixException("Unknown dataset " + datasetName);
+                }
+                if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+                        //get external dataset details
+                        ExternalDatasetDetails edsd = ((ExternalDatasetDetails)dataset.getDatasetDetails());
+                        //get adapter name
+                        String adapter = edsd.getAdapter();
+                        //if not an hdfs adapter, throw an exception
+                        if(!adapter.equals(HDFSAdapterFactory.HDFS_ADAPTER_NAME) && !adapter.equals(HiveAdapter.class.getName()))
+                        {
+                                throw new AsterixException("Cannot index an external dataset with adapter type(" + adapter + ").");
+                        }
+                        //get the item type
+                        ARecordType externalItemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
+                        //number of primary keys here depends on the file input, 3 for rcfiles and 2 for text and sequence files.
+                        numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
+                        itemType = createExternalItemTypeWithRID(externalItemType);
+                        payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
+                        numSecondaryKeys = createIndexStmt.getKeyFields().size();
+                        //splits and constraints <--They don't exist-->
+                        primaryFileSplitProvider = null;
+                        primaryPartitionConstraint = null;
+                        //create secondary split and constraints
+                        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
+                                        .splitProviderAndPartitionConstraintsForExternalDataset(dataverseName, datasetName,
+                                                        secondaryIndexName);
+                        secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+                        secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
+                        // Must be called in this order.
+                        setExternalRIDDescAndComparators();
+                        setExternalSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
+                        numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
+                }
+                else
+                {
+                        itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
+                        payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
+                        numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+                        numSecondaryKeys = createIndexStmt.getKeyFields().size();
+                        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
+                                        .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
+                        primaryFileSplitProvider = primarySplitsAndConstraint.first;
+                        primaryPartitionConstraint = primarySplitsAndConstraint.second;
+                        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
+                                        .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
+                                                        secondaryIndexName);
+                        secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+                        secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
+                        // Must be called in this order.
+                        setPrimaryRecDescAndComparators();
+                        setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
+                        numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
+                }
         }
-        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
-        }
-        itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
-        payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-        numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-        numSecondaryKeys = createIndexStmt.getKeyFields().size();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
-        primaryFileSplitProvider = primarySplitsAndConstraint.first;
-        primaryPartitionConstraint = primarySplitsAndConstraint.second;
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
-                        secondaryIndexName);
-        secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
-        secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
-        // Must be called in this order.
-        setPrimaryRecDescAndComparators();
-        setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
-        numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
-    }
 
     protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
@@ -264,23 +310,235 @@
         return keyProviderOp;
     }
 
+protected ARecordType createExternalItemTypeWithRID(
+                        ARecordType externalItemType) throws AsterixException {
+
+                String[] fieldsNames = new String[externalItemType.getFieldNames().length+numPrimaryKeys];
+                IAType[] fieldsTypes = new IAType[externalItemType.getFieldTypes().length+numPrimaryKeys];
+
+                //add RID fields names and types
+                if(AqlMetadataProvider.isOptimizeExternalIndexes())
+                {
+                        fieldsNames[0] = "_file-number";
+                        fieldsTypes[0] = BuiltinType.AINT32;
+                }
+                else
+                {
+                        fieldsNames[0] = "_file-name";
+                        fieldsTypes[0] = BuiltinType.ASTRING;
+                }
+                fieldsNames[1] = "_byte-location";
+                fieldsTypes[1] = BuiltinType.AINT64;
+                if(numPrimaryKeys == 3)
+                {       
+                        //add the row number for rc files
+                        fieldsNames[2] = "_row-number";
+                        fieldsTypes[2] = BuiltinType.AINT32;
+                }
+                
+                //add the original fields names and types
+                for(int i=0; i < externalItemType.getFieldNames().length; i++)
+                {
+                        fieldsNames[i+numPrimaryKeys] = externalItemType.getFieldNames()[i];
+                        fieldsTypes[i+numPrimaryKeys] = externalItemType.getFieldTypes()[i];
+                }
+                return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
+        }
+
+        protected void setExternalRIDDescAndComparators() throws AlgebricksException {
+
+                ISerializerDeserializer[] externalRecFields = new ISerializerDeserializer[itemType.getFieldNames().length];
+                ITypeTraits[] externalTypeTraits = new ITypeTraits[itemType.getFieldNames().length];
+
+                primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
+                primaryBloomFilterKeyFields = new int[numPrimaryKeys];
+                ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
+                
+                if(AqlMetadataProvider.isOptimizeExternalIndexes())
+                {
+                        primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true);
+                }
+                else
+                {
+                        primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.ASTRING, true);
+                }
+                primaryComparatorFactories[1] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT64, true);
+
+                primaryBloomFilterKeyFields[0]=0;
+                primaryBloomFilterKeyFields[1]=1;
+
+                if(numPrimaryKeys == 3)
+                {
+                        primaryComparatorFactories[2] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true);
+                        primaryBloomFilterKeyFields[2]=2;
+                }
+
+                for(int i=0; i < itemType.getFieldNames().length; i++)
+                {
+                        externalRecFields[i] = serdeProvider.getSerializerDeserializer(itemType.getFieldTypes()[i]); 
+                        externalTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType.getFieldTypes()[i]);
+                }
+                primaryRecDesc = new RecordDescriptor(externalRecFields, externalTypeTraits);
+        }
+
+protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+                        AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
+                secondaryKeyFields = createIndexStmt.getKeyFields();
+                secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys+ numPrimaryKeys];
+                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();
+                ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
+                IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
+                                .getBinaryComparatorFactoryProvider();
+
+                for (int i = 0; i < numSecondaryKeys; i++) {
+                        secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+                                        itemType, secondaryKeyFields.get(i), 0);
+                        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
+                        IAType keyType = keyTypePair.first;
+                        anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
+                        ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
+                        secondaryRecFields[i] = keySerde;
+                        secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
+                        secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
+                        secondaryBloomFilterKeyFields[i] = i;
+                }
+
+                if(AqlMetadataProvider.isOptimizeExternalIndexes())
+                {
+                        secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+                                itemType, "_file-number", 0);
+                }
+                else
+                {
+                        secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+                                        itemType, "_file-name", 0);
+                }
+                secondaryFieldAccessEvalFactories[numSecondaryKeys+1] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+                                itemType, "_byte-location", 0);
+                if(numPrimaryKeys == 3)
+                {
+                        secondaryFieldAccessEvalFactories[numSecondaryKeys+2] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
+                                        itemType, "_row-number", 0);
+                }
+
+                for (int i = 0; i < numPrimaryKeys; i++) {
+                        secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
+                        secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
+                        secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
+                }
+                secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+        }
+
+protected Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(JobSpecification spec) throws Exception {
+                Pair<ExternalDataIndexingOperatorDescriptor,AlgebricksPartitionConstraint> indexingOpAndConstraints = metadataProvider.buildExternalDataIndexingRuntime(spec, itemType, dataset, NonTaggedDataFormat.INSTANCE);
+                AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexingOpAndConstraints.first,
+                                indexingOpAndConstraints.second);
+                return indexingOpAndConstraints;
+        }
+
+protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec) throws AlgebricksException {
+                int[] outColumns = new int[numSecondaryKeys + numPrimaryKeys];
+                int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
+                for (int i = 0; i < numSecondaryKeys + numPrimaryKeys; i++) {
+                        outColumns[i] = i;
+                        projectionList[i] = i;
+                }
+
+                IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
+                for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
+                        sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
+                                        secondaryFieldAccessEvalFactories[i]);
+                }
+                AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
+                AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
+                                new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
+                return asterixAssignOp;
+        }
+
+        protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
+                        IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc,
+                        AlgebricksPartitionConstraint partitionConstraints) {
+                int[] sortFields = new int[secondaryComparatorFactories.length];
+                for (int i = 0; i < secondaryComparatorFactories.length; i++) {
+                        sortFields[i] = i;
+                }
+                ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
+                                physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
+                AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, partitionConstraints);
+                return sortOp;
+        }
+
+protected ARecordType createSecondaryItemType(ARecordType externalItemType, boolean isRCFile) throws AsterixException
+        {
+
+                String[] fieldsNames = new String[numSecondaryKeys+numPrimaryKeys];
+                IAType[] fieldsTypes = new IAType[numSecondaryKeys+numPrimaryKeys];
+
+                //first create the secondary index fields
+                for(int i=0; i<numSecondaryKeys; i++)
+                {
+                        fieldsNames[i] = secondaryKeyFields.get(i);
+                        try {
+                                fieldsTypes[i] = externalItemType.getFieldType(fieldsNames[i]);
+                        } catch (IOException e) {
+                                // TODO Auto-generated catch block
+                                throw new AsterixException(e);
+                        }
+                }
+
+                //second add RID fields (File name or number and byte location)
+                if(AqlMetadataProvider.isOptimizeExternalIndexes())
+                {
+                        fieldsNames[numSecondaryKeys] = "_file-number";
+                        fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
+                }
+                else
+                {
+                        fieldsNames[numSecondaryKeys] = "_file-name";
+                        fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
+                }
+                fieldsNames[numSecondaryKeys+1] = "_byte-location";
+                fieldsTypes[numSecondaryKeys+1] = BuiltinType.AINT64;
+
+                if(isRCFile)
+                {
+                        fieldsNames[numSecondaryKeys+2] = "_row-Number";
+                        fieldsTypes[numSecondaryKeys+2] = BuiltinType.AINT32;
+                }
+
+                //return type
+                return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
+        }
+
     protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
         // -Infinity
         int[] lowKeyFields = null;
         // +Infinity
         int[] highKeyFields = null;
+        ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+        JobId jobId = JobIdFactory.generateJobId();
+        metadataProvider.setJobId(jobId);
+        boolean isWriteTransaction = metadataProvider.isWriteTransaction();
+        IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(jobId, isWriteTransaction);
+        spec.setJobletEventListenerFactory(jobEventListenerFactory);
+
+        ISearchOperationCallbackFactory searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(
+                jobId, dataset.getDatasetId(), primaryBloomFilterKeyFields, txnSubsystemProvider,
+                ResourceType.LSM_BTREE);
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
                 primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
-                                .getBloomFilterFalsePositiveRate()), false,
-                NoOpOperationCallbackFactory.INSTANCE);
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        storageProperties.getBloomFilterFalsePositiveRate()), false, searchCallbackFactory);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
@@ -333,11 +591,10 @@
             fieldPermutation[i] = i;
         }
         TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
-                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
-                NoOpOperationCallbackFactory.INSTANCE);
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+                secondaryBloomFilterKeyFields, fieldPermutation, fillFactor, false, numElementsHint, false,
+                dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
@@ -376,3 +633,4 @@
         return asterixSelectOp;
     }
 }
+
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 17590c5..6f64aa2 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
@@ -22,11 +22,13 @@
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 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.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.resource.LSMInvertedIndexLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -176,8 +178,8 @@
 
         IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
         LSMInvertedIndexCreateOperatorDescriptor invIndexCreateOp = new LSMInvertedIndexCreateOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
                 invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
                 localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexCreateOp,
@@ -267,8 +269,8 @@
         IIndexDataflowHelperFactory dataflowHelperFactory = createDataflowHelperFactory();
         LSMInvertedIndexBulkLoadOperatorDescriptor invIndexBulkLoadOp = new LSMInvertedIndexBulkLoadOperatorDescriptor(
                 spec, fieldPermutation, false, numElementsHint, false,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, secondaryFileSplitProvider,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, secondaryFileSplitProvider,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, tokenTypeTraits, tokenComparatorFactories,
                 invListsTypeTraits, primaryComparatorFactories, tokenizerFactory, dataflowHelperFactory,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, invIndexBulkLoadOp,
@@ -280,18 +282,24 @@
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         if (!isPartitioned) {
             return new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
-                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    new SecondaryIndexOperationTrackerProvider(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
+                            dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     storageProperties.getBloomFilterFalsePositiveRate());
         } else {
             return new PartitionedLSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
-                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
+                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    new SecondaryIndexOperationTrackerProvider(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
+                            dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                     storageProperties.getBloomFilterFalsePositiveRate());
         }
     }
+    
+    @Override
+	protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+			AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
+    	throw new AsterixException("Cannot create inverted index on external dataset due to composite RID Fields.");
+	}
 }
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 17632aa..6ce694c 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
@@ -18,34 +18,47 @@
 
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
+import edu.uci.ics.asterix.external.util.ExternalIndexHashPartitionComputerFactory;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Index;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 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.runtime.formats.NonTaggedDataFormat;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 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.jobgen.impl.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
+import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
@@ -84,14 +97,14 @@
                 localResourceMetadata, LocalResource.LSMRTreeResource);
 
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, null,
                 new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
                         primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+                                LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
                                 keyType, secondaryComparatorFactories.length), storageProperties
                                 .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
@@ -148,8 +161,121 @@
     }
 
     @Override
+	protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
+			AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
+		secondaryKeyFields = createIndexStmt.getKeyFields();
+		if (numSecondaryKeys != 1) {
+			throw new AsterixException(
+					"Cannot use "
+							+ numSecondaryKeys
+							+ " fields as a key for the R-tree index. There can be only one field as a key for the R-tree index.");
+		}
+		Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), itemType);
+		IAType spatialType = spatialTypePair.first;
+		anySecondaryKeyIsNullable = spatialTypePair.second;
+		if (spatialType == null) {
+			throw new AsterixException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
+		}
+		int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
+		numNestedSecondaryKeyFields = numDimensions * 2;
+		secondaryFieldAccessEvalFactories = metadataProvider.getFormat().createMBRFactory(itemType, secondaryKeyFields.get(0),
+				numPrimaryKeys, numDimensions);
+		secondaryComparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
+		valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
+		ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys
+		                                                                           + 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);
+			secondaryRecFields[i] = keySerde;
+			secondaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
+					nestedKeyType, true);
+			secondaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
+			valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
+		}
+
+		// Add serializers and comparators for primary index fields.
+		for (int i = 0; i < numPrimaryKeys; i++) {
+			secondaryRecFields[numNestedSecondaryKeyFields + i] = primaryRecDesc.getFields()[i];
+			secondaryTypeTraits[numNestedSecondaryKeyFields + i] = primaryRecDesc.getTypeTraits()[i];
+		}
+		secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
+	}
+    
+    @Override
     public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+			Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> RIDScanOpAndConstraints;
+			AlgebricksMetaOperatorDescriptor asterixAssignOp;
+			try
+			{
+				//create external indexing scan operator
+				RIDScanOpAndConstraints = createExternalIndexingOp(spec);
+				//create assign operator
+				asterixAssignOp = createExternalAssignOp(spec);
+				AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixAssignOp,
+						RIDScanOpAndConstraints.second);
+			}
+			catch(Exception e)
+			{
+				throw new AsterixException("Failed to create external index scanning and loading job");
+			}
+
+			// 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);
+			}
+
+			// Create secondary RTree bulk load op.
+			AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+			TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+					spec,
+					numNestedSecondaryKeyFields,
+					new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
+	                        primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
+	                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+	                                LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+	                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+	                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+	                                keyType, secondaryComparatorFactories.length), storageProperties
+	                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+			// Connect the operators.
+			// Create a hash partitioning connector
+			ExternalDatasetDetails edsd = (ExternalDatasetDetails)dataset.getDatasetDetails();
+			IBinaryHashFunctionFactory[] hashFactories = null;
+			if(edsd.getProperties().get(HDFSAdapterFactory.KEY_INPUT_FORMAT).trim().equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
+			{
+				hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
+			}
+			else
+			{
+				hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
+			}	 
+			//select partitioning keys (always the first 2 after secondary keys)
+			int[] keys = new int[2];
+			keys[0] = numSecondaryKeys;
+			keys[1] = numSecondaryKeys + 1;
+
+			IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
+					new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
+			spec.connect(new OneToOneConnectorDescriptor(spec), RIDScanOpAndConstraints.first, 0, asterixAssignOp, 0);
+			if (anySecondaryKeyIsNullable) {
+				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+				spec.connect(hashConn, selectOp, 0, secondaryBulkLoadOp, 0);
+			} else {
+				spec.connect(hashConn, asterixAssignOp, 0, secondaryBulkLoadOp, 0);
+			}
+			spec.addRoot(secondaryBulkLoadOp);
+			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+			return spec;
+		}
+		else
+		{
 
         // Create dummy key provider for feeding the primary index scan. 
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -174,10 +300,10 @@
                 numNestedSecondaryKeyFields,
                 new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
                         primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
+                                LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
                                 keyType, secondaryComparatorFactories.length), storageProperties
                                 .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
@@ -193,5 +319,7 @@
         spec.addRoot(secondaryBulkLoadOp);
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         return spec;
+		}
     }
 }
+
diff --git a/asterix-app/src/main/resources/asterix-build-configuration.xml b/asterix-app/src/main/resources/asterix-build-configuration.xml
index 6a6332d..9c4d15b 100644
--- a/asterix-app/src/main/resources/asterix-build-configuration.xml
+++ b/asterix-app/src/main/resources/asterix-build-configuration.xml
@@ -1,50 +1,40 @@
-<!--
- ! 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.
- !-->
+<!-- ! 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. ! -->
 <asterixConfiguration xmlns="asterixconf">
-  <metadataNode>nc1</metadataNode>
-  <store>
-     <ncId>nc1</ncId>
-     <storeDirs>nc1data</storeDirs> 
-  </store>
-  <store>
-     <ncId>nc2</ncId>
-     <storeDirs>nc2data</storeDirs> 
-  </store>
-  <transactionLogDir>
-  	 <ncId>nc1</ncId>
-  	 <txnLogDirPath>target/txnLogDir/nc1</txnLogDirPath> 
-  </transactionLogDir>
-  <transactionLogDir>
-  	 <ncId>nc2</ncId>
-  	 <txnLogDirPath>target/txnLogDir/nc2</txnLogDirPath> 
-  </transactionLogDir>
-  <property>
-     <name>log.level</name>
-     <value>WARNING</value>
-     <description>Log level for running tests/build</description>
-  </property>
-  <property>
+	<metadataNode>nc1</metadataNode>
+	<store>
+		<ncId>nc1</ncId>
+		<storeDirs>nc1data</storeDirs>
+	</store>
+	<store>
+		<ncId>nc2</ncId>
+		<storeDirs>nc2data</storeDirs>
+	</store>
+	<transactionLogDir>
+		<ncId>nc1</ncId>
+		<txnLogDirPath>target/txnLogDir/nc1</txnLogDirPath>
+	</transactionLogDir>
+	<transactionLogDir>
+		<ncId>nc2</ncId>
+		<txnLogDirPath>target/txnLogDir/nc2</txnLogDirPath>
+	</transactionLogDir>
+	<property>
+		<name>log.level</name>
+		<value>WARNING</value>
+		<description>Log level for running tests/build</description>
+	</property>
+	<property>
 		<name>storage.memorycomponent.numpages</name>
 		<value>8</value>
 		<description>The number of pages to allocate for a memory component.
 			(Default = 8)
 		</description>
 	</property>
-  <property>
-     <name>txn.log.groupcommitinterval</name>
-     <value>1</value>
-     <description>The group commit wait time in milliseconds.</description>
-  </property>
 </asterixConfiguration>
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index b56fe7c..8935d5d 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,8 +1,9 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
index d193dd1..a44cbc0 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
@@ -1,62 +1,63 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModDate", "FieldType": "datetime" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
index 607bfd1..6037036 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
@@ -1,11 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index 0078603..8935d5d 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -1,9 +1,9 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 2, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 8, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 3, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 1, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 9, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 7, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 4, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 5, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 6, "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index cecdb85..a44cbc0 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,62 +1,63 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModDate", "FieldType": "datetime" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index 607bfd1..6037036 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -1,11 +1,12 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
index cf7b309..29f6242 100644
--- a/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/insert-and-scan-dataset-with-index/insert-and-scan-dataset-with-index.2.update.aql
@@ -8,8 +8,6 @@
  
 use dataverse test;
 
-use dataverse test;
-
 load dataset test.employee
 using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
 (("path"="nc1://data/names.adm"),("format"="delimited-text"),("delimiter"="|"));
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
index 7be9c57..445122f 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
@@ -1,7 +1,7 @@
-{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ] }, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 125, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 119, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 122, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 123, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 120, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 121, "PendingOp": 0 }
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 124, "PendingOp": 0 }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
index 4233225..2d97f0a 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
@@ -16,10 +16,20 @@
 
 import java.util.Map;
 
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+
 public class ExternalDetailsDecl implements IDatasetDetailsDecl {
     private Map<String, String> properties;
     private String adapter;
+    private Identifier nodegroupName;
 
+    public ExternalDetailsDecl(Map<String, String> properties, String adapter, Identifier nodegroupName) {
+		this.properties = properties;
+		this.adapter = adapter;
+		this.nodegroupName = nodegroupName == null ? new Identifier(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)
+        : nodegroupName;
+	}
+    
     public void setAdapter(String adapter) {
         this.adapter = adapter;
     }
@@ -35,4 +45,12 @@
     public Map<String, String> getProperties() {
         return properties;
     }
+    
+    public void setNodegroupName(Identifier nodegroupName) {
+		this.nodegroupName = nodegroupName;
+	}
+    
+    public Identifier getNodegroupName() {
+		return nodegroupName;
+	}
 }
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index cb6336b..8dd7bfb 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -308,11 +308,11 @@
     <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
     ifNotExists = IfNotExists()
     "using" adapterName = AdapterName() properties = Configuration()
+    ("on" nodeGroupName = Identifier() )?
     ( "hints" hints = Properties() )?
       {
-        ExternalDetailsDecl edd = new ExternalDetailsDecl();
-        edd.setAdapter(adapterName);
-        edd.setProperties(properties);
+        ExternalDetailsDecl edd = new ExternalDetailsDecl(properties,
+        adapterName,nodeGroupName != null? new Identifier(nodeGroupName): null);
         dsetDecl = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    new Identifier(typeName),
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
index 14975ff..edd4b2a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixThreadExecutor.java
@@ -14,12 +14,15 @@
  */
 package edu.uci.ics.asterix.common.api;
 
+import java.util.concurrent.Callable;
 import java.util.concurrent.Executor;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 public class AsterixThreadExecutor implements Executor {
     public final static AsterixThreadExecutor INSTANCE = new AsterixThreadExecutor();
-    private final Executor executor = Executors.newCachedThreadPool(AsterixThreadFactory.INSTANCE);
+    private final ExecutorService executorService = Executors.newCachedThreadPool(AsterixThreadFactory.INSTANCE);
 
     private AsterixThreadExecutor() {
 
@@ -27,6 +30,10 @@
 
     @Override
     public void execute(Runnable command) {
-        executor.execute(command);
+        executorService.execute(command);
+    }
+
+    public Future<Object> submit(Callable command) {
+        return (Future<Object>) executorService.submit(command);
     }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
index d035303..4287212 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -15,6 +15,7 @@
 package edu.uci.ics.asterix.common.api;
 
 import java.io.IOException;
+import java.util.List;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -62,5 +63,5 @@
 
     public double getBloomFilterFalsePositiveRate();
 
-    public IVirtualBufferCache getVirtualBufferCache(int datasetID);
+    public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
index 85f90e4..d7ff15d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
@@ -31,6 +31,9 @@
     private static final String STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.memorycomponent.numpages";
     private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 1024; // ... so 32MB components
 
+    private static final String STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY = "storage.memorycomponent.numcomponents";
+    private static final int STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT = 2; // 2 components
+
     private static final String STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY = "storage.memorycomponent.globalbudget";
     private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = 536870912; // 512MB
 
@@ -53,7 +56,7 @@
         return accessor.getProperty(STORAGE_BUFFERCACHE_SIZE_KEY, STORAGE_BUFFERCACHE_SIZE_DEFAULT,
                 PropertyInterpreters.getLongPropertyInterpreter());
     }
-    
+
     public int getBufferCacheNumPages() {
         return (int) (getBufferCacheSize() / getBufferCachePageSize());
     }
@@ -73,6 +76,11 @@
                 PropertyInterpreters.getIntegerPropertyInterpreter());
     }
 
+    public int getMemoryComponentsNum() {
+        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_KEY,
+                STORAGE_MEMORYCOMPONENT_NUMCOMPONENTS_DEFAULT, PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
     public long getMemoryComponentGlobalBudget() {
         return accessor.getProperty(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY,
                 STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT, PropertyInterpreters.getLongPropertyInterpreter());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
index 5a40ece..a1dd52a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -25,12 +25,6 @@
     private static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
     private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = ((long)2 << 30); // 2GB
 
-    private static final String TXN_LOG_DISKSECTORSIZE_KEY = "txn.log.disksectorsize";
-    private static final int TXN_LOG_DISKSECTORSIZE_DEFAULT = 4096;
-
-    private static final String TXN_LOG_GROUPCOMMITINTERVAL_KEY = "txn.log.groupcommitinterval";
-    private static int TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT = 10; // 0.1ms
-
     private static final String TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY = "txn.log.checkpoint.lsnthreshold";
     private static final int TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT = (64 << 20); // 64M
 
@@ -75,16 +69,6 @@
                 PropertyInterpreters.getLongPropertyInterpreter());
     }
 
-    public int getLogDiskSectorSize() {
-        return accessor.getProperty(TXN_LOG_DISKSECTORSIZE_KEY, TXN_LOG_DISKSECTORSIZE_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
-    }
-
-    public int getGroupCommitInterval() {
-        return accessor.getProperty(TXN_LOG_GROUPCOMMITINTERVAL_KEY, TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT,
-                PropertyInterpreters.getIntegerPropertyInterpreter());
-    }
-
     public int getCheckpointLSNThreshold() {
         return accessor.getProperty(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY, TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT,
                 PropertyInterpreters.getIntegerPropertyInterpreter());
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
index bd2828d..9efc9fd 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/AsterixVirtualBufferCacheProvider.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.common.context;
 
+import java.util.List;
+
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.IVirtualBufferCache;
@@ -29,9 +31,9 @@
     }
 
     @Override
-    public IVirtualBufferCache getVirtualBufferCache(IHyracksTaskContext ctx) {
+    public List<IVirtualBufferCache> getVirtualBufferCaches(IHyracksTaskContext ctx) {
         return ((IAsterixAppRuntimeContext) ctx.getJobletContext().getApplicationContext().getApplicationObject())
-                .getVirtualBufferCache(datasetID);
+                .getVirtualBufferCaches(datasetID);
     }
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
index c48115c..3610478 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
@@ -26,13 +26,18 @@
 
 public class BaseOperationTracker implements ILSMOperationTracker {
 
+    protected final DatasetLifecycleManager datasetLifecycleManager;
     protected final ILSMIOOperationCallback ioOpCallback;
     protected long lastLSN;
     protected long firstLSN;
+    protected final int datasetID;
 
-    public BaseOperationTracker(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+    public BaseOperationTracker(DatasetLifecycleManager datasetLifecycleManager,
+            ILSMIOOperationCallbackFactory ioOpCallbackFactory, int datasetID) {
+        this.datasetLifecycleManager = datasetLifecycleManager;
         this.ioOpCallback = ioOpCallbackFactory == null ? NoOpIOOperationCallback.INSTANCE : ioOpCallbackFactory
                 .createIOOperationCallback(this);
+        this.datasetID = datasetID;
         resetLSNs();
     }
 
@@ -63,11 +68,17 @@
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+            datasetLifecycleManager.declareActiveIOOperation(datasetID);
+        }
     }
 
     @Override
     public void afterOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
+        if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+            datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
+        }
     }
 
     @Override
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
index 3ffa73d..cf69bfe 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
@@ -15,10 +15,13 @@
 
 package edu.uci.ics.asterix.common.context;
 
+import java.util.List;
+
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IndexException;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
+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.api.ILSMMergePolicy;
@@ -34,9 +37,10 @@
         this.ctx = ctx;
     }
 
-    public void diskComponentAdded(final ILSMIndex index, int totalNumDiskComponents) throws HyracksDataException,
-            IndexException {
-        if (!ctx.isShuttingdown() && totalNumDiskComponents >= threshold) {
+    @Override
+    public void diskComponentAdded(final ILSMIndex index) throws HyracksDataException, IndexException {
+        List<ILSMComponent> immutableComponents = index.getImmutableComponents();
+        if (!ctx.isShuttingdown() && immutableComponents.size() >= threshold) {
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
             accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
index 7232f01..617b6ff 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -31,7 +31,9 @@
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 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.impls.NoOpOperationCallback;
 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.api.IVirtualBufferCache;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.MultitenantVirtualBufferCache;
@@ -42,7 +44,7 @@
 
 public class DatasetLifecycleManager implements IIndexLifecycleManager, ILifeCycleComponent {
     private final AsterixStorageProperties storageProperties;
-    private final Map<Integer, MultitenantVirtualBufferCache> datasetVirtualBufferCaches;
+    private final Map<Integer, List<IVirtualBufferCache>> datasetVirtualBufferCaches;
     private final Map<Integer, ILSMOperationTracker> datasetOpTrackers;
     private final Map<Integer, DatasetInfo> datasetInfos;
     private final ILocalResourceRepository resourceRepository;
@@ -53,7 +55,7 @@
             ILocalResourceRepository resourceRepository) {
         this.storageProperties = storageProperties;
         this.resourceRepository = resourceRepository;
-        datasetVirtualBufferCaches = new HashMap<Integer, MultitenantVirtualBufferCache>();
+        datasetVirtualBufferCaches = new HashMap<Integer, List<IVirtualBufferCache>>();
         datasetOpTrackers = new HashMap<Integer, ILSMOperationTracker>();
         datasetInfos = new HashMap<Integer, DatasetInfo>();
         capacity = storageProperties.getMemoryComponentGlobalBudget();
@@ -98,29 +100,67 @@
     public synchronized void unregister(long resourceID) throws HyracksDataException {
         int did = getDIDfromRID(resourceID);
         DatasetInfo dsInfo = datasetInfos.get(did);
-        IndexInfo iInfo = dsInfo.indexes.remove(resourceID);
+        IndexInfo iInfo = dsInfo.indexes.get(resourceID);
+
         if (dsInfo == null || iInfo == null) {
             throw new HyracksDataException("Index with resource ID " + resourceID + " does not exist.");
         }
 
-        if (iInfo.referenceCount != 0) {
-            dsInfo.indexes.put(resourceID, iInfo);
+        PrimaryIndexOperationTracker opTracker = (PrimaryIndexOperationTracker) datasetOpTrackers.get(dsInfo.datasetID);
+        if (iInfo.referenceCount != 0 || (opTracker != null && opTracker.getNumActiveOperations() != 0)) {
             throw new HyracksDataException("Cannot remove index while it is open.");
         }
 
+        // TODO: use fine-grained counters, one for each index instead of a single counter per dataset.
+
+        // First wait for any ongoing IO operations
+        while (dsInfo.numActiveIOOps > 0) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+
+        // Flush and wait for it to finish, it is separated from the above wait so they don't deadlock each other.
+        // TODO: Find a better way to do this.
+        flushAndWaitForIO(dsInfo, iInfo);
+
         if (iInfo.isOpen) {
-            iInfo.index.deactivate(true);
+            iInfo.index.deactivate(false);
         }
 
+        dsInfo.indexes.remove(resourceID);
         if (dsInfo.referenceCount == 0 && dsInfo.isOpen && dsInfo.indexes.isEmpty()) {
-            IVirtualBufferCache vbc = getVirtualBufferCache(did);
-            assert vbc != null;
-            used -= (vbc.getNumPages() * vbc.getPageSize());
+            List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
+            assert vbcs != null;
+            for (IVirtualBufferCache vbc : vbcs) {
+                used -= (vbc.getNumPages() * vbc.getPageSize());
+            }
             datasetInfos.remove(did);
+            datasetVirtualBufferCaches.remove(did);
+            datasetOpTrackers.remove(did);
         }
 
     }
 
+    public synchronized void declareActiveIOOperation(int datasetID) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(datasetID);
+        if (dsInfo == null) {
+            throw new HyracksDataException("Failed to find a dataset with ID " + datasetID);
+        }
+        dsInfo.incrementActiveIOOps();
+    }
+
+    public synchronized void undeclareActiveIOOperation(int datasetID) throws HyracksDataException {
+        DatasetInfo dsInfo = datasetInfos.get(datasetID);
+        if (dsInfo == null) {
+            throw new HyracksDataException("Failed to find a dataset with ID " + datasetID);
+        }
+        dsInfo.decrementActiveIOOps();
+        notifyAll();
+    }
+
     @Override
     public synchronized void open(long resourceID) throws HyracksDataException {
         int did = getDIDfromRID(resourceID);
@@ -137,9 +177,12 @@
         }
 
         if (!dsInfo.isOpen) {
-            IVirtualBufferCache vbc = getVirtualBufferCache(did);
-            assert vbc != null;
-            long additionalSize = vbc.getNumPages() * vbc.getPageSize();
+            List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(did);
+            assert vbcs != null;
+            long additionalSize = 0;
+            for (IVirtualBufferCache vbc : vbcs) {
+                additionalSize += vbc.getNumPages() * vbc.getPageSize();
+            }
             while (used + additionalSize > capacity) {
                 if (!evictCandidateDataset()) {
                     throw new HyracksDataException("Cannot activate index since memory budget would be exceeded.");
@@ -165,26 +208,63 @@
         List<DatasetInfo> datasetInfosList = new ArrayList<DatasetInfo>(datasetInfos.values());
         Collections.sort(datasetInfosList);
         for (DatasetInfo dsInfo : datasetInfosList) {
-            ILSMOperationTracker opTracker = datasetOpTrackers.get(dsInfo.datasetID);
-            if (opTracker != null && ((PrimaryIndexOperationTracker) opTracker).getNumActiveOperations() == 0
-                    && dsInfo.referenceCount == 0 && dsInfo.isOpen) {
+            PrimaryIndexOperationTracker opTracker = (PrimaryIndexOperationTracker) datasetOpTrackers
+                    .get(dsInfo.datasetID);
+            if (opTracker != null && opTracker.getNumActiveOperations() == 0 && dsInfo.referenceCount == 0
+                    && dsInfo.isOpen) {
+
+                // First wait for any ongoing IO operations
+                while (dsInfo.numActiveIOOps > 0) {
+                    try {
+                        wait();
+                    } catch (InterruptedException e) {
+                        throw new HyracksDataException(e);
+                    }
+                }
+
+                for (IndexInfo iInfo : dsInfo.indexes.values()) {
+                    // TODO: This is not efficient since we flush the indexes sequentially. 
+                    // Think of a way to allow submitting the flush requests concurrently. We don't do them concurrently because this
+                    // may lead to a deadlock scenario between the DatasetLifeCycleManager and the PrimaryIndexOperationTracker.
+                    flushAndWaitForIO(dsInfo, iInfo);
+                }
+
                 for (IndexInfo iInfo : dsInfo.indexes.values()) {
                     if (iInfo.isOpen) {
-                        iInfo.index.deactivate(true);
+                        iInfo.index.deactivate(false);
                         iInfo.isOpen = false;
                     }
                     assert iInfo.referenceCount == 0;
                 }
-
-                IVirtualBufferCache vbc = getVirtualBufferCache(dsInfo.datasetID);
-                used -= vbc.getNumPages() * vbc.getPageSize();
                 dsInfo.isOpen = false;
+
+                List<IVirtualBufferCache> vbcs = getVirtualBufferCaches(dsInfo.datasetID);
+                for (IVirtualBufferCache vbc : vbcs) {
+                    used -= vbc.getNumPages() * vbc.getPageSize();
+                }
                 return true;
+
             }
         }
         return false;
     }
 
+    private void flushAndWaitForIO(DatasetInfo dsInfo, IndexInfo iInfo) throws HyracksDataException {
+        if (iInfo.isOpen) {
+            ILSMIndexAccessor accessor = (ILSMIndexAccessor) iInfo.index.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            accessor.scheduleFlush(((BaseOperationTracker) iInfo.index.getOperationTracker()).getIOOperationCallback());
+        }
+        // Wait for the above flush op.
+        while (dsInfo.numActiveIOOps > 0) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                throw new HyracksDataException(e);
+            }
+        }
+    }
+
     @Override
     public synchronized void close(long resourceID) throws HyracksDataException {
         int did = getDIDfromRID(resourceID);
@@ -213,15 +293,22 @@
         return openIndexes;
     }
 
-    public IVirtualBufferCache getVirtualBufferCache(int datasetID) {
+    public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID) {
         synchronized (datasetVirtualBufferCaches) {
-            MultitenantVirtualBufferCache vbc = datasetVirtualBufferCaches.get(datasetID);
-            if (vbc == null) {
-                vbc = new MultitenantVirtualBufferCache(new VirtualBufferCache(new HeapBufferAllocator(),
-                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages()));
-                datasetVirtualBufferCaches.put(datasetID, vbc);
+            List<IVirtualBufferCache> vbcs = datasetVirtualBufferCaches.get(datasetID);
+            if (vbcs == null) {
+                vbcs = new ArrayList<IVirtualBufferCache>();
+                for (int i = 0; i < storageProperties.getMemoryComponentsNum(); i++) {
+                    MultitenantVirtualBufferCache vbc = new MultitenantVirtualBufferCache(
+                            new VirtualBufferCache(new HeapBufferAllocator(),
+                                    storageProperties.getMemoryComponentPageSize(),
+                                    storageProperties.getMemoryComponentNumPages()
+                                            / storageProperties.getMemoryComponentsNum()));
+                    vbcs.add(vbc);
+                }
+                datasetVirtualBufferCaches.put(datasetID, vbcs);
             }
-            return vbc;
+            return vbcs;
         }
     }
 
@@ -245,7 +332,9 @@
         }
         Set<ILSMIndex> datasetIndexes = new HashSet<ILSMIndex>();
         for (IndexInfo iInfo : dsInfo.indexes.values()) {
-            datasetIndexes.add(iInfo.index);
+            if (iInfo.isOpen) {
+                datasetIndexes.add(iInfo.index);
+            }
         }
         return datasetIndexes;
     }
@@ -280,6 +369,7 @@
         private final Map<Long, IndexInfo> indexes;
         private final int datasetID;
         private long lastAccess;
+        private int numActiveIOOps;
 
         public DatasetInfo(int datasetID) {
             this.indexes = new HashMap<Long, IndexInfo>();
@@ -297,6 +387,14 @@
             lastAccess = System.currentTimeMillis();
         }
 
+        public void incrementActiveIOOps() {
+            numActiveIOOps++;
+        }
+
+        public void decrementActiveIOOps() {
+            numActiveIOOps--;
+        }
+
         @Override
         public int compareTo(DatasetInfo i) {
             // sort by (isOpen, referenceCount, lastAccess) ascending, where true < false
@@ -376,4 +474,4 @@
         }
         outputStream.write(sb.toString().getBytes());
     }
-}
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
index 53b9192..2ed4b0ec 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -18,7 +18,6 @@
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 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;
@@ -26,35 +25,27 @@
 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.IVirtualBufferCache;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 
 public class PrimaryIndexOperationTracker extends BaseOperationTracker {
 
-    private final DatasetLifecycleManager datasetLifecycleManager;
-    private final IVirtualBufferCache datasetBufferCache;
-    private final int datasetID;
-    // Number of active operations on a ILSMIndex instance.
-    private AtomicInteger numActiveOperations;
+    // Number of active operations on an ILSMIndex instance.
+    private final AtomicInteger numActiveOperations;
 
     public PrimaryIndexOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID,
             ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
-        super(ioOpCallbackFactory);
-        this.datasetLifecycleManager = datasetLifecycleManager;
-        this.numActiveOperations = new AtomicInteger(0);
-        this.datasetID = datasetID;
-        datasetBufferCache = datasetLifecycleManager.getVirtualBufferCache(datasetID);
+        super(datasetLifecycleManager, ioOpCallbackFactory, datasetID);
+        this.numActiveOperations = new AtomicInteger();
     }
 
     @Override
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        numActiveOperations.incrementAndGet();
-
-        // Increment transactor-local active operations count.
-        AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
-        if (opCallback != null) {
-            opCallback.incrementLocalNumActiveOperations();
+        if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+            numActiveOperations.incrementAndGet();
+        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+            datasetLifecycleManager.declareActiveIOOperation(datasetID);
         }
     }
 
@@ -62,8 +53,7 @@
     public void afterOperation(ILSMIndex index, 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 && searchCallback != NoOpOperationCallback.INSTANCE)
-                || opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+        if (opType == LSMOperationType.SEARCH || opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
             completeOperation(index, opType, searchCallback, modificationCallback);
         }
     }
@@ -71,50 +61,46 @@
     @Override
     public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        int nActiveOps = numActiveOperations.decrementAndGet();
-        // Decrement transactor-local active operations count.
-        AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
-        if (opCallback != null) {
-            opCallback.decrementLocalNumActiveOperations();
+        if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+            numActiveOperations.decrementAndGet();
+        } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
+            datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
         }
-        if (opType != LSMOperationType.FLUSH) {
-            flushIfFull(nActiveOps);
+
+        if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
+            flushIfFull();
         }
     }
 
-    private void flushIfFull(int nActiveOps) throws HyracksDataException {
-        // If we need a flush, and this is the last completing operation, then schedule the flush.
-        if (datasetBufferCache.isFull() && nActiveOps == 0) {
-            Set<ILSMIndex> indexes = datasetLifecycleManager.getDatasetIndexes(datasetID);
-            for (ILSMIndex lsmIndex : indexes) {
-                ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
-                        NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-                accessor.scheduleFlush(((BaseOperationTracker) lsmIndex.getOperationTracker()).getIOOperationCallback());
+    private void flushIfFull() throws HyracksDataException {
+        // If we need a flush, and this is the last completing operation, then schedule the flush. 
+        // TODO: Is there a better way to check if we need to flush instead of communicating with the datasetLifecycleManager each time?
+        Set<ILSMIndex> indexes = datasetLifecycleManager.getDatasetIndexes(datasetID);
+        boolean needsFlush = false;
+        for (ILSMIndex lsmIndex : indexes) {
+            if (((ILSMIndexInternal) lsmIndex).hasFlushRequestForCurrentMutableComponent()) {
+                needsFlush = true;
+                break;
             }
-
+        }
+        synchronized (this) {
+            if (needsFlush && numActiveOperations.get() == 0) {
+                for (ILSMIndex lsmIndex : indexes) {
+                    ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
+                            NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
+                    accessor.scheduleFlush(((BaseOperationTracker) lsmIndex.getOperationTracker())
+                            .getIOOperationCallback());
+                }
+            }
         }
     }
 
     public void exclusiveJobCommitted() throws HyracksDataException {
         numActiveOperations.set(0);
-        flushIfFull(0);
-    }
-
-    private AbstractOperationCallback getOperationCallback(ISearchOperationCallback searchCallback,
-            IModificationOperationCallback modificationCallback) {
-
-        if (modificationCallback == NoOpOperationCallback.INSTANCE || modificationCallback == null) {
-            return null;
-        }
-        if (searchCallback != null && searchCallback != NoOpOperationCallback.INSTANCE) {
-            return (AbstractOperationCallback) searchCallback;
-        } else {
-            return (AbstractOperationCallback) modificationCallback;
-        }
+        flushIfFull();
     }
 
     public int getNumActiveOperations() {
         return numActiveOperations.get();
     }
-
-}
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
index 129a1a7..bc19543 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/dataflow/AsterixLSMInsertDeleteOperatorNodePushable.java
@@ -47,7 +47,6 @@
                 if (tupleFilter != null) {
                     frameTuple.reset(accessor, i);
                     if (!tupleFilter.accept(frameTuple)) {
-                        lsmAccessor.noOp();
                         continue;
                     }
                 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
index fcc6dad..da08cd8 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -45,7 +45,7 @@
         opTracker.resetLSNs();
     }
 
-    protected abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
+    public abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
 
     protected void putLSNIntoMetadata(ITreeIndex treeIndex, List<ILSMComponent> oldComponents)
             throws HyracksDataException {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index 40377f4..b6025cb 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -20,7 +20,7 @@
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 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.btree.impls.LSMBTreeDiskComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 
 public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
@@ -33,21 +33,21 @@
     public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
         if (oldComponents != null && newComponent != null) {
-            LSMBTreeImmutableComponent btreeComponent = (LSMBTreeImmutableComponent) newComponent;
+            LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) newComponent;
             putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
         }
     }
 
     @Override
-    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
-        if (oldComponents == null) {
+    public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents == null) {
             // Implies a flush IO operation.
             return opTracker.getLastLSN();
         }
-        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
-        for (ILSMComponent c : oldComponents) {
-            BTree btree = ((LSMBTreeImmutableComponent) c).getBTree();
+        for (ILSMComponent c : diskComponents) {
+            BTree btree = ((LSMBTreeDiskComponent) c).getBTree();
             maxLSN = Math.max(getTreeIndexLSN(btree), maxLSN);
         }
         return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
index eb9878c..4f99ae6 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -20,7 +20,7 @@
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 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;
+import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponent;
 
 public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
 
@@ -32,21 +32,21 @@
     public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
         if (oldComponents != null && newComponent != null) {
-            LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) newComponent;
+            LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) newComponent;
             putLSNIntoMetadata(invIndexComponent.getDeletedKeysBTree(), oldComponents);
         }
     }
 
     @Override
-    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
-        if (oldComponents == null) {
+    public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents == null) {
             // Implies a flush IO operation.
             return opTracker.getLastLSN();
         }
-        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
-        for (Object o : oldComponents) {
-            LSMInvertedIndexImmutableComponent invIndexComponent = (LSMInvertedIndexImmutableComponent) o;
+        for (Object o : diskComponents) {
+            LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) o;
             maxLSN = Math.max(getTreeIndexLSN(invIndexComponent.getDeletedKeysBTree()), maxLSN);
         }
         return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
index 63016f1..cd7b7a0 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -20,7 +20,7 @@
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 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;
+import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponent;
 
 public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
 
@@ -32,22 +32,22 @@
     public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
         if (oldComponents != null && newComponent != null) {
-            LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) newComponent;
+            LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) newComponent;
             putLSNIntoMetadata(rtreeComponent.getRTree(), oldComponents);
             putLSNIntoMetadata(rtreeComponent.getBTree(), oldComponents);
         }
     }
 
     @Override
-    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
-        if (oldComponents == null) {
+    public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents == null) {
             // Implies a flush IO operation.
             return opTracker.getLastLSN();
         }
-        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
-        for (Object o : oldComponents) {
-            LSMRTreeImmutableComponent rtreeComponent = (LSMRTreeImmutableComponent) o;
+        for (Object o : diskComponents) {
+            LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) o;
             maxLSN = Math.max(getTreeIndexLSN(rtreeComponent.getRTree()), maxLSN);
         }
         return maxLSN;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
index c951826..d4b26f7 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
@@ -15,8 +15,6 @@
 
 package edu.uci.ics.asterix.common.transactions;
 
-import java.util.concurrent.atomic.AtomicInteger;
-
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
 
@@ -28,7 +26,6 @@
     protected final int[] primaryKeyFields;
     protected final ITransactionContext txnCtx;
     protected final ILockManager lockManager;
-    protected final AtomicInteger transactorLocalNumActiveOperations;
     protected final long[] longHashes;
 
     public AbstractOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
@@ -37,7 +34,6 @@
         this.primaryKeyFields = primaryKeyFields;
         this.txnCtx = txnCtx;
         this.lockManager = lockManager;
-        this.transactorLocalNumActiveOperations = new AtomicInteger(0);
         this.longHashes = new long[2];
     }
 
@@ -45,16 +41,4 @@
         MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
         return Math.abs((int) longHashes[0]);
     }
-
-    public int getLocalNumActiveOperations() {
-        return transactorLocalNumActiveOperations.get();
-    }
-
-    public void incrementLocalNumActiveOperations() {
-        transactorLocalNumActiveOperations.incrementAndGet();
-    }
-
-    public void decrementLocalNumActiveOperations() {
-        transactorLocalNumActiveOperations.decrementAndGet();
-    }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
deleted file mode 100644
index a10bf08..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
-    ByteBuffer buffer;
-
-    public Buffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    protected Buffer() {
-    }
-
-    public void setBuffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    @Override
-    public byte getByte(int offset) {
-        return buffer.get(offset);
-    }
-
-    @Override
-    public byte getByte() {
-        return buffer.get();
-    }
-
-    @Override
-    public void getBytes(byte[] bytes, int offset, int size) {
-        System.arraycopy(buffer.array(), offset, bytes, 0, size);
-    }
-
-    @Override
-    public int getSize() {
-        return buffer.capacity();
-    }
-
-    @Override
-    public int readInt() {
-        return buffer.getInt();
-    }
-
-    @Override
-    public int readInt(int offset) {
-        return buffer.getInt(offset);
-    }
-
-    @Override
-    public long readLong(int offset) {
-        return buffer.getLong(offset);
-    }
-
-    @Override
-    public void put(byte b) {
-        buffer.put(b);
-    }
-
-    @Override
-    public void put(int offset, byte b) {
-        buffer.put(offset, b);
-    }
-
-    @Override
-    public void put(byte[] bytes, int start, int length) {
-        buffer.put(bytes, start, length);
-
-    }
-
-    @Override
-    public void put(byte[] bytes) {
-        buffer.put(bytes);
-    }
-
-    @Override
-    public void writeInt(int value) {
-        buffer.putInt(value);
-    }
-
-    @Override
-    public void writeInt(int offset, int value) {
-        buffer.putInt(offset, value);
-
-    }
-
-    @Override
-    public void writeLong(long value) {
-        buffer.putLong(value);
-    }
-
-    @Override
-    public void writeLong(int offset, long value) {
-        buffer.putLong(offset, value);
-
-    }
-
-    @Override
-    public byte[] getArray() {
-        return buffer.array();
-    }
-
-    @Override
-    public void erase() {
-        Arrays.fill(buffer.array(), (byte) 0);
-    }
-
-    @Override
-    public ByteBuffer getByteBuffer() {
-        return buffer;
-    }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
deleted file mode 100644
index fbe9b4c..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-/**
- * Represent a buffer that is backed by a physical file. Provider custom APIs
- * for accessing a chunk of the underlying file.
- */
-public class FileBasedBuffer extends Buffer implements IFileBasedBuffer {
-
-    private String filePath;
-    private FileChannel fileChannel;
-    private RandomAccessFile raf;
-    private int bufferSize;
-
-    private int bufferLastFlushOffset;
-    private int bufferNextWriteOffset;
-    private final int diskSectorSize;
-
-    private final ReadWriteLock latch;
-    private final AtomicInteger referenceCount;
-
-    public FileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
-        this.filePath = filePath;
-        buffer = ByteBuffer.allocate(bufferSize);
-        raf = new RandomAccessFile(new File(filePath), "rw");
-        fileChannel = raf.getChannel();
-        fileChannel.position(offset);
-        fileChannel.read(buffer);
-        buffer.position(0);
-        this.bufferSize = bufferSize;
-        buffer.limit(bufferSize);
-        bufferLastFlushOffset = 0;
-        bufferNextWriteOffset = 0;
-        this.diskSectorSize = diskSectorSize;
-        latch = new ReentrantReadWriteLock(true);
-        referenceCount = new AtomicInteger(0);
-    }
-
-    public String getFilePath() {
-        return filePath;
-    }
-
-    public void setFilePath(String filePath) {
-        this.filePath = filePath;
-    }
-
-    @Override
-    public int getSize() {
-        return bufferSize;
-    }
-
-    public void clear() {
-        buffer.clear();
-    }
-
-    @Override
-    public void flush() throws IOException {
-        //flush
-        int pos = bufferLastFlushOffset;
-        int limit = (((bufferNextWriteOffset - 1) / diskSectorSize) + 1) * diskSectorSize;
-        buffer.position(pos);
-        buffer.limit(limit);
-        fileChannel.write(buffer);
-        fileChannel.force(false);
-
-        //update variables
-        bufferLastFlushOffset = limit;
-        bufferNextWriteOffset = limit;
-        buffer.limit(bufferSize);
-    }
-
-    @Override
-    public void put(byte b) {
-        buffer.put(b);
-    }
-
-    @Override
-    public void put(int offset, byte b) {
-        buffer.put(offset, b);
-    }
-
-    @Override
-    public void put(byte[] bytes, int start, int length) {
-        buffer.put(bytes, start, length);
-    }
-
-    @Override
-    public void put(byte[] bytes) {
-        buffer.put(bytes);
-    }
-
-    @Override
-    public void writeInt(int value) {
-        buffer.putInt(value);
-    }
-
-    @Override
-    public void writeInt(int index, int value) {
-        buffer.putInt(index, value);
-    }
-
-    @Override
-    public void writeLong(long value) {
-        buffer.putLong(value);
-    }
-
-    @Override
-    public void writeLong(int index, long value) {
-        buffer.putLong(index, value);
-    }
-
-    /**
-     * Resets the buffer with content (size as specified) from a given file
-     * starting at offset.
-     */
-    @Override
-    public void reset(String filePath, long diskNextWriteOffset, int bufferSize) throws IOException {
-        if (!filePath.equals(this.filePath)) {
-            raf.close();//required?
-            fileChannel.close();
-            raf = new RandomAccessFile(filePath, "rw");
-            this.filePath = filePath;
-        }
-        fileChannel = raf.getChannel();
-        fileChannel.position(diskNextWriteOffset);
-        erase();
-        buffer.position(0);
-        buffer.limit(bufferSize);
-        this.bufferSize = bufferSize;
-
-        bufferLastFlushOffset = 0;
-        bufferNextWriteOffset = 0;
-    }
-
-    @Override
-    public void close() throws IOException {
-        fileChannel.close();
-    }
-
-    @Override
-    public void open(String filePath, long offset, int bufferSize) throws IOException {
-        raf = new RandomAccessFile(filePath, "rw");
-        fileChannel = raf.getChannel();
-        fileChannel.position(offset);
-        erase();
-        buffer.position(0);
-        buffer.limit(bufferSize);
-        this.bufferSize = bufferSize;
-        bufferLastFlushOffset = 0;
-        bufferNextWriteOffset = 0;
-    }
-
-    @Override
-    public long getDiskNextWriteOffset() throws IOException {
-        return fileChannel.position();
-    }
-
-    @Override
-    public void setDiskNextWriteOffset(long offset) throws IOException {
-        fileChannel.position(offset);
-    }
-
-    @Override
-    public int getBufferLastFlushOffset() {
-        return bufferLastFlushOffset;
-    }
-
-    @Override
-    public void setBufferLastFlushOffset(int offset) {
-        this.bufferLastFlushOffset = offset;
-    }
-
-    @Override
-    public int getBufferNextWriteOffset() {
-        synchronized (fileChannel) {
-            return bufferNextWriteOffset;
-        }
-    }
-
-    @Override
-    public void setBufferNextWriteOffset(int offset) {
-        synchronized (fileChannel) {
-            if (bufferNextWriteOffset < offset) {
-                bufferNextWriteOffset = offset;
-            }
-        }
-    }
-
-    @Override
-    public void acquireWriteLatch() {
-        latch.writeLock().lock();
-    }
-
-    @Override
-    public void releaseWriteLatch() {
-        latch.writeLock().unlock();
-    }
-
-    @Override
-    public void acquireReadLatch() {
-        latch.readLock().lock();
-    }
-
-    @Override
-    public void releaseReadLatch() {
-        latch.readLock().unlock();
-    }
-
-    @Override
-    public void incRefCnt() {
-        referenceCount.incrementAndGet();
-    }
-    
-    @Override
-    public void decRefCnt() {
-        referenceCount.decrementAndGet();
-    }
-    
-    @Override
-    public int getRefCnt() {
-        return referenceCount.get();
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
deleted file mode 100644
index 64beb86..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * A utility class that provides operations on files such as creation and
- * loading content in a buffer. It also provides API for creating directories
- */
-public class FileUtil {
-
-    public static final String lineSeparator = System.getProperty("line.separator");
-
-    public static boolean createFileIfNotExists(String path) throws IOException {
-        File file = new File(path);
-        File parentFile = file.getParentFile();
-        if (parentFile != null) {
-            parentFile.mkdirs();
-        }
-        return file.createNewFile();
-    }
-
-    public static boolean createNewDirectory(String path) throws IOException {
-        return (new File(path)).mkdir();
-    }
-
-    public static IFileBasedBuffer getFileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
-        IFileBasedBuffer fileBasedBuffer = new FileBasedBuffer(filePath, offset, bufferSize, diskSectorSize);
-        return fileBasedBuffer;
-    }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
index 05ac025..2f522b9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
@@ -14,6 +14,8 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
+import java.util.List;
+
 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;
@@ -58,5 +60,5 @@
 
     public IIOManager getIOManager();
 
-    public IVirtualBufferCache getVirtualBufferCache(int datasetID);
+    public List<IVirtualBufferCache> getVirtualBufferCaches(int datasetID);
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
deleted file mode 100644
index fd4af81..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support.
- */
-public interface IBuffer {
-
-    public byte getByte(int offset);
-
-    public int getSize();
-
-    public byte getByte();
-
-    public void getBytes(byte[] bytes, int start, int size);
-
-    public int readInt();
-
-    public int readInt(int offset);
-
-    public long readLong(int offset);
-
-    public void put(byte b);
-
-    public void put(int offset, byte b);
-
-    public void put(byte[] bytes, int start, int length);
-
-    public void put(byte[] bytes);
-
-    public void writeInt(int value);
-
-    public void writeLong(long value);
-
-    public void writeInt(int offset, int value);
-
-    public void writeLong(int offset, long value);
-
-    public byte[] getArray();
-
-    public void erase();
-
-    public ByteBuffer getByteBuffer();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
deleted file mode 100644
index 1c8cb76..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * Represents a closeable resource that implements a close(@see
- * TransactionContext) method. When a transaction commits/aborts, the close
- * method is called on each of the ICloseable resources that were involved in
- * the transaction. This gives an opportunity to release all resources and do a
- * cleanup. An example of ICloseable is the @see TreeLogger.
- */
-public interface ICloseable {
-
-    /**
-     * This method is invoked at the commit/abort of a transaction that involved
-     * a ICloseable resource. It is used to do a clean up by the involved
-     * resource before the transaction ends.
-     * 
-     * @param context
-     * @throws ACIDException
-     */
-    public void close(ITransactionContext context) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
deleted file mode 100644
index 1d466e3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.io.IOException;
-
-/**
- * Represent a buffer that is backed by a physical file. Provides custom APIs
- * for accessing a chunk of the underlying file.
- */
-
-public interface IFileBasedBuffer extends IBuffer {
-
-    public void flush() throws IOException;
-
-    /**
-     * Resets the buffer with content (size as specified) from a given file
-     * starting at offset.
-     */
-    public void reset(String filePath, long offset, int size) throws IOException;
-
-    public long getDiskNextWriteOffset() throws IOException;
-
-    public void setDiskNextWriteOffset(long writePosition) throws IOException;
-
-    public void close() throws IOException;
-    
-    public void open(String filePath, long offset, int size) throws IOException;
-
-    public int getBufferLastFlushOffset();
-
-    public void setBufferLastFlushOffset(int offset);
-
-    public int getBufferNextWriteOffset();
-
-    public void setBufferNextWriteOffset(int offset);
-    
-    public void acquireWriteLatch();
-
-    public void releaseWriteLatch();
-
-    public void acquireReadLatch();
-
-    public void releaseReadLatch();
-    
-    public void incRefCnt();
-    
-    public void decRefCnt();
-    
-    public int getRefCnt();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
index 698a788..a752afa 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
@@ -64,20 +64,9 @@
      * @param txnContext
      * @throws ACIDException
      *             TODO
-     * @return true if the lock count is 0, false otherwise.
+     * @return 
      */
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
-            throws ACIDException;
-
-    /**
-     * @param datasetId
-     * @param entityHashValue
-     * @param txnContext
-     * @throws ACIDException
-     *             TODO
-     * @return true if the lock count is 0, false otherwise.
-     */
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
+    public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
             throws ACIDException;
 
     /**
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
deleted file mode 100644
index 5c5c149..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * 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.common.transactions;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a predicate or a filter that needs to be applied when selectively
- * retrieving logs.
- */
-public interface ILogFilter {
-
-    public boolean accept(IBuffer buffer, long startOffset, int length);
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 9cc4d0d..8913f8a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -14,73 +14,10 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
 public interface ILogManager {
 
-    /**
-     * @param logType
-     * @param context
-     * @param datasetId
-     * @param PKHashValue
-     * @param resourceId
-     * @param resourceMgrId
-     * @param logContentSize
-     * @param reusableLogContentObject
-     * @param logger
-     * @param logicalLogLocator
-     * @throws ACIDException
-     */
-    void log(byte logType, ITransactionContext context, int datasetId, int PKHashValue, long resourceId,
-            byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
-            LogicalLogLocator logicalLogLocator) throws ACIDException;
+    public void log(ILogRecord logRecord);
 
-    /**
-     * @param physicalLogLocator
-     *            specifies the physical location from where the logs need to be
-     *            read
-     * @param logFilter
-     *            specifies the filtering criteria for the retrieved logs
-     * @return LogCursor an iterator for the retrieved logs
-     * @throws ACIDException
-     */
-    public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
-            ACIDException;
-
-    /**
-     * @param lsnValue
-     *            TODO
-     * @param logicalLogLocator
-     *            TODO
-     * @throws ACIDException
-     */
-    public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
-    /**
-     * Retrieves the configuration parameters of the ILogManager
-     * 
-     * @return LogManagerProperties: the configuration parameters for the
-     *         ILogManager
-     */
-    public LogManagerProperties getLogManagerProperties();
-
-    /**
-     * Returns the ILogRecordHelper instance associated with this ILogManager
-     * instance
-     * 
-     * @return ILogRecordHelper: the utility (class) for writing/reading log
-     *         header.
-     */
-    public ILogRecordHelper getLogRecordHelper();
-
-    /**
-     * Returns the Transaction Provider associated with this ILogManager
-     * instance
-     * 
-     * @return TransactionSubsystem
-     */
-    public ITransactionSubsystem getTransactionSubsystem();
+    public ILogReader getLogReader(boolean isRecoveryMode);
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
similarity index 78%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
index 08940a7..be205a9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
@@ -14,9 +14,10 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
+public interface ILogPage {
 
-public interface ILoggerRepository {
+    public void append(ILogRecord logRecord, long appendLsn);
 
-    public ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException;
-}
+    public void flush();
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
similarity index 70%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
index e455d3cc..fa92865 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
@@ -14,18 +14,18 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import java.io.IOException;
-
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 
-/**
- * Provides a cursor over the logs created to date.
- */
-public interface ILogCursor {
+public interface ILogReader {
 
-    public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException;
+    public void initializeScan(long beginLSN) throws ACIDException;
 
-    public ILogFilter getLogFilter();
+    //for scanning
+    public ILogRecord next() throws ACIDException;
 
-}
+    //for random reading
+    public ILogRecord read(long readLSN) throws ACIDException;
+
+    public void close() throws ACIDException;
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
new file mode 100644
index 0000000..d13ef6c
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
@@ -0,0 +1,122 @@
+/*
+ * 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.common.transactions;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ILogRecord {
+
+    public static final int JOB_COMMIT_LOG_SIZE = 13;
+    public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 29;
+    public static final int UPDATE_LOG_BASE_SIZE = 64;
+
+    public boolean readLogRecord(ByteBuffer buffer);
+
+    public void writeLogRecord(ByteBuffer buffer);
+
+    public void formJobCommitLogRecord(ITransactionContext txnCtx);
+
+    public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
+            ITupleReference tupleReference, int[] primaryKeyFields);
+
+    public ITransactionContext getTxnCtx();
+
+    public void setTxnCtx(ITransactionContext txnCtx);
+
+    public boolean isFlushed();
+
+    public void isFlushed(boolean isFlushed);
+
+    public byte getLogType();
+
+    public void setLogType(byte logType);
+
+    public int getJobId();
+
+    public void setJobId(int jobId);
+
+    public int getDatasetId();
+
+    public void setDatasetId(int datasetId);
+
+    public int getPKHashValue();
+
+    public void setPKHashValue(int PKHashValue);
+
+    public long getPrevLSN();
+
+    public void setPrevLSN(long prevLsn);
+
+    public long getResourceId();
+
+    public void setResourceId(long resourceId);
+
+    public byte getResourceType();
+
+    public void setResourceType(byte resourceType);
+
+    public int getLogSize();
+
+    public void setLogSize(int logSize);
+
+    public byte getNewOp();
+
+    public void setNewOp(byte newOp);
+
+    public int getNewValueSize();
+
+    public void setNewValueSize(int newValueSize);
+
+    public ITupleReference getNewValue();
+
+    public void setNewValue(ITupleReference newValue);
+
+    public byte getOldOp();
+
+    public void setOldOp(byte oldOp);
+
+    public int getOldValueSize();
+
+    public void setOldValueSize(int oldValueSize);
+
+    public ITupleReference getOldValue();
+
+    public void setOldValue(ITupleReference oldValue);
+
+    public long getChecksum();
+
+    public void setChecksum(long checksum);
+
+    public long getLSN();
+
+    public void setLSN(long LSN);
+
+    public String getLogRecordForDisplay();
+
+    public void computeAndSetLogSize();
+
+    public int getPKValueSize();
+
+    public ITupleReference getPKValue();
+
+    public void setPKFields(int[] primaryKeyFields);
+
+    public void computeAndSetPKValueSize();
+
+    public void setPKValue(ITupleReference PKValue);
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
deleted file mode 100644
index a299a0d..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.common.transactions;
-
-/**
- * Helper class 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.
- */
-
-public interface ILogRecordHelper {
-
-    public byte getLogType(LogicalLogLocator logicalLogLocator);
-
-    public int getJobId(LogicalLogLocator logicalLogLocator);
-
-    public int getDatasetId(LogicalLogLocator logicalLogLocator);
-
-    public int getPKHashValue(LogicalLogLocator logicalLogLocator);
-
-    public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
-
-    public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
-
-    public long getResourceId(LogicalLogLocator logicalLogLocator);
-
-    public byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
-
-    public int getLogContentSize(LogicalLogLocator logicalLogLocater);
-
-    public long getLogChecksum(LogicalLogLocator logicalLogLocator);
-
-    public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
-
-    public int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
-
-    public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
-
-    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
-            int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
-            int logRecordSize);
-
-    public boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
-
-    public int getLogRecordSize(byte logType, int logBodySize);
-
-    public int getLogHeaderSize(byte logType);
-
-    public int getLogChecksumSize();
-
-    public int getCommitLogSize();
-
-}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
deleted file mode 100644
index 939f8e5..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * 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.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * An interface providing call back APIs that are invoked {@link ILogManager} for providing the content for the log record and doing any pre/post
- * processing.
- */
-public interface ILogger {
-
-    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
-            throws ACIDException;
-
-    public void log(ITransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
-            ReusableLogContentObject reusableLogContentObject) throws ACIDException;
-
-    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
-            throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
index be41a6b..6d2f3cb 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
@@ -17,7 +17,7 @@
 import java.io.IOException;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 
 /**
  * Provides API for failure recovery. Failure could be at application level and
@@ -33,6 +33,12 @@
         HEALTHY,
         CORRUPTED
     }
+    
+    public class ResourceType {
+        public static final byte LSM_BTREE = 0;
+        public static final byte LSM_RTREE = 1;
+        public static final byte LSM_INVERTED_INDEX = 2;
+    }
 
     /**
      * Returns the state of the system.
@@ -69,5 +75,5 @@
      */
     public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException;
 
-    public void checkpoint(boolean isSharpCheckpoint) throws ACIDException;
+    public void checkpoint(boolean isSharpCheckpoint) throws ACIDException, HyracksDataException;
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
deleted file mode 100644
index 36ce317..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * 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.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-
-/**
- * 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.
-     * 
-     * @return a unique identifier for the resource manager. The number of
-     *         resource managers in the system are expected to be handful and
-     *         can be uniquely identified by using a single byte as an id.
-     */
-    public byte getResourceManagerId();
-
-    /**
-     * Undo the operation corresponding to a given log record.
-     * 
-     * @param logRecordHelper
-     *            (@see ILogRecordHelper) An implementation of the
-     *            ILogRecordHelper interface that is used to parse the log
-     *            record and extract useful information from the content.
-     * @param LogicalLogLocator
-     *            (@see LogicalLogLocator) The locationof the log record that
-     *            needs to be undone.
-     * @throws ACIDException
-     */
-    public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
-    /**
-     * Redo the operation corresponding to a given log record.
-     * 
-     * @param logRecordHelper
-     *            (@see ILogRecordHelper) An implementation of the
-     *            ILogRecordHelper interface that is used to parse the log
-     *            record and extract useful information from the content.
-     * @param LogicalLogLocator
-     *            (@see LogicalLogLocator) The locationof the log record that
-     *            needs to be undone.
-     * @throws ACIDException
-     */
-    public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
index 3ebb963..dc33e69 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
@@ -14,61 +14,38 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
 public interface ITransactionContext {
 
-    public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback);
-
-    public void updateLastLSNForIndexes(long lastLSN);
-
-    public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException;
-
-    public int getActiveOperationCountOnIndexes() throws HyracksDataException;
-
-    public LogicalLogLocator getFirstLogLocator();
-
-    public LogicalLogLocator getLastLogLocator();
-
-    public void addCloseableResource(ICloseable resource);
+    public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
+            boolean isPrimaryIndex);
 
     public JobId getJobId();
 
-    public void setStartWaitTime(long time);
+    public void setTimeout(boolean isTimeout);
 
-    public long getStartWaitTime();
+    public boolean isTimeout();
 
-    public void setStatus(int status);
+    public void setTxnState(int txnState);
 
-    public int getStatus();
+    public int getTxnState();
 
-    public void setTxnState(TransactionState txnState);
+    public long getFirstLSN();
 
-    public TransactionState getTxnState();
+    public long getLastLSN();
 
-    public void releaseResources() throws ACIDException;
+    public void setLastLSN(long resourceId, long LSN);
 
-    public void setLastLSN(long lsn);
+    public boolean isWriteTxn();
 
-    public TransactionType getTransactionType();
-
-    public void setTransactionType(TransactionType transactionType);
+    public void setWriteTxn(boolean isWriterTxn);
 
     public String prettyPrint();
 
-    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_STATUS = 1;
+    public void setMetadataTransaction(boolean isMetadataTxn);
 
-    public enum TransactionType {
-        READ,
-        READ_WRITE
-    }
+    public boolean isMetadataTransaction();
 
-    public void setExclusiveJobLevelCommit();
-
+    public void notifyOptracker(boolean isJobLevelCommit);
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
index 9a99dc9..77e960b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
@@ -29,12 +29,10 @@
      * transaction has committed. ABORTED: The transaction has aborted.
      * TIMED_OUT: The transaction has timed out waiting to acquire a lock.
      */
-    public enum TransactionState {
-        ACTIVE,
-        COMMITTED,
-        ABORTED,
-        TIMED_OUT,
-    };
+    public static final int ACTIVE = 0;
+    public static final int COMMITTED = 1;
+    public static final int ABORTED = 2;
+    public static final int TIMED_OUT = 3;
 
     /**
      * Begins a transaction identified by a transaction id and returns the
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
index 63ee5d0..b6c934a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-
 public interface ITransactionSubsystem {
 
     public ILogManager getLogManager();
@@ -25,10 +24,6 @@
 
     public IRecoveryManager getRecoveryManager();
 
-    public TransactionalResourceManagerRepository getTransactionalResourceRepository();
-
-    public ILoggerRepository getTreeLoggerRepository();
-
     public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
 
     public String getId();
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
index 4b5eb9b..dd1e7b4 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
@@ -22,6 +22,7 @@
 
     private static final long serialVersionUID = 2084227360840799662L;
 
+    public static final String lineSeparator = System.getProperty("line.separator");
     public static final int LOG_MAGIC_NUMBER = 123456789;
     public static final String LOG_DIR_SUFFIX = ".txnLogDir";
     private static final String DEFAULT_LOG_FILE_PREFIX = "asterix_transaction_log";
@@ -35,14 +36,10 @@
     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;
-    // default disk sector size
-    private final int diskSectorSize;
 
     public LogManagerProperties(AsterixTransactionProperties txnProperties, String nodeId) {
         this.logDirKey = new String(nodeId + LOG_DIR_SUFFIX);
@@ -51,12 +48,9 @@
         long logPartitionSize = txnProperties.getLogPartitionSize();
         this.logDir = txnProperties.getLogDirectory(nodeId);
         this.logFilePrefix = DEFAULT_LOG_FILE_PREFIX;
-        this.groupCommitWaitPeriod = txnProperties.getGroupCommitInterval();
-
         this.logBufferSize = logPageSize * numLogPages;
         //make sure that the log partition size is the multiple of log buffer size.
         this.logPartitionSize = (logPartitionSize / logBufferSize) * logBufferSize;
-        this.diskSectorSize = txnProperties.getLogDiskSectorSize();
     }
 
     public long getLogPartitionSize() {
@@ -83,27 +77,17 @@
         return logBufferSize;
     }
 
-    public long getGroupCommitWaitPeriod() {
-        return groupCommitWaitPeriod;
-    }
-
     public String getLogDirKey() {
         return logDirKey;
     }
 
-    public int getDiskSectorSize() {
-        return diskSectorSize;
-    }
-
     public String toString() {
         StringBuilder builder = new StringBuilder();
-        builder.append("log_dir_ : " + logDir + FileUtil.lineSeparator);
-        builder.append("log_file_prefix" + logFilePrefix + FileUtil.lineSeparator);
-        builder.append("log_page_size : " + logPageSize + FileUtil.lineSeparator);
-        builder.append("num_log_pages : " + numLogPages + FileUtil.lineSeparator);
-        builder.append("log_partition_size : " + logPartitionSize + FileUtil.lineSeparator);
-        builder.append("group_commit_wait_period : " + groupCommitWaitPeriod + FileUtil.lineSeparator);
-        builder.append("disk_sector_size : " + diskSectorSize + FileUtil.lineSeparator);
+        builder.append("log_dir_ : " + logDir + lineSeparator);
+        builder.append("log_file_prefix" + logFilePrefix + lineSeparator);
+        builder.append("log_page_size : " + logPageSize + lineSeparator);
+        builder.append("num_log_pages : " + numLogPages + lineSeparator);
+        builder.append("log_partition_size : " + logPartitionSize + lineSeparator);
         return builder.toString();
     }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
deleted file mode 100644
index 58fc62a..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * A utility class providing helper methods for the {@link ILogManager}
- */
-public class LogUtil {
-
-    private static final Logger LOGGER = Logger.getLogger(LogUtil.class.getName());
-
-    // read the log directory and initialize log anchor to point to the
-    // current log partition file and the offset where the log manager shall
-    // continue to insert log records.
-
-    public static PhysicalLogLocator initializeLogAnchor(ILogManager logManager) throws ACIDException {
-        int fileId = 0;
-        long offset = 0;
-        LogManagerProperties logManagerProperties = logManager.getLogManagerProperties();
-        File logDir = new File(logManagerProperties.getLogDir());
-        try {
-            if (logDir.exists()) {
-                List<String> logFiles = getLogFiles(logManagerProperties);
-                if (logFiles == null || logFiles.size() == 0) {
-                    FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
-                    if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
-                    }
-                } else {
-                    File logFile = new File(LogUtil.getLogFilePath(logManagerProperties,
-                            Long.parseLong(logFiles.get(logFiles.size() - 1))));
-                    fileId = logFiles.size() - 1;
-                    offset = logFile.length();
-                }
-            } else {
-                FileUtil.createNewDirectory(logManagerProperties.getLogDir());
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
-                }
-                FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
-                }
-            }
-        } catch (IOException ioe) {
-            throw new ACIDException("Unable to initialize log anchor", ioe);
-        }
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" file id :" + fileId + " offset " + offset);
-        }
-        return new PhysicalLogLocator(fileId, offset, logManager);
-    }
-
-    public static List<String> getLogFiles(final LogManagerProperties logManagerProperties) {
-        File logDir = new File(logManagerProperties.getLogDir());
-        String[] logFiles = new String[0];
-        List<String> logFileCollection = new ArrayList<String>();
-        if (logDir.exists()) {
-            logFiles = logDir.list(new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    if (name.startsWith(logManagerProperties.getLogFilePrefix())) {
-                        return true;
-                    }
-                    return false;
-                }
-            });
-        }
-        for (String logFile : logFiles) {
-            logFileCollection.add(logFile.substring(logManagerProperties.getLogFilePrefix().length() + 1));
-        }
-        Collections.sort(logFileCollection, new Comparator<String>() {
-            @Override
-            public int compare(String arg0, String arg1) {
-                return Integer.parseInt(arg0) - Integer.parseInt(arg1);
-            }
-        });
-        return logFileCollection;
-    }
-
-    public static long getFileId(String logFilePath, LogManagerProperties logManagerProperties) {
-        String logFileName = logFilePath;
-        if (logFilePath.contains(File.separator)) {
-            logFileName = logFilePath.substring(logFilePath.lastIndexOf(File.separator));
-        }
-        return Long.parseLong(logFileName.substring(logFileName.indexOf(logManagerProperties.getLogFilePrefix())));
-    }
-
-    public static String getLogFilePath(LogManagerProperties logManagerProperties, long fileId) {
-        return logManagerProperties.getLogDir() + File.separator + logManagerProperties.getLogFilePrefix() + "_"
-                + fileId;
-    }
-
-    public static LogicalLogLocator getDummyLogicalLogLocator(ILogManager logManager) {
-        LogicalLogLocator logicalLogLocator = new LogicalLogLocator(-1, null, -1, logManager);
-        return logicalLogLocator;
-    }
-
-    /*
-     * given a lsn, get the offset within the log file where the corresponding
-     * log record is (to be) placed.
-     */
-    public static long getFileOffset(ILogManager logManager, long lsn) {
-        return lsn % logManager.getLogManagerProperties().getLogPartitionSize();
-    }
-
-    /*
-     * given a lsn, get the file id that contains the log record.
-     */
-    public static long getFileId(ILogManager logManager, long lsn) {
-        return lsn / logManager.getLogManagerProperties().getLogPartitionSize();
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
deleted file mode 100644
index 9ba7682..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Represents a location of a log record. The location has two parts to it. A
- * LogicalLogLocator extends PhyscialLogLocator and hence can also be used to
- * determine the physical location of the log record on the local filesystem. In
- * addition to the physical location, a LogicalLogLocator also contains a handle
- * to an in-memory buffer and an offset within the buffer where the log record
- * resides.
- */
-public class LogicalLogLocator extends PhysicalLogLocator {
-
-    private IBuffer buffer;
-    private AtomicInteger memoryOffset;
-
-    public LogicalLogLocator(long lsnValue, IBuffer buffer, int bufferOffset, ILogManager logManager) {
-        super(lsnValue, logManager);
-        this.buffer = buffer;
-        this.memoryOffset = new AtomicInteger(bufferOffset);
-
-    }
-
-    public IBuffer getBuffer() {
-        return buffer;
-    }
-
-    public void setBuffer(IBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public int getMemoryOffset() {
-        return memoryOffset.get();
-    }
-
-    public void setMemoryOffset(int memoryOffset) {
-        this.memoryOffset.set(memoryOffset);
-    }
-
-    @Override
-    public String toString() {
-        return super.toString() + " " + "memoryOffset:" + memoryOffset;
-    }
-
-    public boolean checkValidity() {
-        return true;
-    }
-
-    public long increaseMemoryOffset(int delta) {
-        return memoryOffset.addAndGet(delta);
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
similarity index 66%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
index fb865aa..4480aba 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
@@ -14,31 +14,39 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-public class MutableResourceId{
-    long id;
+public class MutableLong {
+    private long val;
 
-    public MutableResourceId(long id) {
-        this.id = id;
+    
+    public MutableLong() {
+    }
+    
+    public MutableLong(long val) {
+        this.val = val;
+    }
+    
+    public void set(long val) {
+        this.val = val;
     }
 
-    public void setId(long id) {
-        this.id = id;
+    public long get() {
+        return val;
     }
-
-    public long getId() {
-        return id;
-    }
-
+    
     @Override
     public int hashCode() {
-        return (int)id;
+        return (int)val;
     }
 
     @Override
     public boolean equals(Object o) {
-        if ((o == null) || !(o instanceof MutableResourceId)) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof MutableLong)) {
             return false;
         }
-        return ((MutableResourceId) o).id == this.id;
+        return ((MutableLong) o).val == val;
     }
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
deleted file mode 100644
index b83a3f3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * Represents the physical location of a log record. The physical location i
- * deciphered from the contained lsn that is broken down into a file id and an
- * offset within the file. The mapping between fileId and the path on the local
- * file system is maintained by the log manager (@see ILogManager) path on the
- * local file system.
- */
-public class PhysicalLogLocator {
-
-    // The log sequence number corresponding to the log record that is being
-    // referred to.
-    private final AtomicLong lsn;
-
-    // Handle to the log manager that wrote the log record.
-    private final ILogManager logManager;
-
-    public static long getLsnValue(long fileId, long offset, ILogManager logManager) {
-        return fileId * logManager.getLogManagerProperties().getLogPartitionSize() + offset;
-    }
-
-    public PhysicalLogLocator(long lsn, ILogManager logManager) {
-        this.lsn = new AtomicLong(lsn);
-        this.logManager = logManager;
-    }
-
-    public PhysicalLogLocator(long fileId, long offset, ILogManager logManager) {
-        this.lsn = new AtomicLong(getLsnValue(fileId, offset, logManager));
-        this.logManager = logManager;
-    }
-
-    @Override
-    public String toString() {
-        return "lsn :" + lsn.get();
-    }
-
-    public long getFileId() {
-        return LogUtil.getFileId(logManager, lsn.get());
-    }
-
-    public boolean compareAndSet(long expect, long update) {
-        return lsn.compareAndSet(expect, update);
-    }
-
-    public long getFileOffset() {
-        return LogUtil.getFileOffset(logManager, lsn.get());
-    }
-
-    public long getLsn() {
-        return lsn.get();
-    }
-
-    public long incrementLsn() {
-        return lsn.incrementAndGet();
-    }
-
-    public long incrementLsn(long delta) {
-        return lsn.addAndGet(delta);
-    }
-
-    public void setLsn(long lsn) {
-        this.lsn.set(lsn);
-    }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
deleted file mode 100644
index b513fad..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.common.transactions;
-
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-
-/**
- * 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-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
deleted file mode 100644
index 64c90f4..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.common.transactions;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * 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-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
index 88c65ff..861cce5 100644
--- a/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
+++ b/asterix-common/src/test/java/edu/uci/ics/asterix/test/aql/TestsUtils.java
@@ -325,6 +325,8 @@
 
         List<CompilationUnit> cUnits = testCaseCtx.getTestCase().getCompilationUnit();
         for (CompilationUnit cUnit : cUnits) {
+            LOGGER.info("Starting [TEST]: " + testCaseCtx.getTestCase().getFilePath() + "/"
+                    + cUnit.getName() + " ... ");
             testFileCtxs = testCaseCtx.getTestFiles(cUnit);
             expectedResultFileCtxs = testCaseCtx.getExpectedResultFiles(cUnit);
 
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index f8d5ea2..ffc0eac 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -153,6 +153,17 @@
 			<artifactId>jdom</artifactId>
 			<version>1.0</version>
 		</dependency>
+		<dependency>
+			<groupId>javax.jdo</groupId>
+			<artifactId>jdo2-api</artifactId>
+			<version>2.3-20090302111651</version>
+		</dependency>
+		<dependency>
+			<groupId>org.apache.hive</groupId>
+			<artifactId>hive-exec</artifactId>
+			<version>0.11.0</version>
+		</dependency>
 	</dependencies>
 
 </project>
+
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
index 4ca3d72..e9d2175 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
@@ -20,12 +20,21 @@
 
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
-
+import org.apache.hadoop.conf.Configuration;
+import edu.uci.ics.asterix.external.dataset.adapter.HDFSAccessByRIDAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.HDFSIndexingAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.om.types.BuiltinType;
 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.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.hdfs.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
 import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -35,76 +44,190 @@
  */
 @SuppressWarnings("deprecation")
 public class HDFSAdapterFactory implements IGenericDatasetAdapterFactory {
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    public static final String HDFS_ADAPTER_NAME = "hdfs";
-    public static final String CLUSTER_LOCATIONS = "cluster-locations";
-    public static transient String SCHEDULER = "hdfs-scheduler";
+	public static final String HDFS_ADAPTER_NAME = "hdfs";
+	public static final String CLUSTER_LOCATIONS = "cluster-locations";
+	public static transient String SCHEDULER = "hdfs-scheduler";
 
-    public static final String KEY_HDFS_URL = "hdfs";
-    public static final String KEY_PATH = "path";
-    public static final String KEY_INPUT_FORMAT = "input-format";
-    public static final String INPUT_FORMAT_TEXT = "text-input-format";
-    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+	public static final String KEY_HDFS_URL = "hdfs";
+	public static final String KEY_PATH = "path";
+	public static final String KEY_INPUT_FORMAT = "input-format";
+	public static final String INPUT_FORMAT_TEXT = "text-input-format";
+	public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+	public static final String INPUT_FORMAT_RC = "rc-input-format";
+	public static final String KEY_DELIMITER = "delimiter";
+	public static final String KEY_FORMAT = "format";
+	public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
 
-    private transient AlgebricksPartitionConstraint clusterLocations;
-    private String[] readSchedule;
-    private boolean executed[];
-    private InputSplitsFactory inputSplitsFactory;
-    private ConfFactory confFactory;
-    private boolean setup = false;
+	private transient AlgebricksPartitionConstraint clusterLocations;
+	private String[] readSchedule;
+	private boolean executed[];
+	private InputSplitsFactory inputSplitsFactory;
+	private ConfFactory confFactory;
+	private boolean setup = false;
 
-    private static final Map<String, String> formatClassNames = initInputFormatMap();
+	private static final Map<String, String> formatClassNames = initInputFormatMap();
 
-    private static Map<String, String> initInputFormatMap() {
-        Map<String, String> formatClassNames = new HashMap<String, String>();
-        formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
-        formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
-        return formatClassNames;
-    }
+	private static Map<String, String> initInputFormatMap() {
+		Map<String, String> formatClassNames = new HashMap<String, String>();
+		formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
+		formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
+		formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
+		return formatClassNames;
+	}
 
-    @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-        if (!setup) {
-            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-            configureJobConf(configuration);
-            JobConf conf = configureJobConf(configuration);
-            confFactory = new ConfFactory(conf);
+	@Override
+	public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+		if (!setup) {
+			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+			configureJobConf(configuration);
+			JobConf conf = configureJobConf(configuration);
+			confFactory = new ConfFactory(conf);
 
-            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
 
-            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-            inputSplitsFactory = new InputSplitsFactory(inputSplits);
+			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+			inputSplitsFactory = new InputSplitsFactory(inputSplits);
 
-            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-            readSchedule = scheduler.getLocationConstraints(inputSplits);
-            executed = new boolean[readSchedule.length];
-            Arrays.fill(executed, false);
+			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+			readSchedule = scheduler.getLocationConstraints(inputSplits);
+			executed = new boolean[readSchedule.length];
+			Arrays.fill(executed, false);
 
-            setup = true;
-        }
-        JobConf conf = confFactory.getConf();
-        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-        HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
-        hdfsAdapter.configure(configuration);
-        return hdfsAdapter;
-    }
+			setup = true;
+		}
+		JobConf conf = confFactory.getConf();
+		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+		HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
 
-    @Override
-    public String getName() {
-        return HDFS_ADAPTER_NAME;
-    }
+		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
+		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
+		{
+			char delimeter = 0x01;
+			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
+			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
+		}
 
-    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
-        JobConf conf = new JobConf();
-        conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
-        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
-        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
-        conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
-        conf.set("mapred.input.format.class",
-                (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
-        return conf;
-    }
+		hdfsAdapter.configure(configuration);
+		return hdfsAdapter;
+	}
+
+	@Override
+	public IControlledAdapter createAccessByRIDAdapter(
+			Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
+		Configuration conf = configureHadoopConnection(configuration);
+		clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+		
+		//Create RID record desc
+		RecordDescriptor ridRecordDesc = null;
+
+		//If input format is rcfile, configure parser expected format to delimeted text with control char 0x01 as delimiter
+		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
+		{
+			char delimeter = 0x01;
+			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
+			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
+			ridRecordDesc = getRIDRecDesc(true, files != null);
+		}
+		else
+		{
+			ridRecordDesc = getRIDRecDesc(false, files != null);
+		}
+		HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf, files);
+		adapter.configure(configuration);
+		return adapter;
+	}
+
+	@Override
+	public IDatasourceAdapter createIndexingAdapter(Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
+		if (!setup) {
+			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+			configureJobConf(configuration);
+			JobConf conf = configureJobConf(configuration);
+			confFactory = new ConfFactory(conf);
+
+			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+			inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+			readSchedule = scheduler.getLocationConstraints(inputSplits);
+			executed = new boolean[readSchedule.length];
+			Arrays.fill(executed, false);
+
+			setup = true;
+		}
+		JobConf conf = confFactory.getConf();
+		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default) as delimiter
+		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
+		{
+			char delimeter = 0x01;
+			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
+			configuration.put(KEY_DELIMITER, Character.toString(delimeter));	
+		}
+		HDFSIndexingAdapter hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
+		hdfsIndexingAdapter.configure(configuration);
+		return hdfsIndexingAdapter;
+	}
+
+	@Override
+	public String getName() {
+		return HDFS_ADAPTER_NAME;
+	}
+
+	private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+		JobConf conf = new JobConf();
+		conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
+		conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+		conf.setClassLoader(HDFSAdapter.class.getClassLoader());
+		conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
+		conf.set("mapred.input.format.class",
+				(String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
+		return conf;
+	}
+
+	public static Configuration configureHadoopConnection(Map<String, Object> configuration)
+	{
+		Configuration conf = new Configuration();
+		conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
+		conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+		return conf;
+	}
+	
+	public static RecordDescriptor getRIDRecDesc(boolean isRCFile, boolean optimize){
+		int numOfPrimaryKeys = 2;
+		if(isRCFile)
+		{
+			numOfPrimaryKeys++;
+		}
+		@SuppressWarnings("rawtypes")
+		ISerializerDeserializer[] serde = new ISerializerDeserializer[numOfPrimaryKeys];
+		ITypeTraits[] tt = new ITypeTraits[numOfPrimaryKeys];
+		if(optimize)
+		{
+			serde[0] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+			tt[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT32);
+		}
+		else
+		{
+			serde[0] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+			tt[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ASTRING);
+		}
+		serde[1] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+		tt[1] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT64);
+		if(isRCFile)
+		{
+			//we add the row number for rc-files
+			serde[2] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+			tt[2] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT32);
+		}
+		return new RecordDescriptor(serde, tt);
+	}
+
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
index 409eb7a..64c8153 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
@@ -18,15 +18,20 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 
+import edu.uci.ics.asterix.external.dataset.adapter.HDFSAccessByRIDAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.HiveIndexingAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 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.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
 import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -36,109 +41,185 @@
  */
 @SuppressWarnings("deprecation")
 public class HiveAdapterFactory implements IGenericDatasetAdapterFactory {
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    public static final String HDFS_ADAPTER_NAME = "hdfs";
-    public static final String CLUSTER_LOCATIONS = "cluster-locations";
-    public static transient String SCHEDULER = "hdfs-scheduler";
+	public static final String HDFS_ADAPTER_NAME = "hdfs";
+	public static final String CLUSTER_LOCATIONS = "cluster-locations";
+	public static transient String SCHEDULER = "hdfs-scheduler";
 
-    public static final String KEY_HDFS_URL = "hdfs";
-    public static final String KEY_PATH = "path";
-    public static final String KEY_INPUT_FORMAT = "input-format";
-    public static final String INPUT_FORMAT_TEXT = "text-input-format";
-    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+	public static final String KEY_HDFS_URL = "hdfs";
+	public static final String KEY_PATH = "path";
+	public static final String KEY_INPUT_FORMAT = "input-format";
+	public static final String INPUT_FORMAT_TEXT = "text-input-format";
+	public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
+	public static final String INPUT_FORMAT_RC = "rc-input-format";
 
-    public static final String KEY_FORMAT = "format";
-    public static final String KEY_PARSER_FACTORY = "parser";
-    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
-    public static final String FORMAT_ADM = "adm";
+	public static final String KEY_FORMAT = "format";
+	public static final String KEY_PARSER_FACTORY = "parser";
+	public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+	public static final String FORMAT_ADM = "adm";
+	public static final String KEY_DELIMITER = "delimiter";
 
-    public static final String HIVE_DATABASE = "database";
-    public static final String HIVE_TABLE = "table";
-    public static final String HIVE_HOME = "hive-home";
-    public static final String HIVE_METASTORE_URI = "metastore-uri";
-    public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
-    public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
+	public static final String HIVE_DATABASE = "database";
+	public static final String HIVE_TABLE = "table";
+	public static final String HIVE_HOME = "hive-home";
+	public static final String HIVE_METASTORE_URI = "metastore-uri";
+	public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
+	public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
 
-    private String[] readSchedule;
-    private boolean executed[];
-    private InputSplitsFactory inputSplitsFactory;
-    private ConfFactory confFactory;
-    private transient AlgebricksPartitionConstraint clusterLocations;
-    private boolean setup = false;
+	private String[] readSchedule;
+	private boolean executed[];
+	private InputSplitsFactory inputSplitsFactory;
+	private ConfFactory confFactory;
+	private transient AlgebricksPartitionConstraint clusterLocations;
+	private boolean setup = false;
 
-    private static final Map<String, String> formatClassNames = initInputFormatMap();
+	private static final Map<String, String> formatClassNames = initInputFormatMap();
 
-    private static Map<String, String> initInputFormatMap() {
-        Map<String, String> formatClassNames = new HashMap<String, String>();
-        formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
-        formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
-        return formatClassNames;
-    }
+	private static Map<String, String> initInputFormatMap() {
+		Map<String, String> formatClassNames = new HashMap<String, String>();
+		formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
+		formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
+		formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
+		return formatClassNames;
+	}
 
-    @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-        if (!setup) {
-            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-            configureJobConf(configuration);
-            JobConf conf = configureJobConf(configuration);
-            confFactory = new ConfFactory(conf);
 
-            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+	@Override
+	public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
+		Configuration conf = HDFSAdapterFactory.configureHadoopConnection(configuration);
+		clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+		//Create RID record desc
+		RecordDescriptor ridRecordDesc = null;
 
-            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-            inputSplitsFactory = new InputSplitsFactory(inputSplits);
+		//If input format is rcfile, configure parser expected format to delimeted text with control char 0x01 as delimiter
+		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
+		{
+			char delimeter = 0x01;
+			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
+			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
+			ridRecordDesc = HDFSAdapterFactory.getRIDRecDesc(true, files != null);
+		}
+		else
+		{
+			ridRecordDesc = HDFSAdapterFactory.getRIDRecDesc(false, files != null);
+		}
+		HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf, files);
+		adapter.configure(configuration);
+		return adapter;
+	}
 
-            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-            readSchedule = scheduler.getLocationConstraints(inputSplits);
-            executed = new boolean[readSchedule.length];
-            Arrays.fill(executed, false);
+	@Override
+	public IDatasourceAdapter createIndexingAdapter(
+			Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
+		if (!setup) {
+			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+			configureJobConf(configuration);
+			JobConf conf = configureJobConf(configuration);
+			confFactory = new ConfFactory(conf);
 
-            setup = true;
-        }
-        JobConf conf = confFactory.getConf();
-        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-        HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
-        hiveAdapter.configure(configuration);
-        return hiveAdapter;
-    }
+			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
 
-    @Override
-    public String getName() {
-        return "hive";
-    }
+			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+			inputSplitsFactory = new InputSplitsFactory(inputSplits);
 
-    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
-        JobConf conf = new JobConf();
+			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+			readSchedule = scheduler.getLocationConstraints(inputSplits);
+			executed = new boolean[readSchedule.length];
+			Arrays.fill(executed, false);
 
-        /** configure hive */
-        String database = (String) configuration.get(HIVE_DATABASE);
-        String tablePath = null;
-        if (database == null) {
-            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
-        } else {
-            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
-                    + configuration.get(HIVE_TABLE);
-        }
-        configuration.put(HDFSAdapter.KEY_PATH, tablePath);
-        if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
-            throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
-        }
+			setup = true;
+		}
+		JobConf conf = confFactory.getConf();
+		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+		HiveIndexingAdapter hiveIndexingAdapter = new HiveIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
 
-        if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
-                .get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))) {
-            throw new IllegalArgumentException("file input format"
-                    + configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
-        }
+		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
+		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
+		{
+			char delimeter = 0x01;
+			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
+			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
+		}
 
-        /** configure hdfs */
-        conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
-        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
-        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
-        conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
-        conf.set("mapred.input.format.class",
-                (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
-        return conf;
-    }
-}
+		hiveIndexingAdapter.configure(configuration);
+		return hiveIndexingAdapter;
+	}
+
+	@Override
+	public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+		if (!setup) {
+			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+			configureJobConf(configuration);
+			JobConf conf = configureJobConf(configuration);
+			confFactory = new ConfFactory(conf);
+
+			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+
+			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+			inputSplitsFactory = new InputSplitsFactory(inputSplits);
+
+			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+			readSchedule = scheduler.getLocationConstraints(inputSplits);
+			executed = new boolean[readSchedule.length];
+			Arrays.fill(executed, false);
+
+			setup = true;
+		}
+		JobConf conf = confFactory.getConf();
+		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+		HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+
+		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
+		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
+		{
+			char delimeter = 0x01;
+			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
+			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
+		}
+
+		hiveAdapter.configure(configuration);
+		return hiveAdapter;
+	}
+
+	@Override
+	public String getName() {
+		return "hive";
+	}
+
+	private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+		JobConf conf = new JobConf();
+
+		/** configure hive */
+		String database = (String) configuration.get(HIVE_DATABASE);
+		String tablePath = null;
+		if (database == null) {
+			tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
+		} else {
+			tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
+					+ configuration.get(HIVE_TABLE);
+		}
+		configuration.put(HDFSAdapter.KEY_PATH, tablePath);
+		if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
+			throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
+		}
+
+		if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
+				.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE) || configuration
+				.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_RC))) {
+			throw new IllegalArgumentException("file input format"
+					+ configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
+		}
+
+		/** configure hdfs */
+		conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
+		conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+		conf.setClassLoader(HDFSAdapter.class.getClassLoader());
+		conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
+		conf.set("mapred.input.format.class",
+				(String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
+		return conf;
+	}
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
index 0a178a7..f046f88 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
@@ -14,8 +14,10 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.util.HashMap;
 import java.util.Map;
 
+import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
 
@@ -40,4 +42,29 @@
      */
     public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception;
 
+    /**
+     * Creates an instance of IDatasourceAdapter that is used to read records and their RIDs.
+     * 
+     * @param configuration
+     *            The configuration parameters for the adapter that is instantiated.
+     *            The passed-in configuration is used to configure the created instance of the adapter.
+     * @param atype
+     *            The type for the ADM records that are returned by the adapter (contains both original fields and RID fields).
+     * @return An instance of IDatasourceAdapter.
+     * @throws Exception
+     */
+    public IDatasourceAdapter createIndexingAdapter(Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception;
+
+    /**
+     * Creates an instance of IDatasourceAdapter that is used to read records using their RIDs.
+     * 
+     * @param configuration
+     *            The configuration parameters for the adapter that is instantiated.
+     *            The passed-in configuration is used to configure the created instance of the adapter.
+     * @param atype
+     *            The type for the ADM records that are returned by the adapter.
+     * @return An instance of IControlledAdapter.
+     * @throws Exception
+     */
+    public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception;
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
index e680232..4fae7e7 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
@@ -14,11 +14,14 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
+import java.util.HashMap;
 import java.util.Map;
 
+import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 
 /**
  * Factory class for creating an instance of NCFileSystemAdapter. An
@@ -26,18 +29,31 @@
  * an NC.
  */
 public class NCFileSystemAdapterFactory implements IGenericDatasetAdapterFactory {
-    private static final long serialVersionUID = 1L;
-    public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
+	private static final long serialVersionUID = 1L;
+	public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
 
-    @Override
-    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-        NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
-        fsAdapter.configure(configuration);
-        return fsAdapter;
-    }
+	@Override
+	public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+		NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
+		fsAdapter.configure(configuration);
+		return fsAdapter;
+	}
 
-    @Override
-    public String getName() {
-        return NC_FILE_SYSTEM_ADAPTER_NAME;
-    }
+	@Override
+	public String getName() {
+		return NC_FILE_SYSTEM_ADAPTER_NAME;
+	}
+
+	@Override
+	public IDatasourceAdapter createIndexingAdapter(
+			Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
+		throw new NotImplementedException("Indexing Adapter is not implemented for NC FileSystem Data");
+	}
+
+
+
+	@Override
+	public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
+		throw new NotImplementedException("Access by RID Adapter is not implemented for NC FileSystem Data");
+	}
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
new file mode 100644
index 0000000..aa91a56
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
@@ -0,0 +1,78 @@
+package edu.uci.ics.asterix.external.data.operator;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
+import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+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.api.job.IOperatorDescriptorRegistry;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class ExternalDataAccessByRIDOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
+
+	/**
+	 * This operator is used to access external data residing in hdfs using record ids pushed in frame buffers
+	 */
+	private static final long serialVersionUID = 1L;
+	private final Map<String, Object> adapterConfiguration;
+	private final IAType atype;
+	private IGenericDatasetAdapterFactory datasourceAdapterFactory;
+	private IControlledAdapter adapter;
+	private final HashMap<Integer, String> files;
+	
+	public ExternalDataAccessByRIDOperatorDescriptor(
+			IOperatorDescriptorRegistry spec, Map<String, Object> arguments, IAType atype,
+			RecordDescriptor outRecDesc,IGenericDatasetAdapterFactory dataSourceAdapterFactory, HashMap<Integer, String> files) {
+		super(spec, 1, 1);
+		this.atype = atype;
+		this.adapterConfiguration = arguments;
+		this.datasourceAdapterFactory = dataSourceAdapterFactory;
+		this.recordDescriptors[0] = outRecDesc;
+		this.files = files;
+	}
+
+	@Override
+	public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+			IRecordDescriptorProvider recordDescProvider, int partition,
+			int nPartitions) throws HyracksDataException {
+		return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
+			@Override
+			public void open() throws HyracksDataException {
+				//create the access by index adapter
+				try {
+					adapter = datasourceAdapterFactory.createAccessByRIDAdapter(adapterConfiguration, atype, files);
+					adapter.initialize(ctx);
+				} catch (Exception e) {
+					throw new HyracksDataException("error during creation of external read by RID adapter", e);
+				}
+				writer.open();
+			}
+
+			@Override
+			public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+				adapter.processNextFrame(buffer, writer);
+			}
+
+			@Override
+			public void close() throws HyracksDataException {
+				//close adapter and flush remaining frame if needed
+				adapter.close(writer);
+				//close writer
+				writer.close();
+			}
+
+			@Override
+			public void fail() throws HyracksDataException {
+				writer.fail();
+			}
+		};	
+	}
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
new file mode 100644
index 0000000..9ff1f06
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
@@ -0,0 +1,64 @@
+package edu.uci.ics.asterix.external.data.operator;
+
+import java.util.Map;
+
+import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
+import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
+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.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
+
+/*
+ * A single activity operator that provides the functionality of scanning data along 
+ * with their RIDs using an instance of the configured adapter.
+ */
+
+public class ExternalDataIndexingOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor{
+
+	private static final long serialVersionUID = 1L;
+
+	private final Map<String, Object> adapterConfiguration;
+	private final Map<String,Integer> files;
+	private final IAType atype;
+	private IGenericDatasetAdapterFactory datasourceAdapterFactory;
+
+	public ExternalDataIndexingOperatorDescriptor(JobSpecification spec, Map<String, Object> arguments, IAType atype,
+			RecordDescriptor rDesc, IGenericDatasetAdapterFactory dataSourceAdapterFactory, Map<String,Integer> files) {
+		super(spec, 0, 1);
+		recordDescriptors[0] = rDesc;
+		this.adapterConfiguration = arguments;
+		this.atype = atype;
+		this.datasourceAdapterFactory = dataSourceAdapterFactory;
+		this.files = files;
+	}
+
+	@Override
+	public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+			IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+					throws HyracksDataException {
+
+		return new AbstractUnaryOutputSourceOperatorNodePushable() {
+			@Override
+			public void initialize() throws HyracksDataException {
+				writer.open();
+				IDatasourceAdapter adapter = null;
+				try {
+					adapter = ((IGenericDatasetAdapterFactory) datasourceAdapterFactory).createIndexingAdapter(
+							adapterConfiguration, atype, files);
+					adapter.initialize(ctx);
+					adapter.start(partition, writer);
+				} catch (Exception e) {
+					throw new HyracksDataException("exception during reading from external data source", e);
+				} finally {
+					writer.close();
+				}
+			}
+		};
+	}
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
new file mode 100644
index 0000000..86a060c
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
@@ -0,0 +1,1170 @@
+/*
+ * 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.external.dataset.adapter;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.RCFile;
+import org.apache.hadoop.hive.ql.io.RCFile.Reader;
+import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.AString;
+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.runtime.operators.file.ControlledADMTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.ControlledDelimitedDataTupleParserFactory;
+import edu.uci.ics.asterix.runtime.operators.file.ControlledTupleParser;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+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.comm.util.ByteBufferInputStream;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+/**
+ * Provides functionality for fetching specific external data records stored in an HDFS instance
+ * using their RID.
+ */
+@SuppressWarnings({ "deprecation" })
+public class HDFSAccessByRIDAdapter extends FileSystemBasedAdapter implements IControlledAdapter{
+
+	private static final long serialVersionUID = 1L;
+	private boolean newFrame;
+	private transient ByteBuffer frameBuffer;
+	private String inputFormat;
+	private Configuration conf;
+	private transient FileSystem fs;
+	private RecordDescriptor inRecDesc;
+	private final HashMap<Integer, String> files;
+
+	public HDFSAccessByRIDAdapter(IAType atype, String inputFormat, AlgebricksPartitionConstraint clusterLocations, RecordDescriptor inRecDesc, Configuration conf, HashMap<Integer,String> files) {
+		super(atype);
+		this.inputFormat = inputFormat;
+		this.conf = conf;
+		this.inRecDesc = inRecDesc;
+		this.files = files;
+	}
+
+	@Override
+	public void configure(Map<String, Object> arguments) throws Exception {
+		this.configuration = arguments;
+		fs = FileSystem.get(conf);
+		String specifiedFormat = (String) configuration.get(KEY_FORMAT);
+		if (specifiedFormat == null) {
+			throw new IllegalArgumentException(" Unspecified data format");
+		} else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
+			parserFactory = getDelimitedDataTupleParserFactory((ARecordType) atype);
+		} else if (FORMAT_ADM.equalsIgnoreCase((String)configuration.get(KEY_FORMAT))) {
+			parserFactory = new ControlledADMTupleParserFactory((ARecordType) atype);
+		} else {
+			throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
+		}
+	}
+
+	@Override
+	protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType) throws AsterixException {
+		int n = recordType.getFieldTypes().length;
+		IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
+		for (int i = 0; i < n; i++) {
+			ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
+			IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
+			if (vpf == null) {
+				throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
+			}
+			fieldParserFactories[i] = vpf;
+		}
+		String delimiterValue = (String) configuration.get(KEY_DELIMITER);
+		if (delimiterValue != null && delimiterValue.length() > 1) {
+			throw new AsterixException("improper delimiter");
+		}
+
+		Character delimiter = delimiterValue.charAt(0);
+		return new ControlledDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
+	}
+
+	@Override
+	public void start(int partition, IFrameWriter writer) throws Exception {
+		throw new NotImplementedException("Access by RID adapter doesn't support start function");
+	}
+
+	public void processNextFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException
+	{
+		frameBuffer = buffer;
+		newFrame = true;
+		((ControlledTupleParser)parser).parseNext(writer);
+	}
+
+	public void close(IFrameWriter writer) throws HyracksDataException
+	{
+		((ControlledTupleParser)parser).close(writer);
+	}
+
+	public AdapterType getAdapterType() {
+		return AdapterType.READ;
+	}
+
+	@Override
+	public void initialize(IHyracksTaskContext ctx) throws Exception {
+		this.ctx = ctx;
+		//create parser and initialize it with an instance of the inputStream
+		parser = parserFactory.createTupleParser(ctx);
+		((ControlledTupleParser)parser).initialize(getInputStream(0));
+	}
+
+	@Override
+	public InputStream getInputStream(int partition) throws IOException {
+
+		//if files map is not null, then it is optimized and we should return optimized inputStream, else return regular
+		if(files == null)
+		{	
+
+			//different input stream implementation based on the input format
+			if(inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
+			{
+				return new InputStream() {
+					private RCFile.Reader reader;
+					private int rowDifference;
+					private String lastFileName = "";
+					private String newFileName;
+					private long lastByteLocation = 0;
+					private long newByteLocation = 0;
+					private int lastRowNumber = 0;
+					private int newRowNumber = 0;
+					private LongWritable key;
+					private BytesRefArrayWritable value;
+					private int EOL = "\n".getBytes()[0];
+					private byte delimiter = 0x01;
+					private boolean pendingValue = false;
+					private int currentTupleIdx;
+					private int numberOfTuplesInCurrentFrame;
+					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
+					private ByteBufferInputStream bbis = new ByteBufferInputStream();
+					private DataInputStream dis = new DataInputStream(bbis);
+
+					@Override
+					public void close()
+					{
+						if (reader != null)
+						{
+							reader.close();
+						}
+						try {
+							super.close();
+						} catch (IOException e) {
+							e.printStackTrace();
+						}
+					}
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+						if(newFrame)
+						{
+							//first time called with this frame
+							//reset frame buffer
+							tupleAccessor.reset(frameBuffer);
+							//get number of tuples in frame
+							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
+							//set tuple index to first tuple
+							currentTupleIdx = 0;
+							//set new frame to false
+							newFrame = false;
+							pendingValue = false;
+						}
+
+						//check and see if there is a pending value
+						//Double check this
+						int numBytes = 0;
+						if (pendingValue) {
+							//last value didn't fit into buffer
+							int sizeOfNextTuple = getTupleSize(value) + 1;
+							if(sizeOfNextTuple > len)
+							{
+								return 0;
+							}
+							copyCurrentTuple(buffer, offset + numBytes);
+							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = false;
+							//move to next tuple
+							currentTupleIdx++;
+						}
+
+						//No pending value or done with pending value
+						//check if there are more tuples in the frame
+						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
+						{
+							//get 3 things from the current tuple in the frame(File name, byte location and row number)
+							//get the fileName
+							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
+							newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
+							//check if it is a new file
+							if(!lastFileName.equals(newFileName))//stringBuilder.toString()))
+							{
+								//new file
+								lastFileName = newFileName;
+								//close old file
+								if(reader != null)
+								{
+									reader.close();
+								}
+								//open new file
+								reader = new Reader(fs, new Path(lastFileName), conf);
+								//read and save byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								lastByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(lastByteLocation);
+								//read and save rowNumber
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
+								lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
+								//loop until row
+								for(int i=0; i < lastRowNumber; i++)
+								{
+									//this loop perform a single I/O and move to the next record in the block which is already in memory
+									//if no more records in the current block, it perform another I/O and get the next block
+									//<this should never happen here>
+									reader.next(key);
+								}
+								//read record
+								reader.getCurrentRow(value);
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = getTupleSize(value) + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = true;
+									return numBytes;
+								}
+								copyCurrentTuple(buffer, offset + numBytes);
+								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							else
+							{
+								//same file
+								//get the byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								newByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+
+								//check if same block
+								if(lastByteLocation != newByteLocation)
+								{
+									//new block
+									lastByteLocation = newByteLocation;
+									//seek
+									reader.seek(lastByteLocation);
+									//read and save rowNumber
+									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
+									lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
+									//loop until row
+									for(int i=0; i < lastRowNumber; i++)
+									{
+										reader.next(key);
+									}
+									//read record
+									reader.getCurrentRow(value);
+									//copy it to the buffer if there is enough space
+									int sizeOfNextTuple = getTupleSize(value) + 1;
+									if(sizeOfNextTuple + numBytes > len)
+									{
+										//mark waiting value
+										pendingValue = true;
+										return numBytes;
+									}
+									copyCurrentTuple(buffer, offset + numBytes);
+									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+									numBytes += sizeOfNextTuple;
+								}
+								else
+								{
+									//same block
+									//get the row number
+									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
+									newRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
+
+									//calculate row difference
+									rowDifference = newRowNumber - lastRowNumber;
+
+									//update last row number
+									lastRowNumber = newRowNumber;
+
+									//move to the new row
+									for(int i=0; i < rowDifference; i++)
+									{
+										reader.next(key);
+									}
+									//read record
+									reader.getCurrentRow(value);
+
+									//copy it to the buffer if there is enough space
+									int sizeOfNextTuple = getTupleSize(value) + 1;
+									if(sizeOfNextTuple + numBytes > len)
+									{
+										//mark waiting value
+										pendingValue = true;
+										return numBytes;
+									}
+									copyCurrentTuple(buffer, offset + numBytes);
+									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+									numBytes += sizeOfNextTuple;
+								}
+							}
+							//move to next tuple
+							currentTupleIdx++;
+						}	
+						//no more tuples in frame
+						return (numBytes == 0) ? -1 : numBytes;
+					}
+
+					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
+						int rcOffset = 0;
+						for(int i=0; i< value.size(); i++)
+						{
+							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
+							rcOffset += value.get(i).getLength() + 1;
+							buffer[rcOffset - 1] = delimiter;
+						}
+					}
+
+					private int getTupleSize(BytesRefArrayWritable value2) {
+						int size=0;
+						//loop over rc column and add lengths
+						for(int i=0; i< value.size(); i++)
+						{
+							size += value.get(i).getLength();
+						}
+						//add delimeters bytes sizes
+						size += value.size() -1;
+						return size;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+				};
+			}
+			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT))
+			{
+				return new InputStream() {
+					private FSDataInputStream reader;
+					private String lastFileName = "";
+					private String newFileName;
+					private int EOL = "\n".getBytes()[0];
+					private int currentTupleIdx;
+					private int numberOfTuplesInCurrentFrame;
+					private long byteLocation;
+					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
+					private String value;
+					private String pendingValue = null;
+					private ByteBufferInputStream bbis = new ByteBufferInputStream();
+					private DataInputStream dis = new DataInputStream(bbis);
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+						if(newFrame)
+						{
+							//first time called with this frame
+							//reset frame buffer
+							tupleAccessor.reset(frameBuffer);
+							//get number of tuples in frame
+							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
+							//set tuple index to first tuple
+							currentTupleIdx = 0;
+							//set new frame to false
+							newFrame = false;
+						}
+
+						//check and see if there is a pending value
+						int numBytes = 0;
+						if (pendingValue != null) {
+							//last value didn't fit into buffer
+							int sizeOfNextTuple = pendingValue.length() + 1;
+							if(sizeOfNextTuple > len)
+							{
+								return 0;
+							}
+							//there is enough space
+							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.length());
+							buffer[offset + numBytes + pendingValue.length()] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = null;
+							//move to next tuple
+							currentTupleIdx++;
+						}
+
+						//No pending value or done with pending value
+						//check if there are more tuples in the frame
+						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
+						{
+							//get the fileName
+							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
+							newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
+							//check if it is a new file
+							if(!lastFileName.equals(newFileName))
+							{
+								//new file
+								lastFileName = newFileName;
+								//close old file
+								if(reader != null)
+								{
+									reader.close();
+								}
+								//open new file
+								reader = fs.open(new Path(lastFileName));
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								value = reader.readLine();
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.length() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
+								buffer[offset + numBytes + value.length()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							else
+							{
+								//same file, just seek and read
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								value = reader.readLine();
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.length() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
+								buffer[offset + numBytes + value.length()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							currentTupleIdx++;
+						}
+						return (numBytes == 0) ? -1 : numBytes;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+
+					@Override
+					public void close(){
+						try {
+							if (reader != null)
+							{
+								reader.close();
+							}
+							super.close();
+						} catch (IOException e) {
+							e.printStackTrace();
+						}
+					}
+
+				};
+			}
+			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))
+			{
+				return new InputStream() {
+					private SequenceFile.Reader reader;
+					private Writable key;
+					private Text value;
+					private String lastFileName = "";
+					private String newFileName;
+					private long byteLocation;
+					private int EOL = "\n".getBytes()[0];
+					private int currentTupleIdx;
+					private int numberOfTuplesInCurrentFrame;
+					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
+					private Text pendingValue = null;
+					private ByteBufferInputStream bbis = new ByteBufferInputStream();
+					private DataInputStream dis = new DataInputStream(bbis);
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+
+						if(newFrame)
+						{
+							//first time called with this frame
+							//reset frame buffer
+							tupleAccessor.reset(frameBuffer);
+							//get number of tuples in frame
+							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
+							//set tuple index to first tuple
+							currentTupleIdx = 0;
+							//set new frame to false
+							newFrame = false;
+						}
+
+						//check and see if there is a pending value
+						//Double check this
+						int numBytes = 0;
+						if (pendingValue != null) {
+							//last value didn't fit into buffer
+							int sizeOfNextTuple = pendingValue.getLength() + 1;
+							if(sizeOfNextTuple > len)
+							{
+								return 0;
+							}
+							//there is enough space
+							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+							buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = null;
+							//move to next tuple
+							currentTupleIdx++;
+						}
+
+						//No pending value or done with pending value
+						//check if there are more tuples in the frame
+						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
+						{
+							//get the fileName]
+							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
+							newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
+							//check if it is a new file
+							if(!lastFileName.equals(newFileName))
+							{
+								//new file
+								lastFileName = newFileName;
+								//close old file
+								if(reader != null)
+								{
+									reader.close();
+								}
+								//open new file
+								reader = new SequenceFile.Reader(fs,new Path(lastFileName),conf);
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								reader.next(key, value);
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.getLength() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							else
+							{
+								//same file, just seek and read
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								reader.next(key, value);
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.getLength() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							currentTupleIdx++;
+						}
+						return (numBytes == 0) ? -1 : numBytes;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+
+					@Override
+					public void close(){
+						try {
+							if (reader != null)
+							{
+								reader.close();
+							}
+							super.close();
+						} catch (IOException e) {
+							e.printStackTrace();
+						}
+					}
+				};
+			}
+			//unknow format
+			throw new IOException("Unknown input format");
+		}
+		else
+		{
+			//optimized
+			//different input stream implementation based on the input format
+			if(inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
+			{
+				return new InputStream() {
+					private RCFile.Reader reader;
+					private int rowDifference;
+					private int lastFileNumber = -1;
+					private int newFileNumber = 0;
+					private long lastByteLocation = 0;
+					private long newByteLocation = 0;
+					private int lastRowNumber = 0;
+					private int newRowNumber = 0;
+					private LongWritable key;
+					private BytesRefArrayWritable value;
+					private int EOL = "\n".getBytes()[0];
+					private byte delimiter = 0x01;
+					private boolean pendingValue = false;
+					private int currentTupleIdx;
+					private int numberOfTuplesInCurrentFrame;
+					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
+					private ByteBufferInputStream bbis = new ByteBufferInputStream();
+					private DataInputStream dis = new DataInputStream(bbis);
+
+					@Override
+					public void close()
+					{
+						if (reader != null)
+						{
+							reader.close();
+						}
+						try {
+							super.close();
+						} catch (IOException e) {
+							e.printStackTrace();
+						}
+					}
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+						if(newFrame)
+						{
+							//first time called with this frame
+							//reset frame buffer
+							tupleAccessor.reset(frameBuffer);
+							//get number of tuples in frame
+							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
+							//set tuple index to first tuple
+							currentTupleIdx = 0;
+							//set new frame to false
+							newFrame = false;
+							pendingValue = false;
+						}
+
+						//check and see if there is a pending value
+						//Double check this
+						int numBytes = 0;
+						if (pendingValue) {
+							//last value didn't fit into buffer
+							int sizeOfNextTuple = getTupleSize(value) + 1;
+							if(sizeOfNextTuple > len)
+							{
+								return 0;
+							}
+							copyCurrentTuple(buffer, offset + numBytes);
+							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = false;
+							//move to next tuple
+							currentTupleIdx++;
+						}
+
+						//No pending value or done with pending value
+						//check if there are more tuples in the frame
+						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
+						{
+							//get 3 things from the current tuple in the frame(File name, byte location and row number)
+							//get the fileName
+							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
+							newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
+							//check if it is a new file
+							if(lastFileNumber != newFileNumber)
+							{
+								//new file
+								lastFileNumber = newFileNumber;
+								//close old file
+								if(reader != null)
+								{
+									reader.close();
+								}
+								//open new file
+								reader = new Reader(fs, new Path(files.get(newFileNumber)), conf);
+								//read and save byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								lastByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(lastByteLocation);
+								//read and save rowNumber
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
+								lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
+								//loop until row
+								for(int i=0; i < lastRowNumber; i++)
+								{
+									//this loop perform a single I/O and move to the next record in the block which is already in memory
+									//if no more records in the current block, it perform another I/O and get the next block
+									//<this should never happen here>
+									reader.next(key);
+								}
+								//read record
+								reader.getCurrentRow(value);
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = getTupleSize(value) + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = true;
+									return numBytes;
+								}
+								copyCurrentTuple(buffer, offset + numBytes);
+								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							else
+							{
+								//same file
+								//get the byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								newByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+
+								//check if same block
+								if(lastByteLocation != newByteLocation)
+								{
+									//new block
+									lastByteLocation = newByteLocation;
+									//seek
+									reader.seek(lastByteLocation);
+									//read and save rowNumber
+									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
+									lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
+									//loop until row
+									for(int i=0; i < lastRowNumber; i++)
+									{
+										reader.next(key);
+									}
+									//read record
+									reader.getCurrentRow(value);
+									//copy it to the buffer if there is enough space
+									int sizeOfNextTuple = getTupleSize(value) + 1;
+									if(sizeOfNextTuple + numBytes > len)
+									{
+										//mark waiting value
+										pendingValue = true;
+										return numBytes;
+									}
+									copyCurrentTuple(buffer, offset + numBytes);
+									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+									numBytes += sizeOfNextTuple;
+								}
+								else
+								{
+									//same block
+									//get the row number
+									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
+									newRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
+
+									//calculate row difference
+									rowDifference = newRowNumber - lastRowNumber;
+
+									//update last row number
+									lastRowNumber = newRowNumber;
+
+									//move to the new row
+									for(int i=0; i < rowDifference; i++)
+									{
+										reader.next(key);
+									}
+									//read record
+									reader.getCurrentRow(value);
+
+									//copy it to the buffer if there is enough space
+									int sizeOfNextTuple = getTupleSize(value) + 1;
+									if(sizeOfNextTuple + numBytes > len)
+									{
+										//mark waiting value
+										pendingValue = true;
+										return numBytes;
+									}
+									copyCurrentTuple(buffer, offset + numBytes);
+									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+									numBytes += sizeOfNextTuple;
+								}
+							}
+							//move to next tuple
+							currentTupleIdx++;
+						}	
+						//no more tuples in frame
+						return (numBytes == 0) ? -1 : numBytes;
+					}
+
+					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
+						int rcOffset = 0;
+						for(int i=0; i< value.size(); i++)
+						{
+							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
+							rcOffset += value.get(i).getLength() + 1;
+							buffer[rcOffset - 1] = delimiter;
+						}
+					}
+
+					private int getTupleSize(BytesRefArrayWritable value2) {
+						int size=0;
+						//loop over rc column and add lengths
+						for(int i=0; i< value.size(); i++)
+						{
+							size += value.get(i).getLength();
+						}
+						//add delimeters bytes sizes
+						size += value.size() -1;
+						return size;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+				};
+			}
+			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT))
+			{
+				return new InputStream() {
+					private FSDataInputStream reader;
+					private int lastFileNumber = -1;
+					private int newFileNumber = 0;
+					private int EOL = "\n".getBytes()[0];
+					private int currentTupleIdx;
+					private int numberOfTuplesInCurrentFrame;
+					private long byteLocation;
+					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
+					private String value;
+					private String pendingValue = null;
+					private ByteBufferInputStream bbis = new ByteBufferInputStream();
+					private DataInputStream dis = new DataInputStream(bbis);
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+						if(newFrame)
+						{
+							//first time called with this frame
+							//reset frame buffer
+							tupleAccessor.reset(frameBuffer);
+							//get number of tuples in frame
+							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
+							//set tuple index to first tuple
+							currentTupleIdx = 0;
+							//set new frame to false
+							newFrame = false;
+						}
+
+						//check and see if there is a pending value
+						int numBytes = 0;
+						if (pendingValue != null) {
+							//last value didn't fit into buffer
+							int sizeOfNextTuple = pendingValue.length() + 1;
+							if(sizeOfNextTuple > len)
+							{
+								return 0;
+							}
+							//there is enough space
+							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.length());
+							buffer[offset + numBytes + pendingValue.length()] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = null;
+							//move to next tuple
+							currentTupleIdx++;
+						}
+
+						//No pending value or done with pending value
+						//check if there are more tuples in the frame
+						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
+						{
+							//get the file number
+							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
+							newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
+							//check if it is a new file
+							if(lastFileNumber != newFileNumber)
+							{
+								//new file
+								lastFileNumber = newFileNumber;
+								//close old file
+								if(reader != null)
+								{
+									reader.close();
+								}
+								
+								//open new file
+								reader = fs.open(new Path(files.get(newFileNumber)));
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								value = reader.readLine();
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.length() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
+								buffer[offset + numBytes + value.length()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							else
+							{
+								//same file, just seek and read
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								value = reader.readLine();
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.length() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
+								buffer[offset + numBytes + value.length()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							currentTupleIdx++;
+						}
+						return (numBytes == 0) ? -1 : numBytes;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+
+					@Override
+					public void close(){
+						try {
+							if (reader != null)
+							{
+								reader.close();
+							}
+							super.close();
+						} catch (IOException e) {
+							e.printStackTrace();
+						}
+					}
+
+				};
+			}
+			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))
+			{
+				return new InputStream() {
+					private SequenceFile.Reader reader;
+					private Writable key;
+					private Text value;
+					private int lastFileNumber = -1;
+					private int newFileNumber = 0;
+					private long byteLocation;
+					private int EOL = "\n".getBytes()[0];
+					private int currentTupleIdx;
+					private int numberOfTuplesInCurrentFrame;
+					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
+					private Text pendingValue = null;
+					private ByteBufferInputStream bbis = new ByteBufferInputStream();
+					private DataInputStream dis = new DataInputStream(bbis);
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+
+						if(newFrame)
+						{
+							//first time called with this frame
+							//reset frame buffer
+							tupleAccessor.reset(frameBuffer);
+							//get number of tuples in frame
+							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
+							//set tuple index to first tuple
+							currentTupleIdx = 0;
+							//set new frame to false
+							newFrame = false;
+						}
+
+						//check and see if there is a pending value
+						//Double check this
+						int numBytes = 0;
+						if (pendingValue != null) {
+							//last value didn't fit into buffer
+							int sizeOfNextTuple = pendingValue.getLength() + 1;
+							if(sizeOfNextTuple > len)
+							{
+								return 0;
+							}
+							//there is enough space
+							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+							buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = null;
+							//move to next tuple
+							currentTupleIdx++;
+						}
+
+						//No pending value or done with pending value
+						//check if there are more tuples in the frame
+						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
+						{
+							//get the fileName]
+							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
+							newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
+							//check if it is a new file
+							if(lastFileNumber != newFileNumber)
+							{
+								//new file
+								lastFileNumber = newFileNumber;
+								//close old file
+								if(reader != null)
+								{
+									reader.close();
+								}
+								//open new file
+								reader = new SequenceFile.Reader(fs,new Path(files.get(newFileNumber)),conf);
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								reader.next(key, value);
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.getLength() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							else
+							{
+								//same file, just seek and read
+								//read byte location
+								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
+								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
+								//seek
+								reader.seek(byteLocation);
+								//read record
+								reader.next(key, value);
+								//copy it to the buffer if there is enough space
+								int sizeOfNextTuple = value.getLength() + 1;
+								if(sizeOfNextTuple + numBytes > len)
+								{
+									//mark waiting value
+									pendingValue = value;
+									return numBytes;
+								}
+								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+							currentTupleIdx++;
+						}
+						return (numBytes == 0) ? -1 : numBytes;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+
+					@Override
+					public void close(){
+						try {
+							if (reader != null)
+							{
+								reader.close();
+							}
+							super.close();
+						} catch (IOException e) {
+							e.printStackTrace();
+						}
+					}
+				};
+			}
+			//unknow format
+			throw new IOException("Unknown input format");
+		}
+	}
+
+	@Override
+	public AlgebricksPartitionConstraint getPartitionConstraint()
+			throws Exception {
+		return partitionConstraint;
+	}
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
index f8b381b..2b355ff 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
@@ -18,6 +18,7 @@
 import java.io.InputStream;
 import java.util.Map;
 
+import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputSplit;
@@ -26,6 +27,8 @@
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
+import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
 
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -34,202 +37,348 @@
 
 /**
  * Provides functionality for fetching external data stored in an HDFS instance.
+ * Note: reader are never closed in adapters. should we make sure they are closed before returning or switching to a different reader?
  */
 @SuppressWarnings({ "deprecation", "rawtypes" })
 public class HDFSAdapter extends FileSystemBasedAdapter {
 
-    private static final long serialVersionUID = 1L;
+	private static final long serialVersionUID = 1L;
 
-    private transient String[] readSchedule;
-    private transient boolean executed[];
-    private transient InputSplit[] inputSplits;
-    private transient JobConf conf;
-    private transient AlgebricksPartitionConstraint clusterLocations;
+	private transient String[] readSchedule;
+	private transient boolean executed[];
+	private transient InputSplit[] inputSplits;
+	private transient JobConf conf;
+	private transient AlgebricksPartitionConstraint clusterLocations;
 
-    private transient String nodeName;
+	private transient String nodeName;
 
-    public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
-            AlgebricksPartitionConstraint clusterLocations) {
-        super(atype);
-        this.readSchedule = readSchedule;
-        this.executed = executed;
-        this.inputSplits = inputSplits;
-        this.conf = conf;
-        this.clusterLocations = clusterLocations;
-    }
+	public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
+			AlgebricksPartitionConstraint clusterLocations) {
+		super(atype);
+		this.readSchedule = readSchedule;
+		this.executed = executed;
+		this.inputSplits = inputSplits;
+		this.conf = conf;
+		this.clusterLocations = clusterLocations;
+	}
 
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        this.configuration = arguments;
-        configureFormat();
-    }
+	@Override
+	public void configure(Map<String, Object> arguments) throws Exception {
+		this.configuration = arguments;
+		configureFormat();
+	}
 
-    public AdapterType getAdapterType() {
-        return AdapterType.READ_WRITE;
-    }
+	public AdapterType getAdapterType() {
+		return AdapterType.READ_WRITE;
+	}
 
-    @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
-        this.ctx = ctx;
-        this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
-    }
+	@Override
+	public void initialize(IHyracksTaskContext ctx) throws Exception {
+		this.ctx = ctx;
+		this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+	}
 
-    private Reporter getReporter() {
-        Reporter reporter = new Reporter() {
+	private Reporter getReporter() {
+		Reporter reporter = new Reporter() {
 
-            @Override
-            public Counter getCounter(Enum<?> arg0) {
-                return null;
-            }
+			@Override
+			public Counter getCounter(Enum<?> arg0) {
+				return null;
+			}
 
-            @Override
-            public Counter getCounter(String arg0, String arg1) {
-                return null;
-            }
+			@Override
+			public Counter getCounter(String arg0, String arg1) {
+				return null;
+			}
 
-            @Override
-            public InputSplit getInputSplit() throws UnsupportedOperationException {
-                return null;
-            }
+			@Override
+			public InputSplit getInputSplit() throws UnsupportedOperationException {
+				return null;
+			}
 
-            @Override
-            public void incrCounter(Enum<?> arg0, long arg1) {
-            }
+			@Override
+			public void incrCounter(Enum<?> arg0, long arg1) {
+			}
 
-            @Override
-            public void incrCounter(String arg0, String arg1, long arg2) {
-            }
+			@Override
+			public void incrCounter(String arg0, String arg1, long arg2) {
+			}
 
-            @Override
-            public void setStatus(String arg0) {
-            }
+			@Override
+			public void setStatus(String arg0) {
+			}
 
-            @Override
-            public void progress() {
-            }
-        };
+			@Override
+			public void progress() {
+			}
+		};
 
-        return reporter;
-    }
+		return reporter;
+	}
 
-    @Override
-    public InputStream getInputStream(int partition) throws IOException {
+	@Override
+	public InputStream getInputStream(int partition) throws IOException {
 
-        return new InputStream() {
+		if(conf.getInputFormat() instanceof RCFileInputFormat)
+		{
+			//if hdfs input format is rc-input-format, we return a different InputStream
+			return new InputStream() {
 
-            private RecordReader<Object, Text> reader;
-            private Object key;
-            private Text value;
-            private boolean hasMore = false;
-            private int EOL = "\n".getBytes()[0];
-            private Text pendingValue = null;
-            private int currentSplitIndex = 0;
+				private RecordReader<LongWritable, BytesRefArrayWritable> reader;
+				private LongWritable key;
+				private BytesRefArrayWritable value;
+				private boolean hasMore = false;
+				private int EOL = "\n".getBytes()[0];
+				private byte delimiter = 0x01;
+				private boolean pendingValue = false;
+				private int currentSplitIndex = 0;
 
-            @SuppressWarnings("unchecked")
-            private boolean moveToNext() throws IOException {
-                for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-                    /**
-                     * read all the partitions scheduled to the current node
-                     */
-                    if (readSchedule[currentSplitIndex].equals(nodeName)) {
-                        /**
-                         * pick an unread split to read
-                         * synchronize among simultaneous partitions in the same machine
-                         */
-                        synchronized (executed) {
-                            if (executed[currentSplitIndex] == false) {
-                                executed[currentSplitIndex] = true;
-                            } else {
-                                continue;
-                            }
-                        }
+				@SuppressWarnings("unchecked")
+				private boolean moveToNext() throws IOException {
+					for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+						/**
+						 * read all the partitions scheduled to the current node
+						 */
+						if (readSchedule[currentSplitIndex].equals(nodeName)) {
+							/**
+							 * pick an unread split to read
+							 * synchronize among simultaneous partitions in the same machine
+							 */
+							synchronized (executed) {
+								if (executed[currentSplitIndex] == false) {
+									executed[currentSplitIndex] = true;
+								} else {
+									continue;
+								}
+							}
 
-                        /**
-                         * read the split
-                         */
-                        reader = getRecordReader(currentSplitIndex);
-                        key = reader.createKey();
-                        value = (Text) reader.createValue();
-                        return true;
-                    }
-                }
-                return false;
-            }
+							/**
+							 * read the split
+							 */
+							reader = getRecordReader(currentSplitIndex);
+							key = reader.createKey();
+							value = reader.createValue();
+							return true;
+						}
+					}
+					return false;
+				}
 
-            @Override
-            public int read(byte[] buffer, int offset, int len) throws IOException {
-                if (reader == null) {
-                    if (!moveToNext()) {
-                        //nothing to read
-                        return -1;
-                    }
-                }
+				@Override
+				public int read(byte[] buffer, int offset, int len) throws IOException {
+					if (reader == null) {
+						if (!moveToNext()) {
+							//nothing to read
+							return -1;
+						}
+					}
 
-                int numBytes = 0;
-                if (pendingValue != null) {
-                    System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-                    buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-                    numBytes += pendingValue.getLength() + 1;
-                    pendingValue = null;
-                }
+					int numBytes = 0;
+					if (pendingValue) {
+						//last value didn't fit into buffer
+						int sizeOfNextTuple = getTupleSize(value) + 1;
+						if(sizeOfNextTuple > len)
+						{
+							return 0;
+						}
+						copyCurrentTuple(buffer, offset + numBytes);
+						buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+						numBytes += sizeOfNextTuple;
+						//set pending to false
+						pendingValue = false;
+					}
 
-                while (numBytes < len) {
-                    hasMore = reader.next(key, value);
-                    if (!hasMore) {
-                        while (moveToNext()) {
-                            hasMore = reader.next(key, value);
-                            if (hasMore) {
-                                //move to the next non-empty split
-                                break;
-                            }
-                        }
-                    }
-                    if (!hasMore) {
-                        return (numBytes == 0) ? -1 : numBytes;
-                    }
-                    int sizeOfNextTuple = value.getLength() + 1;
-                    if (numBytes + sizeOfNextTuple > len) {
-                        // cannot add tuple to current buffer
-                        // but the reader has moved pass the fetched tuple
-                        // we need to store this for a subsequent read call.
-                        // and return this then.
-                        pendingValue = value;
-                        break;
-                    } else {
-                        System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-                        buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-                        numBytes += sizeOfNextTuple;
-                    }
-                }
-                return numBytes;
-            }
+					while (numBytes < len) {
+						hasMore = reader.next(key, value);
+						if (!hasMore) {
+							while (moveToNext()) {
+								hasMore = reader.next(key, value);
+								if (hasMore) {
+									//move to the next non-empty split
+									break;
+								}
+							}
+						}
+						if (!hasMore) {
+							return (numBytes == 0) ? -1 : numBytes;
+						}
+						int sizeOfNextTuple = getTupleSize(value) + 1;
+						if (numBytes + sizeOfNextTuple > len) {
+							// cannot add tuple to current buffer
+							// but the reader has moved pass the fetched tuple
+							// we need to store this for a subsequent read call.
+							// and return this then.
+							pendingValue = true;
+							break;
+						} else {
+							//copy
+							copyCurrentTuple(buffer, offset + numBytes);
+							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+						}
+					}
+					return numBytes;
+				}
 
-            @Override
-            public int read() throws IOException {
-                throw new NotImplementedException("Use read(byte[], int, int");
-            }
+				private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
+					int rcOffset = 0;
+					for(int i=0; i< value.size(); i++)
+					{
+						System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
+						rcOffset += value.get(i).getLength() + 1;
+						buffer[rcOffset - 1] = delimiter;
+					}
+				}
 
-            private RecordReader getRecordReader(int slitIndex) throws IOException {
-                if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-                    SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-                    RecordReader reader = format.getRecordReader(
-                            (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-                    return reader;
-                } else {
-                    TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-                    RecordReader reader = format.getRecordReader(
-                            (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-                    return reader;
-                }
-            }
+				private int getTupleSize(BytesRefArrayWritable value2) {
+					int size=0;
+					//loop over rc column and add lengths
+					for(int i=0; i< value.size(); i++)
+					{
+						size += value.get(i).getLength();
+					}
+					//add delimeters bytes sizes
+					size += value.size() -1;
+					return size;
+				}
 
-        };
+				@Override
+				public int read() throws IOException {
+					throw new NotImplementedException("Use read(byte[], int, int");
+				}
 
-    }
+				private RecordReader getRecordReader(int slitIndex) throws IOException {
+					RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
+					RecordReader reader = format.getRecordReader(
+							(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+					return reader;
+				}
 
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return clusterLocations;
-    }
+			};
+		}
+		else
+		{
+			return new InputStream() {
+
+				private RecordReader<Object, Text> reader;
+				private Object key;
+				private Text value;
+				private boolean hasMore = false;
+				private int EOL = "\n".getBytes()[0];
+				private Text pendingValue = null;
+				private int currentSplitIndex = 0;
+
+				@SuppressWarnings("unchecked")
+				private boolean moveToNext() throws IOException {
+					for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+						/**
+						 * read all the partitions scheduled to the current node
+						 */
+						if (readSchedule[currentSplitIndex].equals(nodeName)) {
+							/**
+							 * pick an unread split to read
+							 * synchronize among simultaneous partitions in the same machine
+							 */
+							synchronized (executed) {
+								if (executed[currentSplitIndex] == false) {
+									executed[currentSplitIndex] = true;
+								} else {
+									continue;
+								}
+							}
+
+							/**
+							 * read the split
+							 */
+							reader = getRecordReader(currentSplitIndex);
+							key = reader.createKey();
+							value = (Text) reader.createValue();
+							return true;
+						}
+					}
+					return false;
+				}
+
+				@Override
+				public int read(byte[] buffer, int offset, int len) throws IOException {
+					if (reader == null) {
+						if (!moveToNext()) {
+							//nothing to read
+							return -1;
+						}
+					}
+
+					int numBytes = 0;
+					if (pendingValue != null) {
+						int sizeOfNextTuple = pendingValue.getLength() + 1;
+						if(sizeOfNextTuple > len)
+						{
+							return 0;
+						}
+						System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+						buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+						numBytes += pendingValue.getLength() + 1;
+						pendingValue = null;
+					}
+
+					while (numBytes < len) {
+						hasMore = reader.next(key, value);
+						if (!hasMore) {
+							while (moveToNext()) {
+								hasMore = reader.next(key, value);
+								if (hasMore) {
+									//move to the next non-empty split
+									break;
+								}
+							}
+						}
+						if (!hasMore) {
+							return (numBytes == 0) ? -1 : numBytes;
+						}
+						int sizeOfNextTuple = value.getLength() + 1;
+						if (numBytes + sizeOfNextTuple > len) {
+							// cannot add tuple to current buffer
+							// but the reader has moved pass the fetched tuple
+							// we need to store this for a subsequent read call.
+							// and return this then.
+							pendingValue = value;
+							break;
+						} else {
+							System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+							buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+						}
+					}
+					return numBytes;
+				}
+
+				@Override
+				public int read() throws IOException {
+					throw new NotImplementedException("Use read(byte[], int, int");
+				}
+
+				private RecordReader getRecordReader(int slitIndex) throws IOException {
+					if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+						SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+						RecordReader reader = format.getRecordReader(
+								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+						return reader;
+					} else {
+						TextInputFormat format = (TextInputFormat) conf.getInputFormat();
+						RecordReader reader = format.getRecordReader(
+								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+						return reader;
+					}
+				}
+
+			};
+		}
+
+	}
+
+	@Override
+	public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+		return clusterLocations;
+	}
 
 }
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
new file mode 100644
index 0000000..59b39c5
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
@@ -0,0 +1,1208 @@
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Map;
+
+import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
+import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.FileSplit;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.RecordReader;
+import org.apache.hadoop.mapred.Reporter;
+import org.apache.hadoop.mapred.SequenceFileInputFormat;
+import org.apache.hadoop.mapred.TextInputFormat;
+import org.apache.hadoop.mapred.Counters.Counter;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+
+/**
+ * Provides functionality for reading external files and attach RID info to them before they are sent to the parser
+ * 
+ *  Room for optimization: get numbers of indexing fields (or fields names for adm) and do a quick filtering before sending to parser
+ *  
+ */
+@SuppressWarnings({ "deprecation", "rawtypes" })
+public class HDFSIndexingAdapter extends FileSystemBasedAdapter {
+
+	private static final long serialVersionUID = 1L;
+	private transient String[] readSchedule;
+	private transient boolean executed[];
+	private transient InputSplit[] inputSplits;
+	private transient JobConf conf;
+	private transient AlgebricksPartitionConstraint clusterLocations;
+	private final Map<String,Integer> files;
+	private transient String nodeName;
+	public static final byte[] fileNameFieldNameWithRecOpeningBraces = "{\"_file-name\":\"".getBytes();
+	public static final byte[] fileNameFieldClosingQuotation = "\"".getBytes();
+	public static final byte[] fileNumberFieldNameWithRecOpeningBraces = "{\"_file-number\":".getBytes();
+	public static final byte[] bytelocationFieldName = ",\"_byte-location\":".getBytes();
+	public static final byte[] bytelocationValueEnd = "i64,".getBytes();
+
+	public HDFSIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
+			AlgebricksPartitionConstraint clusterLocations, Map<String,Integer> files) {
+		super(atype);
+		this.readSchedule = readSchedule;
+		this.executed = executed;
+		this.inputSplits = inputSplits;
+		this.conf = conf;
+		this.clusterLocations = clusterLocations;
+		this.files = files;
+	}
+
+	@Override
+	public void configure(Map<String, Object> arguments) throws Exception {
+		this.configuration = arguments;
+		configureFormat();
+	}
+
+	public AdapterType getAdapterType() {
+		return AdapterType.READ;
+	}
+
+	@Override
+	public void initialize(IHyracksTaskContext ctx) throws Exception {
+		this.ctx = ctx;
+		this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+	}
+
+	private Reporter getReporter() {
+		Reporter reporter = new Reporter() {
+
+			@Override
+			public Counter getCounter(Enum<?> arg0) {
+				return null;
+			}
+
+			@Override
+			public Counter getCounter(String arg0, String arg1) {
+				return null;
+			}
+
+			@Override
+			public InputSplit getInputSplit() throws UnsupportedOperationException {
+				return null;
+			}
+
+			@Override
+			public void incrCounter(Enum<?> arg0, long arg1) {
+			}
+
+			@Override
+			public void incrCounter(String arg0, String arg1, long arg2) {
+			}
+
+			@Override
+			public void setStatus(String arg0) {
+			}
+
+			@Override
+			public void progress() {
+			}
+		};
+
+		return reporter;
+	}
+
+	@Override
+	public InputStream getInputStream(int partition) throws IOException {
+		if(files == null)
+		{
+			if(conf.getInputFormat() instanceof RCFileInputFormat)
+			{
+				//indexing rc input format
+				return new InputStream() {
+
+					private RecordReader<LongWritable, BytesRefArrayWritable> reader;
+					private LongWritable key;
+					private BytesRefArrayWritable value;
+					private boolean hasMore = false;
+					private int EOL = "\n".getBytes()[0];
+					private byte delimiter = 0x01;
+					private boolean pendingValue = false;
+					private int currentSplitIndex = 0;
+					private byte[] fileName;
+					private byte[] byteLocation;
+					private byte[] rowNumberBytes;
+					private long blockByteLocation;
+					private long NextblockByteLocation;
+					private int rowNumber;
+
+					@SuppressWarnings("unchecked")
+					private boolean moveToNext() throws IOException {
+						for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+							/**
+							 * read all the partitions scheduled to the current node
+							 */
+							if (readSchedule[currentSplitIndex].equals(nodeName)) {
+								/**
+								 * pick an unread split to read
+								 * synchronize among simultaneous partitions in the same machine
+								 */
+								synchronized (executed) {
+									if (executed[currentSplitIndex] == false) {
+										executed[currentSplitIndex] = true;
+									} else {
+										continue;
+									}
+								}
+
+								/**
+								 * read the split
+								 */
+								reader = getRecordReader(currentSplitIndex);
+								key = reader.createKey();
+								value = reader.createValue();
+								fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
+								blockByteLocation = reader.getPos();
+								pendingValue = reader.next(key, value);
+								NextblockByteLocation = reader.getPos();
+								rowNumber = 1;
+								byteLocation = String.valueOf(blockByteLocation).getBytes("UTF-8");
+								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
+								return true;
+							}
+						}
+						return false;
+					}
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+						if (reader == null) {
+							if (!moveToNext()) {
+								//nothing to read
+								return -1;
+							}
+						}
+
+						int numBytes = 0;
+						if (pendingValue) {
+							//last value didn't fit into buffer
+							// 1 for EOL
+							int sizeOfNextTuple = getTupleSize(value) + 1;
+							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileName.length + 3 > len) {
+								return 0;
+							}
+
+							//copy filename
+							System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
+							buffer[offset + numBytes + fileName.length] = delimiter;
+							numBytes += fileName.length + 1;
+
+							//copy byte location
+							System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+							buffer[offset + numBytes + byteLocation.length] = delimiter;
+							numBytes += byteLocation.length + 1;
+
+							//copy row number
+							System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
+							buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
+							numBytes += rowNumberBytes.length + 1;
+
+							copyCurrentTuple(buffer, offset + numBytes);
+							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = false;
+						}
+
+						while (numBytes < len) {
+							hasMore = reader.next(key, value);
+							if (!hasMore) {
+								while (moveToNext()) {
+									hasMore = reader.next(key, value);
+									if (hasMore) {
+										//move to the next non-empty split
+										break;
+									}
+								}
+							}
+							if (!hasMore) {
+								return (numBytes == 0) ? -1 : numBytes;
+							}
+
+							//check if moved to next block
+							blockByteLocation = reader.getPos();
+							if(blockByteLocation != NextblockByteLocation)
+							{
+								//moved to a new block, reset stuff
+								//row number
+								rowNumber = 1;
+								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
+
+								//block location
+								byteLocation = String.valueOf(NextblockByteLocation).getBytes("UTF-8");
+								NextblockByteLocation = blockByteLocation;
+							}
+							else
+							{
+								rowNumber += 1;
+								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
+							}
+
+							int sizeOfNextTuple = getTupleSize(value) + 1;
+							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileName.length + 3 > len) {
+								// cannot add tuple to current buffer
+								// but the reader has moved pass the fetched tuple
+								// we need to store this for a subsequent read call.
+								// and return this then.
+								pendingValue = true;
+								break;
+							} else {
+								//copy filename
+								System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
+								buffer[offset + numBytes + fileName.length] = delimiter;
+								numBytes += fileName.length + 1;
+
+								//copy byte location
+								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+								buffer[offset + numBytes + byteLocation.length] = delimiter;
+								numBytes += byteLocation.length + 1;
+
+								//copy row number
+								System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
+								buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
+								numBytes += rowNumberBytes.length + 1;
+
+								copyCurrentTuple(buffer, offset + numBytes);
+								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+						}
+						return numBytes;
+					}
+
+					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
+						int rcOffset = 0;
+						for(int i=0; i< value.size(); i++)
+						{
+							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
+							rcOffset += value.get(i).getLength() + 1;
+							buffer[rcOffset - 1] = delimiter;
+						}
+					}
+
+					private int getTupleSize(BytesRefArrayWritable value2) {
+						int size=0;
+						//loop over rc column and add lengths
+						for(int i=0; i< value.size(); i++)
+						{
+							size += value.get(i).getLength();
+						}
+						//add delimeters bytes sizes
+						size += value.size() -1;
+						return size;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+
+					private RecordReader getRecordReader(int slitIndex) throws IOException {
+						RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
+						RecordReader reader = format.getRecordReader(
+								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+						return reader;
+					}
+
+				};
+			}
+			else
+			{
+				//get content format
+				if(configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT))
+				{
+					//reading data and RIDs for delimited text
+					return new InputStream() {
+
+						private RecordReader<Object, Text> reader;
+						private Object key;
+						private Text value;
+						private boolean hasMore = false;
+						private int EOL = "\n".getBytes()[0];
+						private Text pendingValue = null;
+						private int currentSplitIndex = 0;
+						private byte[] fileName;
+						private byte[] byteLocation;
+						private byte delimiter = ((String)configuration.get(KEY_DELIMITER)).getBytes()[0];
+
+						@SuppressWarnings("unchecked")
+						private boolean moveToNext() throws IOException {
+							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+								/**
+								 * read all the partitions scheduled to the current node
+								 */
+								if (readSchedule[currentSplitIndex].equals(nodeName)) {
+									/**
+									 * pick an unread split to read
+									 * synchronize among simultaneous partitions in the same machine
+									 */
+									synchronized (executed) {
+										if (executed[currentSplitIndex] == false) {
+											executed[currentSplitIndex] = true;
+										} else {
+											continue;
+										}
+									}
+
+									/**
+									 * read the split
+									 */
+									reader = getRecordReader(currentSplitIndex);
+									key = reader.createKey();
+									value = (Text) reader.createValue();
+									fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
+									return true;
+								}
+							}
+							return false;
+						}
+
+						@Override
+						public int read(byte[] buffer, int offset, int len) throws IOException {
+							if (reader == null) {
+								if (!moveToNext()) {
+									//nothing to read
+									return -1;
+								}
+							}
+
+							int numBytes = 0;
+							if (pendingValue != null) {
+								int sizeOfNextTuple = pendingValue.getLength() + 1;
+								if (numBytes + sizeOfNextTuple +byteLocation.length + fileName.length + 2> len)
+								{
+									return numBytes;
+								}
+								//copy filename
+								System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
+								buffer[offset + numBytes + fileName.length] = delimiter;
+								numBytes += fileName.length + 1;
+
+								//copy byte location
+								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+								buffer[offset + numBytes + byteLocation.length] = delimiter;
+								numBytes += byteLocation.length + 1;
+
+								//copy actual value
+								System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+								buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+								numBytes += pendingValue.getLength() + 1;
+								pendingValue = null;
+							}
+
+							while (numBytes < len) {
+								//get reader position before you actually read
+								byteLocation = String.valueOf(reader.getPos()).getBytes();
+								hasMore = reader.next(key, value);
+								if (!hasMore) {
+									while (moveToNext()) {
+										//get reader position before you actually read
+										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
+										hasMore = reader.next(key, value);
+										if (hasMore) {
+											//move to the next non-empty split
+											break;
+										}
+									}
+								}
+								if (!hasMore) {
+									return (numBytes == 0) ? -1 : numBytes;
+								}
+								int sizeOfNextTuple = value.getLength() + 1;
+								if (numBytes + sizeOfNextTuple +byteLocation.length + fileName.length + 2> len) {
+									// cannot add tuple to current buffer
+									// but the reader has moved pass the fetched tuple
+									// we need to store this for a subsequent read call.
+									// and return this then.
+									pendingValue = value;
+									break;
+								} else {
+									//copy filename
+									System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
+									buffer[offset + numBytes + fileName.length] = delimiter;
+									numBytes += fileName.length + 1;
+
+									//copy byte location
+									System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+									buffer[offset + numBytes + byteLocation.length] = delimiter;
+									numBytes += byteLocation.length + 1;
+
+									//Copy actual value
+									System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+									buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+									numBytes += sizeOfNextTuple;
+								}
+							}
+							return numBytes;
+						}
+
+						@Override
+						public int read() throws IOException {
+							throw new NotImplementedException("Use read(byte[], int, int");
+						}
+
+						private RecordReader getRecordReader(int slitIndex) throws IOException {
+							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							} else {
+								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							}
+						}
+
+					};
+				}
+				else if((configuration.get(KEY_FORMAT).equals(FORMAT_ADM)))
+				{
+					//reading data and RIDs for adm formatted data
+					return new InputStream() {
+
+						private RecordReader<Object, Text> reader;
+						private Object key;
+						private Text value;
+						private boolean hasMore = false;
+						private int EOL = "\n".getBytes()[0];
+						private Text pendingValue = null;
+						private int currentSplitIndex = 0;
+						private byte[] fileName;
+						private byte[] byteLocation;
+
+						@SuppressWarnings("unchecked")
+						private boolean moveToNext() throws IOException {
+							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+								/**
+								 * read all the partitions scheduled to the current node
+								 */
+								if (readSchedule[currentSplitIndex].equals(nodeName)) {
+									/**
+									 * pick an unread split to read
+									 * synchronize among simultaneous partitions in the same machine
+									 */
+									synchronized (executed) {
+										if (executed[currentSplitIndex] == false) {
+											executed[currentSplitIndex] = true;
+										} else {
+											continue;
+										}
+									}
+
+									/**
+									 * read the split
+									 */
+									reader = getRecordReader(currentSplitIndex);
+									key = reader.createKey();
+									value = (Text) reader.createValue();
+									fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
+									return true;
+								}
+							}
+							return false;
+						}
+
+						@Override
+						public int read(byte[] buffer, int offset, int len) throws IOException {
+							if (reader == null) {
+								if (!moveToNext()) {
+									//nothing to read
+									return -1;
+								}
+							}
+
+							int numBytes = 0;
+							if (pendingValue != null) {
+								int firstFieldLocation = value.find("\"");
+								int admValueSize = value.getLength();
+								if(firstFieldLocation >= 0)
+								{
+									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
+									int sizeOfNextTupleAndRID = fileNameFieldNameWithRecOpeningBraces.length + fileName.length + fileNameFieldClosingQuotation.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
+									if (numBytes + sizeOfNextTupleAndRID > len) {
+										// still cannot add tuple to current buffer
+										// return 0 so parser would double the buffer size.
+										return 0;
+									} else {
+										//copy fileNameFieldNameWithRecOpeningBraces
+										System.arraycopy(fileNameFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNameFieldNameWithRecOpeningBraces.length);
+										numBytes += fileNameFieldNameWithRecOpeningBraces.length;
+										//copy fileName
+										System.arraycopy(fileName, 0, buffer, offset + numBytes,fileName.length);
+										numBytes += fileName.length;
+										//copy fileName closing quotation
+										System.arraycopy(fileNameFieldClosingQuotation, 0, buffer, offset + numBytes,fileNameFieldClosingQuotation.length);
+										numBytes += fileNameFieldClosingQuotation.length;
+										//copy bytelocationFieldName
+										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
+										numBytes += bytelocationFieldName.length;
+										//copy byte location value
+										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
+										numBytes += byteLocation.length;
+										//copy byte location field end 
+										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
+										numBytes += bytelocationValueEnd.length;
+										//copy the actual adm instance
+										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
+										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
+										numBytes += admValueSize - firstFieldLocation +1;
+									}
+								}
+								pendingValue = null;
+							}
+
+							while (numBytes < len) {
+								//get reader position before you actually read
+								byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
+								hasMore = reader.next(key, value);
+								if (!hasMore) {
+									while (moveToNext()) {
+										//get reader position before you actually read
+										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
+										hasMore = reader.next(key, value);
+										if (hasMore) {
+											//move to the next non-empty split
+											break;
+										}
+									}
+								}
+								if (!hasMore) {
+									return (numBytes == 0) ? -1 : numBytes;
+								}
+								//get the index of the first field name
+								int firstFieldLocation = value.find("\"");
+								int admValueSize = value.getLength();
+								if(firstFieldLocation >= 0)
+								{
+									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
+									int sizeOfNextTupleAndRID = fileNameFieldNameWithRecOpeningBraces.length + fileName.length + fileNameFieldClosingQuotation.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
+									if (numBytes + sizeOfNextTupleAndRID > len) {
+										// cannot add tuple to current buffer
+										// but the reader has moved pass the fetched tuple
+										// we need to store this for a subsequent read call.
+										// and return this then.
+										pendingValue = value;
+										break;
+									} else {
+										//copy fileNameFieldNameWithRecOpeningBraces
+										System.arraycopy(fileNameFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNameFieldNameWithRecOpeningBraces.length);
+										numBytes += fileNameFieldNameWithRecOpeningBraces.length;
+										//copy fileName
+										System.arraycopy(fileName, 0, buffer, offset + numBytes,fileName.length);
+										numBytes += fileName.length;
+										//copy fileName closing quotation
+										System.arraycopy(fileNameFieldClosingQuotation, 0, buffer, offset + numBytes,fileNameFieldClosingQuotation.length);
+										numBytes += fileNameFieldClosingQuotation.length;
+										//copy bytelocationFieldName
+										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
+										numBytes += bytelocationFieldName.length;
+										//copy byte location value
+										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
+										numBytes += byteLocation.length;
+										//copy byte location field end 
+										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
+										numBytes += bytelocationValueEnd.length;
+										//copy the actual adm instance
+										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
+										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
+										numBytes += admValueSize - firstFieldLocation +1;
+									}
+								}
+							}
+							return numBytes;
+						}
+
+						@Override
+						public int read() throws IOException {
+							throw new NotImplementedException("Use read(byte[], int, int");
+						}
+
+						private RecordReader getRecordReader(int slitIndex) throws IOException {
+							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							} else {
+								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							}
+						}
+
+					};
+				}
+				else
+				{
+					throw new IOException("Can't index " +configuration.get(KEY_FORMAT)+" input");
+				}
+			}
+		}
+		else
+		{
+			if(conf.getInputFormat() instanceof RCFileInputFormat)
+			{
+				//indexing rc input format
+				return new InputStream() {
+
+					private RecordReader<LongWritable, BytesRefArrayWritable> reader;
+					private LongWritable key;
+					private BytesRefArrayWritable value;
+					private boolean hasMore = false;
+					private int EOL = "\n".getBytes()[0];
+					private byte delimiter = 0x01;
+					private boolean pendingValue = false;
+					private int currentSplitIndex = 0;
+					private byte[] fileNumber;
+					private byte[] byteLocation;
+					private byte[] rowNumberBytes;
+					private Integer file;
+					private long blockByteLocation;
+					private long NextblockByteLocation;
+					private int rowNumber;
+
+					@SuppressWarnings("unchecked")
+					private boolean moveToNext() throws IOException {
+						for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+							/**
+							 * read all the partitions scheduled to the current node
+							 */
+							if (readSchedule[currentSplitIndex].equals(nodeName)) {
+								/**
+								 * pick an unread split to read
+								 * synchronize among simultaneous partitions in the same machine
+								 */
+								synchronized (executed) {
+									if (executed[currentSplitIndex] == false) {
+										executed[currentSplitIndex] = true;
+									} else {
+										continue;
+									}
+								}
+
+								/**
+								 * read the split
+								 */
+								reader = getRecordReader(currentSplitIndex);
+								key = reader.createKey();
+								value = reader.createValue();
+								//getting the file number
+								file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
+								if(file == null)
+								{
+									throw new HyracksException("a file was not found in the map while indexing");
+								}
+								fileNumber = String.valueOf(file).getBytes("UTF-8");
+								blockByteLocation = reader.getPos();
+								pendingValue = reader.next(key, value);
+								NextblockByteLocation = reader.getPos();
+								rowNumber = 1;
+								byteLocation = String.valueOf(blockByteLocation).getBytes("UTF-8");
+								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
+								return true;
+							}
+						}
+						return false;
+					}
+
+					@Override
+					public int read(byte[] buffer, int offset, int len) throws IOException {
+						if (reader == null) {
+							if (!moveToNext()) {
+								//nothing to read
+								return -1;
+							}
+						}
+
+						int numBytes = 0;
+						if (pendingValue) {
+							//last value didn't fit into buffer
+							// 1 for EOL
+							int sizeOfNextTuple = getTupleSize(value) + 1;
+							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileNumber.length + 3 > len) {
+								return 0;
+							}
+							//copy file number
+							System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
+							buffer[offset + numBytes + fileNumber.length] = delimiter;
+							numBytes += fileNumber.length + 1;
+
+							//copy byte location
+							System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+							buffer[offset + numBytes + byteLocation.length] = delimiter;
+							numBytes += byteLocation.length + 1;
+
+							//copy row number
+							System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
+							buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
+							numBytes += rowNumberBytes.length + 1;
+
+							copyCurrentTuple(buffer, offset + numBytes);
+							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+							numBytes += sizeOfNextTuple;
+							//set pending to false
+							pendingValue = false;
+						}
+
+						while (numBytes < len) {
+							hasMore = reader.next(key, value);
+							if (!hasMore) {
+								while (moveToNext()) {
+									hasMore = reader.next(key, value);
+									if (hasMore) {
+										//move to the next non-empty split
+										break;
+									}
+								}
+							}
+							if (!hasMore) {
+								return (numBytes == 0) ? -1 : numBytes;
+							}
+
+							//check if moved to next block
+							blockByteLocation = reader.getPos();
+							if(blockByteLocation != NextblockByteLocation)
+							{
+								//moved to a new block, reset stuff
+								//row number
+								rowNumber = 1;
+								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
+
+								//block location
+								byteLocation = String.valueOf(NextblockByteLocation).getBytes("UTF-8");
+								NextblockByteLocation = blockByteLocation;
+							}
+							else
+							{
+								rowNumber += 1;
+								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
+							}
+
+							int sizeOfNextTuple = getTupleSize(value) + 1;
+							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileNumber.length + 3 > len) {
+								// cannot add tuple to current buffer
+								// but the reader has moved pass the fetched tuple
+								// we need to store this for a subsequent read call.
+								// and return this then.
+								pendingValue = true;
+								break;
+							} else {
+								//copy file number
+								System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
+								buffer[offset + numBytes + fileNumber.length] = delimiter;
+								numBytes += fileNumber.length + 1;
+
+								//copy byte location
+								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+								buffer[offset + numBytes + byteLocation.length] = delimiter;
+								numBytes += byteLocation.length + 1;
+
+								//copy row number
+								System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
+								buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
+								numBytes += rowNumberBytes.length + 1;
+
+								copyCurrentTuple(buffer, offset + numBytes);
+								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
+								numBytes += sizeOfNextTuple;
+							}
+						}
+						return numBytes;
+					}
+
+					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
+						int rcOffset = 0;
+						for(int i=0; i< value.size(); i++)
+						{
+							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
+							rcOffset += value.get(i).getLength() + 1;
+							buffer[rcOffset - 1] = delimiter;
+						}
+					}
+
+					private int getTupleSize(BytesRefArrayWritable value2) {
+						int size=0;
+						//loop over rc column and add lengths
+						for(int i=0; i< value.size(); i++)
+						{
+							size += value.get(i).getLength();
+						}
+						//add delimeters bytes sizes
+						size += value.size() -1;
+						return size;
+					}
+
+					@Override
+					public int read() throws IOException {
+						throw new NotImplementedException("Use read(byte[], int, int");
+					}
+
+					private RecordReader getRecordReader(int slitIndex) throws IOException {
+						RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
+						RecordReader reader = format.getRecordReader(
+								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+						return reader;
+					}
+
+				};
+			}
+			else
+			{
+				//get content format
+				if(configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT))
+				{
+					//reading data and RIDs for delimited text
+					return new InputStream() {
+
+						private RecordReader<Object, Text> reader;
+						private Object key;
+						private Text value;
+						private boolean hasMore = false;
+						private int EOL = "\n".getBytes()[0];
+						private Text pendingValue = null;
+						private int currentSplitIndex = 0;
+						private Integer file;
+						private byte[] fileNumber;
+						private byte[] byteLocation;
+						private byte delimiter = ((String)configuration.get(KEY_DELIMITER)).getBytes()[0];
+
+						@SuppressWarnings("unchecked")
+						private boolean moveToNext() throws IOException {
+							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+								/**
+								 * read all the partitions scheduled to the current node
+								 */
+								if (readSchedule[currentSplitIndex].equals(nodeName)) {
+									/**
+									 * pick an unread split to read
+									 * synchronize among simultaneous partitions in the same machine
+									 */
+									synchronized (executed) {
+										if (executed[currentSplitIndex] == false) {
+											executed[currentSplitIndex] = true;
+										} else {
+											continue;
+										}
+									}
+
+									/**
+									 * read the split
+									 */
+									reader = getRecordReader(currentSplitIndex);
+									key = reader.createKey();
+									value = (Text) reader.createValue();
+									file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
+									if(file == null)
+									{
+										throw new HyracksException("The file:"+((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath()+" was not found in the map while indexing");
+									}
+									fileNumber = String.valueOf(file).getBytes("UTF-8");
+									return true;
+								}
+							}
+							return false;
+						}
+
+						@Override
+						public int read(byte[] buffer, int offset, int len) throws IOException {
+							if (reader == null) {
+								if (!moveToNext()) {
+									//nothing to read
+									return -1;
+								}
+							}
+
+							int numBytes = 0;
+							if (pendingValue != null) {
+								int sizeOfNextTuple = pendingValue.getLength() + 1;
+								if (numBytes + sizeOfNextTuple +byteLocation.length + fileNumber.length + 2> len)
+								{
+									return numBytes;
+								}
+								//copy file number
+								System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
+								buffer[offset + numBytes + fileNumber.length] = delimiter;
+								numBytes += fileNumber.length + 1;
+
+								//copy byte location
+								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+								buffer[offset + numBytes + byteLocation.length] = delimiter;
+								numBytes += byteLocation.length + 1;
+
+								//copy actual value
+								System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+								buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+								numBytes += pendingValue.getLength() + 1;
+								pendingValue = null;
+							}
+
+							while (numBytes < len) {
+								//get reader position before you actually read
+								byteLocation = String.valueOf(reader.getPos()).getBytes();
+								hasMore = reader.next(key, value);
+								if (!hasMore) {
+									while (moveToNext()) {
+										//get reader position before you actually read
+										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
+										hasMore = reader.next(key, value);
+										if (hasMore) {
+											//move to the next non-empty split
+											break;
+										}
+									}
+								}
+								if (!hasMore) {
+									return (numBytes == 0) ? -1 : numBytes;
+								}
+								int sizeOfNextTuple = value.getLength() + 1;
+								if (numBytes + sizeOfNextTuple +byteLocation.length + fileNumber.length + 2> len) {
+									// cannot add tuple to current buffer
+									// but the reader has moved pass the fetched tuple
+									// we need to store this for a subsequent read call.
+									// and return this then.
+									pendingValue = value;
+									break;
+								} else {
+									//copy file number
+									System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
+									buffer[offset + numBytes + fileNumber.length] = delimiter;
+									numBytes += fileNumber.length + 1;
+
+									//copy byte location
+									System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
+									buffer[offset + numBytes + byteLocation.length] = delimiter;
+									numBytes += byteLocation.length + 1;
+
+									//Copy actual value
+									System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+									buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+									numBytes += sizeOfNextTuple;
+								}
+							}
+							return numBytes;
+						}
+
+						@Override
+						public int read() throws IOException {
+							throw new NotImplementedException("Use read(byte[], int, int");
+						}
+
+						private RecordReader getRecordReader(int slitIndex) throws IOException {
+							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							} else {
+								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							}
+						}
+
+					};
+				}
+				else if((configuration.get(KEY_FORMAT).equals(FORMAT_ADM)))
+				{
+					//reading data and RIDs for adm formatted data
+					return new InputStream() {
+
+						private RecordReader<Object, Text> reader;
+						private Object key;
+						private Text value;
+						private boolean hasMore = false;
+						private int EOL = "\n".getBytes()[0];
+						private Text pendingValue = null;
+						private int currentSplitIndex = 0;
+						private Integer file;
+						private byte[] fileNumber;
+						private byte[] byteLocation;
+
+						@SuppressWarnings("unchecked")
+						private boolean moveToNext() throws IOException {
+							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+								/**
+								 * read all the partitions scheduled to the current node
+								 */
+								if (readSchedule[currentSplitIndex].equals(nodeName)) {
+									/**
+									 * pick an unread split to read
+									 * synchronize among simultaneous partitions in the same machine
+									 */
+									synchronized (executed) {
+										if (executed[currentSplitIndex] == false) {
+											executed[currentSplitIndex] = true;
+										} else {
+											continue;
+										}
+									}
+
+									/**
+									 * read the split
+									 */
+									reader = getRecordReader(currentSplitIndex);
+									key = reader.createKey();
+									value = (Text) reader.createValue();
+									file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
+									if(file == null)
+									{
+										throw new HyracksException("a file was not found in the map while indexing");
+									}
+									fileNumber = String.valueOf(file).getBytes("UTF-8");
+									return true;
+								}
+							}
+							return false;
+						}
+
+						@Override
+						public int read(byte[] buffer, int offset, int len) throws IOException {
+							if (reader == null) {
+								if (!moveToNext()) {
+									//nothing to read
+									return -1;
+								}
+							}
+
+							int numBytes = 0;
+							if (pendingValue != null) {
+								int firstFieldLocation = value.find("\"");
+								int admValueSize = value.getLength();
+								if(firstFieldLocation >= 0)
+								{
+									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
+									int sizeOfNextTupleAndRID = fileNumberFieldNameWithRecOpeningBraces.length + fileNumber.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
+									if (numBytes + sizeOfNextTupleAndRID > len) {
+										// still cannot add tuple to current buffer
+										// return 0 so parser would double the buffer size.
+										return 0;
+									} else {
+										//copy fileNumberFieldNameWithRecOpeningBraces
+										System.arraycopy(fileNumberFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNumberFieldNameWithRecOpeningBraces.length);
+										numBytes += fileNumberFieldNameWithRecOpeningBraces.length;
+										//copy file Number
+										System.arraycopy(fileNumber, 0, buffer, offset + numBytes,fileNumber.length);
+										numBytes += fileNumber.length;
+										//copy bytelocationFieldName
+										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
+										numBytes += bytelocationFieldName.length;
+										//copy byte location value
+										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
+										numBytes += byteLocation.length;
+										//copy byte location field end 
+										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
+										numBytes += bytelocationValueEnd.length;
+										//copy the actual adm instance
+										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
+										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
+										numBytes += admValueSize - firstFieldLocation +1;
+									}
+								}
+								pendingValue = null;
+							}
+
+							while (numBytes < len) {
+								//get reader position before you actually read
+								byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
+								hasMore = reader.next(key, value);
+								if (!hasMore) {
+									while (moveToNext()) {
+										//get reader position before you actually read
+										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
+										hasMore = reader.next(key, value);
+										if (hasMore) {
+											//move to the next non-empty split
+											break;
+										}
+									}
+								}
+								if (!hasMore) {
+									return (numBytes == 0) ? -1 : numBytes;
+								}
+								//get the index of the first field name
+								int firstFieldLocation = value.find("\"");
+								int admValueSize = value.getLength();
+								if(firstFieldLocation >= 0)
+								{
+									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
+									int sizeOfNextTupleAndRID = fileNumberFieldNameWithRecOpeningBraces.length + fileNumber.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
+									if (numBytes + sizeOfNextTupleAndRID > len) {
+										// cannot add tuple to current buffer
+										// but the reader has moved pass the fetched tuple
+										// we need to store this for a subsequent read call.
+										// and return this then.
+										pendingValue = value;
+										break;
+									} else {
+										//copy fileNumberFieldNameWithRecOpeningBraces
+										System.arraycopy(fileNumberFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNumberFieldNameWithRecOpeningBraces.length);
+										numBytes += fileNumberFieldNameWithRecOpeningBraces.length;
+										//copy fileNumber
+										System.arraycopy(fileNumber, 0, buffer, offset + numBytes,fileNumber.length);
+										numBytes += fileNumber.length;
+										//copy bytelocationFieldName
+										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
+										numBytes += bytelocationFieldName.length;
+										//copy byte location value
+										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
+										numBytes += byteLocation.length;
+										//copy byte location field end 
+										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
+										numBytes += bytelocationValueEnd.length;
+										//copy the actual adm instance
+										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
+										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
+										numBytes += admValueSize - firstFieldLocation +1;
+									}
+								}
+							}
+							return numBytes;
+						}
+
+						@Override
+						public int read() throws IOException {
+							throw new NotImplementedException("Use read(byte[], int, int");
+						}
+
+						private RecordReader getRecordReader(int slitIndex) throws IOException {
+							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							} else {
+								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
+								RecordReader reader = format.getRecordReader(
+										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+								return reader;
+							}
+						}
+
+					};
+				}
+				else
+				{
+					throw new IOException("Can't index " +configuration.get(KEY_FORMAT)+" input");
+				}
+			}
+		}
+	}
+
+	@Override
+	public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+		return clusterLocations;
+	}
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
new file mode 100644
index 0000000..178b106
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
@@ -0,0 +1,62 @@
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.util.Map;
+
+import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.mapred.JobConf;
+
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+
+/**
+ * Provides the functionality of fetching data in form of ADM records from a Hive dataset.
+ */
+@SuppressWarnings("deprecation")
+public class HiveIndexingAdapter extends AbstractDatasourceAdapter{
+
+    private static final long serialVersionUID = 1L;
+
+    public static final String HIVE_DATABASE = "database";
+    public static final String HIVE_TABLE = "table";
+    public static final String HIVE_HOME = "hive-home";
+    public static final String HIVE_METASTORE_URI = "metastore-uri";
+    public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
+    public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
+
+    private HDFSIndexingAdapter hdfsIndexingAdapter;
+
+    public HiveIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
+            AlgebricksPartitionConstraint clusterLocations, Map<String,Integer> files) {
+        this.hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
+        this.atype = atype;
+    }
+
+    @Override
+    public AdapterType getAdapterType() {
+        return AdapterType.READ;
+    }
+
+    @Override
+    public void configure(Map<String, Object> arguments) throws Exception {
+        this.configuration = arguments;
+        this.hdfsIndexingAdapter.configure(arguments);
+    }
+
+    @Override
+    public void initialize(IHyracksTaskContext ctx) throws Exception {
+    	hdfsIndexingAdapter.initialize(ctx);
+    }
+
+    @Override
+    public void start(int partition, IFrameWriter writer) throws Exception {
+    	hdfsIndexingAdapter.start(partition, writer);
+    }
+
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return hdfsIndexingAdapter.getPartitionConstraint();
+    }
+
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
new file mode 100644
index 0000000..253f675
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
@@ -0,0 +1,17 @@
+package edu.uci.ics.asterix.external.dataset.adapter;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public interface IControlledAdapter extends Serializable{
+
+	public void initialize(IHyracksTaskContext ctx) throws Exception;
+	
+	public void processNextFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException;
+	
+	public void close(IFrameWriter writer) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java
new file mode 100644
index 0000000..47550a4
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java
@@ -0,0 +1,38 @@
+package edu.uci.ics.asterix.external.util;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.StringTokenizer;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
+import edu.uci.ics.asterix.external.dataset.adapter.AbstractDatasourceAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
+
+public class ExternalDataFilesMetadataProvider {
+	public static ArrayList<FileStatus> getHDFSFileStatus(AbstractDatasourceAdapter adapter) throws IOException
+	{
+		ArrayList<FileStatus> files = new ArrayList<FileStatus>();
+		//Configure hadoop connection
+		Configuration conf = HDFSAdapterFactory.configureHadoopConnection(adapter.getConfiguration());
+		FileSystem fs = FileSystem.get(conf);
+		//get the list of paths from the adapter
+		StringTokenizer tokenizer = new StringTokenizer(((String)adapter.getConfiguration().get(HDFSAdapter.KEY_PATH)),",");
+		Path inputPath = null;
+		FileStatus[] fileStatuses;
+		while(tokenizer.hasMoreTokens())
+		{
+			inputPath = new Path(tokenizer.nextToken().trim());
+			fileStatuses = fs.listStatus(inputPath);
+			for(int i=0; i < fileStatuses.length; i++)
+			{
+				files.add(fileStatuses[i]);
+			}
+		}
+		return files;
+	}
+}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
new file mode 100644
index 0000000..29fcfb1
--- /dev/null
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
@@ -0,0 +1,94 @@
+package edu.uci.ics.asterix.external.util;
+
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
+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.ITuplePartitionComputer;
+import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
+
+/* A class that is used to partition external data tuples when building an index over them
+ * the computer it returns, computes the HDFS block value before using the actual hash partitioning 
+ * function. this way we ensure that records within ranges of 64MB sizes are partitioned together to the same
+ * data node.
+ */
+
+public class ExternalIndexHashPartitionComputerFactory implements ITuplePartitionComputerFactory{
+	private static final long serialVersionUID = 1L;
+	private final int[] hashFields;
+	private final int bytesInHDFSBlock = 67108864;
+	private final IBinaryHashFunctionFactory[] hashFunctionFactories;
+	@SuppressWarnings("unchecked")
+	private final ISerializerDeserializer<AInt64> longSerde = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT64);
+
+	public ExternalIndexHashPartitionComputerFactory(int[] hashFields, IBinaryHashFunctionFactory[] hashFunctionFactories) {
+		this.hashFields = hashFields;
+		this.hashFunctionFactories = hashFunctionFactories;
+	}
+
+	@Override
+	public ITuplePartitionComputer createPartitioner() {
+		final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length];
+		for (int i = 0; i < hashFunctionFactories.length; ++i) {
+			hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction();
+		}
+		return new ITuplePartitionComputer() {
+			private ByteBuffer serializedLong = ByteBuffer.allocate(8);;
+			private AInt64 byteLocation;
+			private ByteBufferInputStream bbis = new ByteBufferInputStream();
+			private DataInputStream dis = new DataInputStream(bbis);
+			@Override
+			public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts){
+				if (nParts == 1) {
+					return 0;
+				}
+				int h = 0;
+				int startOffset = accessor.getTupleStartOffset(tIndex);
+				int slotLength = accessor.getFieldSlotsLength();
+				for (int j = 0; j < hashFields.length; ++j) {
+					int fIdx = hashFields[j];
+					IBinaryHashFunction hashFn = hashFunctions[j];
+					int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
+					int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
+					if(j == 1)
+					{
+						//reset the buffer
+						serializedLong.clear();
+						//read byte location
+						bbis.setByteBuffer(accessor.getBuffer() , accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength() + accessor.getFieldStartOffset(tIndex, hashFields[1]));
+						try {
+							byteLocation = ((AInt64) longSerde.deserialize(dis));
+							//compute the block number, serialize it into a new array and call the hash function
+							serializedLong.putLong(byteLocation.getLongValue()/bytesInHDFSBlock);
+							//call the hash function
+							int fh = hashFn
+									.hash(serializedLong.array(), 0,serializedLong.capacity());
+									h = h * 31 + fh;
+						} catch (HyracksDataException e) {
+							System.err.println("could not serialize the long byte position value!!!");
+							e.printStackTrace();
+						}
+					}
+					else
+					{
+						int fh = hashFn
+						.hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
+						h = h * 31 + fh;
+					}
+				}
+				if (h < 0) {
+					h = -(h + 1);
+				}
+				return h % nParts;
+			}
+		};
+	}
+}
\ No newline at end of file
diff --git a/asterix-installer/src/main/resources/conf/asterix-configuration.xml b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
index 6b3133c..084a3f8 100644
--- a/asterix-installer/src/main/resources/conf/asterix-configuration.xml
+++ b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
@@ -68,6 +68,14 @@
 		</description>
 	</property>
 
+    <property>
+		<name>storage.memorycomponent.numcomponents</name>
+		<value>2</value>
+		<description>The number of memory components to be used per lsm index.
+			(Default = 2)
+		</description>
+	</property>
+	
 	<property>
 		<name>storage.memorycomponent.globalbudget</name>
 		<value>536870192</value>
@@ -118,21 +126,6 @@
 	</property>
 
 	<property>
-		<name>txn.log.disksectorsize</name>
-		<value>4096</value>
-		<description>The size of a disk sector. (Default = "4096")
-		</description>
-	</property>
-
-	<property>
-		<name>txn.log.groupcommitinterval</name>
-		<value>40</value>
-		<description>The group commit wait time in milliseconds. (Default =
-			"40" // 40ms)
-		</description>
-	</property>
-
-	<property>
 		<name>txn.log.checkpoint.lsnthreshold</name>
 		<value>67108864</value>
 		<description>The size of the window that the maximum LSN is allowed to
diff --git a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
index dae1fb1..92d1c8e 100644
--- a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
+++ b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
@@ -19,215 +19,235 @@
 
 public class [LEXER_NAME] {
 
-    public static final int
-        TOKEN_EOF = 0, TOKEN_AUX_NOT_FOUND = 1 [TOKENS_CONSTANTS];
+	public static final int
+	TOKEN_EOF = 0, TOKEN_AUX_NOT_FOUND = 1 [TOKENS_CONSTANTS];
 
-    // Human representation of tokens. Useful for debug.
-    // Is possible to convert a TOKEN_CONSTANT in its image through
-    // [LEXER_NAME].tokenKindToString(TOKEN_CONSTANT); 
-    private static final String[] tokenImage = {
-            "<EOF>", "<AUX_NOT_FOUND>" [TOKENS_IMAGES]
-          };
-    
-    private static final char EOF_CHAR = 4;
-    protected java.io.Reader inputStream;
-    protected int column;
-    protected int line;
-    protected boolean prevCharIsCR;
-    protected boolean prevCharIsLF;
-    protected char[] buffer;
-    protected int bufsize;
-    protected int bufpos;
-    protected int tokenBegin;
-    protected int endOf_USED_Buffer;
-    protected int endOf_UNUSED_Buffer;
-    protected int maxUnusedBufferSize;
+	// Human representation of tokens. Useful for debug.
+	// Is possible to convert a TOKEN_CONSTANT in its image through
+	// [LEXER_NAME].tokenKindToString(TOKEN_CONSTANT); 
+	private static final String[] tokenImage = {
+		"<EOF>", "<AUX_NOT_FOUND>" [TOKENS_IMAGES]
+	};
 
-// ================================================================================
-//  Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
-// ================================================================================
+	private static final char EOF_CHAR = 4;
+	protected java.io.Reader inputStream;
+	protected int column;
+	protected int line;
+	protected boolean prevCharIsCR;
+	protected boolean prevCharIsLF;
+	protected char[] buffer;
+	protected int bufsize;
+	protected int bufpos;
+	protected int tokenBegin;
+	protected int endOf_USED_Buffer;
+	protected int endOf_UNUSED_Buffer;
+	protected int maxUnusedBufferSize;
 
-    [LEXER_AUXFUNCTIONS]
+	// ================================================================================
+	//  Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
+	// ================================================================================
 
-// ================================================================================
-//  Main method. Return a TOKEN_CONSTANT
-// ================================================================================            
-            
-    public int next() throws [LEXER_NAME]Exception, IOException{
-        char currentChar = buffer[bufpos];
-        while (currentChar == ' ' || currentChar=='\t' || currentChar == '\n' || currentChar=='\r')
-            currentChar = readNextChar(); 
-        tokenBegin = bufpos;
-        if (currentChar==EOF_CHAR) return TOKEN_EOF;
+	[LEXER_AUXFUNCTIONS]
 
-        [LEXER_LOGIC]
-    }
+			// ================================================================================
+			//  Main method. Return a TOKEN_CONSTANT
+			// ================================================================================            
 
-// ================================================================================
-//  Public interface
-// ================================================================================
-    
-    public [LEXER_NAME](java.io.Reader stream) throws IOException{
-        reInit(stream);
-    }
+			public int next() throws [LEXER_NAME]Exception, IOException{
+		if(bufpos < 0)
+			readNextChar();
+		char currentChar = buffer[bufpos];
+		while (currentChar == ' ' || currentChar=='\t' || currentChar == '\n' || currentChar=='\r')
+			currentChar = readNextChar(); 
+		tokenBegin = bufpos;
+		if (currentChar==EOF_CHAR) return TOKEN_EOF;
 
-    public void reInit(java.io.Reader stream) throws IOException{
-        done();
-        inputStream    = stream;
-        bufsize        = 4096;
-        line           = 1;
-        column         = 0;
-        bufpos         = -1;
-        endOf_UNUSED_Buffer = bufsize;
-        endOf_USED_Buffer = 0;
-        prevCharIsCR   = false;
-        prevCharIsLF   = false;
-        buffer         = new char[bufsize];
-        tokenBegin     = -1;
-        maxUnusedBufferSize = 4096/2;
-        readNextChar();
-    }
+		[LEXER_LOGIC]
+	}
 
-    public String getLastTokenImage() {
-        if (bufpos >= tokenBegin)
-            return new String(buffer, tokenBegin, bufpos - tokenBegin);
-          else
-            return new String(buffer, tokenBegin, bufsize - tokenBegin) +
-                                  new String(buffer, 0, bufpos);
-    }
-    
-    public static String tokenKindToString(int token) {
-        return tokenImage[token]; 
-    }
+	//used when done with stream, must be called exiplicitly now.
+	public void close()throws IOException
+	{
+		inputStream.close();
+	}
 
-    public void done(){
-        buffer = null;
-    }
+	//used before processing a new patch in the inputStream
+	public void reset(){
+		line           = 1;
+		column         = 0;
+		bufpos         = -1;
+		endOf_UNUSED_Buffer = bufsize;
+		endOf_USED_Buffer = 0;
+		prevCharIsCR   = false;
+		prevCharIsLF   = false;
+		tokenBegin     = -1;
+	}
 
-// ================================================================================
-//  Parse error management
-// ================================================================================    
-    
-    protected int parseError(String reason) throws [LEXER_NAME]Exception {
-        StringBuilder message = new StringBuilder();
-        message.append(reason).append("\n");
-        message.append("Line: ").append(line).append("\n");
-        message.append("Row: ").append(column).append("\n");
-        throw new [LEXER_NAME]Exception(message.toString());
-    }
+	// ================================================================================
+	//  Public interface
+	// ================================================================================
 
-    protected int parseError(int ... tokens) throws [LEXER_NAME]Exception {
-        StringBuilder message = new StringBuilder();
-        message.append("Error while parsing. ");
-        message.append(" Line: ").append(line);
-        message.append(" Row: ").append(column);
-        message.append(" Expecting:");
-        for (int tokenId : tokens){
-            message.append(" ").append([LEXER_NAME].tokenKindToString(tokenId));
-        }
-        throw new [LEXER_NAME]Exception(message.toString());
-    }
-    
-    protected void updateLineColumn(char c){
-        column++;
-    
-        if (prevCharIsLF)
-        {
-            prevCharIsLF = false;
-            line += (column = 1);
-        }
-        else if (prevCharIsCR)
-        {
-            prevCharIsCR = false;
-            if (c == '\n')
-            {
-                prevCharIsLF = true;
-            }
-            else
-            {
-                line += (column = 1);
-            }
-        }
-        
-        if (c=='\r') {
-            prevCharIsCR = true;
-        } else if(c == '\n') {
-            prevCharIsLF = true;
-        }
-    }
-    
-// ================================================================================
-//  Read data, buffer management. It uses a circular (and expandable) buffer
-// ================================================================================    
+	public [LEXER_NAME](java.io.Reader stream) throws IOException{
+		reInit(stream);
+	}
 
-    protected char readNextChar() throws IOException {
-        if (++bufpos >= endOf_USED_Buffer)
-            fillBuff();
-        char c = buffer[bufpos];
-        updateLineColumn(c);
-        return c;
-    }
+	public void reInit(java.io.Reader stream) throws IOException{
+		done();
+		inputStream    = stream;
+		bufsize        = 4096;
+		line           = 1;
+		column         = 0;
+		bufpos         = -1;
+		endOf_UNUSED_Buffer = bufsize;
+		endOf_USED_Buffer = 0;
+		prevCharIsCR   = false;
+		prevCharIsLF   = false;
+		buffer         = new char[bufsize];
+		tokenBegin     = -1;
+		maxUnusedBufferSize = 4096/2;
+	}
 
-    protected boolean fillBuff() throws IOException {
-        if (endOf_UNUSED_Buffer == endOf_USED_Buffer) // If no more unused buffer space 
-        {
-          if (endOf_UNUSED_Buffer == bufsize)         // -- If the previous unused space was
-          {                                           // -- at the end of the buffer
-            if (tokenBegin > maxUnusedBufferSize)     // -- -- If the first N bytes before
-            {                                         //       the current token are enough
-              bufpos = endOf_USED_Buffer = 0;         // -- -- -- setup buffer to use that fragment 
-              endOf_UNUSED_Buffer = tokenBegin;
-            }
-            else if (tokenBegin < 0)                  // -- -- If no token yet
-              bufpos = endOf_USED_Buffer = 0;         // -- -- -- reuse the whole buffer
-            else
-              ExpandBuff(false);                      // -- -- Otherwise expand buffer after its end
-          }
-          else if (endOf_UNUSED_Buffer > tokenBegin)  // If the endOf_UNUSED_Buffer is after the token
-            endOf_UNUSED_Buffer = bufsize;            // -- set endOf_UNUSED_Buffer to the end of the buffer
-          else if ((tokenBegin - endOf_UNUSED_Buffer) < maxUnusedBufferSize)
-          {                                           // If between endOf_UNUSED_Buffer and the token
-            ExpandBuff(true);                         // there is NOT enough space expand the buffer                          
-          }                                           // reorganizing it
-          else 
-            endOf_UNUSED_Buffer = tokenBegin;         // Otherwise there is enough space at the start
-        }                                             // so we set the buffer to use that fragment
-        int i;
-        if ((i = inputStream.read(buffer, endOf_USED_Buffer, endOf_UNUSED_Buffer - endOf_USED_Buffer)) == -1)
-        {
-            inputStream.close();
-            buffer[endOf_USED_Buffer]=(char)EOF_CHAR;
-            endOf_USED_Buffer++;
-            return false;
-        }
-            else
-                endOf_USED_Buffer += i;
-        return true;
-    }
+	public String getLastTokenImage() {
+		if (bufpos >= tokenBegin)
+			return new String(buffer, tokenBegin, bufpos - tokenBegin);
+		else
+			return new String(buffer, tokenBegin, bufsize - tokenBegin) +
+					new String(buffer, 0, bufpos);
+	}
+
+	public static String tokenKindToString(int token) {
+		return tokenImage[token]; 
+	}
+
+	public void done(){
+		buffer = null;
+	}
+
+	// ================================================================================
+			//  Parse error management
+			// ================================================================================    
+
+	protected int parseError(String reason) throws [LEXER_NAME]Exception {
+		StringBuilder message = new StringBuilder();
+		message.append(reason).append("\n");
+		message.append("Line: ").append(line).append("\n");
+		message.append("Row: ").append(column).append("\n");
+		throw new [LEXER_NAME]Exception(message.toString());
+	}
+
+	protected int parseError(int ... tokens) throws [LEXER_NAME]Exception {
+		StringBuilder message = new StringBuilder();
+		message.append("Error while parsing. ");
+		message.append(" Line: ").append(line);
+		message.append(" Row: ").append(column);
+		message.append(" Expecting:");
+		for (int tokenId : tokens){
+			message.append(" ").append([LEXER_NAME].tokenKindToString(tokenId));
+		}
+		throw new [LEXER_NAME]Exception(message.toString());
+	}
+
+	protected void updateLineColumn(char c){
+		column++;
+
+		if (prevCharIsLF)
+		{
+			prevCharIsLF = false;
+			line += (column = 1);
+		}
+		else if (prevCharIsCR)
+		{
+			prevCharIsCR = false;
+			if (c == '\n')
+			{
+				prevCharIsLF = true;
+			}
+			else
+			{
+				line += (column = 1);
+			}
+		}
+
+		if (c=='\r') {
+			prevCharIsCR = true;
+		} else if(c == '\n') {
+			prevCharIsLF = true;
+		}
+	}
+
+	// ================================================================================
+	//  Read data, buffer management. It uses a circular (and expandable) buffer
+	// ================================================================================    
+
+	protected char readNextChar() throws IOException {
+		if (++bufpos >= endOf_USED_Buffer)
+			fillBuff();
+		char c = buffer[bufpos];
+		updateLineColumn(c);
+		return c;
+	}
+
+	protected boolean fillBuff() throws IOException {
+		if (endOf_UNUSED_Buffer == endOf_USED_Buffer) // If no more unused buffer space 
+		{
+			if (endOf_UNUSED_Buffer == bufsize)         // -- If the previous unused space was
+			{                                           // -- at the end of the buffer
+				if (tokenBegin > maxUnusedBufferSize)     // -- -- If the first N bytes before
+				{                                         //       the current token are enough
+					bufpos = endOf_USED_Buffer = 0;         // -- -- -- setup buffer to use that fragment 
+					endOf_UNUSED_Buffer = tokenBegin;
+				}
+				else if (tokenBegin < 0)                  // -- -- If no token yet
+					bufpos = endOf_USED_Buffer = 0;         // -- -- -- reuse the whole buffer
+				else
+					ExpandBuff(false);                      // -- -- Otherwise expand buffer after its end
+			}
+			else if (endOf_UNUSED_Buffer > tokenBegin)  // If the endOf_UNUSED_Buffer is after the token
+				endOf_UNUSED_Buffer = bufsize;            // -- set endOf_UNUSED_Buffer to the end of the buffer
+			else if ((tokenBegin - endOf_UNUSED_Buffer) < maxUnusedBufferSize)
+			{                                           // If between endOf_UNUSED_Buffer and the token
+				ExpandBuff(true);                         // there is NOT enough space expand the buffer                          
+			}                                           // reorganizing it
+			else 
+				endOf_UNUSED_Buffer = tokenBegin;         // Otherwise there is enough space at the start
+		}                                             // so we set the buffer to use that fragment
+		int i;
+		if ((i = inputStream.read(buffer, endOf_USED_Buffer, endOf_UNUSED_Buffer - endOf_USED_Buffer)) == -1)
+		{
+			//moved outside
+			//inputStream.close();
+			buffer[endOf_USED_Buffer]=(char)EOF_CHAR;
+			endOf_USED_Buffer++;
+			return false;
+		}
+		else
+			endOf_USED_Buffer += i;
+		return true;
+	}
 
 
-    protected void ExpandBuff(boolean wrapAround)
-    {
-      char[] newbuffer = new char[bufsize + maxUnusedBufferSize];
+	protected void ExpandBuff(boolean wrapAround)
+	{
+		char[] newbuffer = new char[bufsize + maxUnusedBufferSize];
 
-      try {
-        if (wrapAround) {
-          System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
-          System.arraycopy(buffer, 0, newbuffer, bufsize - tokenBegin, bufpos);
-          buffer = newbuffer;
-          endOf_USED_Buffer = (bufpos += (bufsize - tokenBegin));
-        }
-        else {
-          System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
-          buffer = newbuffer;
-          endOf_USED_Buffer = (bufpos -= tokenBegin);
-        }
-      } catch (Throwable t) {
-          throw new Error(t.getMessage());
-      }
+		try {
+			if (wrapAround) {
+				System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+				System.arraycopy(buffer, 0, newbuffer, bufsize - tokenBegin, bufpos);
+				buffer = newbuffer;
+				endOf_USED_Buffer = (bufpos += (bufsize - tokenBegin));
+			}
+			else {
+				System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+				buffer = newbuffer;
+				endOf_USED_Buffer = (bufpos -= tokenBegin);
+			}
+		} catch (Throwable t) {
+			throw new Error(t.getMessage());
+		}
 
-      bufsize += maxUnusedBufferSize;
-      endOf_UNUSED_Buffer = bufsize;
-      tokenBegin = 0;
-    }    
+		bufsize += maxUnusedBufferSize;
+		endOf_UNUSED_Buffer = bufsize;
+		tokenBegin = 0;
+	}    
 }
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 8f0eedb..2de5d78 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
@@ -31,6 +31,7 @@
 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.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
@@ -268,6 +269,39 @@
         }
         return dataset;
     }
+    
+    @Override
+   	public List<ExternalFile> getDatasetExternalFiles(
+   			MetadataTransactionContext mdTxnCtx, Dataset dataset)
+   			throws MetadataException {
+       	List<ExternalFile> externalFiles;
+           try {
+           	externalFiles = metadataNode.getExternalDatasetFiles(mdTxnCtx.getJobId(), dataset);
+           } catch (RemoteException e) {
+               throw new MetadataException(e);
+           }
+           return externalFiles;
+   	}
+    
+    @Override
+	public void addExternalFile(MetadataTransactionContext mdTxnCtx,
+			ExternalFile externalFile) throws MetadataException {
+    	try {
+            metadataNode.addExternalDatasetFile(mdTxnCtx.getJobId(), externalFile);
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+	}
+    
+    @Override
+	public void dropExternalFile(MetadataTransactionContext mdTxnCtx,
+			ExternalFile externalFile) throws MetadataException {
+        try {
+            metadataNode.dropExternalFile(mdTxnCtx.getJobId(), externalFile.getDataverseName(), externalFile.getDatasetName(), externalFile.getFileNumber());
+        } catch (RemoteException e) {
+            throw new MetadataException(e);
+        }
+	}
 
     @Override
     public List<Index> getDatasetIndexes(MetadataTransactionContext ctx, String dataverseName, String datasetName)
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 b980337..81ce4f8 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
@@ -27,9 +27,8 @@
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -42,6 +41,8 @@
 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.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -51,6 +52,7 @@
 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.ExternalFileTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.NodeGroupTupleTranslator;
@@ -60,6 +62,7 @@
 import edu.uci.ics.asterix.metadata.valueextractors.MetadataEntityValueExtractor;
 import edu.uci.ics.asterix.metadata.valueextractors.NestedDatatypeNameValueExtractor;
 import edu.uci.ics.asterix.metadata.valueextractors.TupleCopyValueExtractor;
+import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AMutableString;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -87,6 +90,7 @@
 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;
+import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 
 public class MetadataNode implements IMetadataNode {
     private static final long serialVersionUID = 1L;
@@ -109,7 +113,8 @@
 
     @Override
     public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
-        transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+        txnCtx.setMetadataTransaction(true);
     }
 
     @Override
@@ -168,48 +173,56 @@
                 Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
                         dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
 
-                addIndex(jobId, primaryIndex);
-                ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
-                        dataset.getDatasetName());
-                insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
-            }
-            // Add entry in datatype secondary index.
-            ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
-                    dataset.getDatasetName());
-            insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
-        } catch (TreeIndexDuplicateKeyException e) {
-            throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
-                    + " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
-        } catch (Exception e) {
-            throw new MetadataException(e);
-        }
-    }
+				addIndex(jobId, primaryIndex);
+				ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
+						dataset.getDatasetName());
+				insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
+			}
+			else
+			{
+				//added for external data
+				ExternalDatasetDetails id = (ExternalDatasetDetails) dataset.getDatasetDetails();
+				ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
+						dataset.getDatasetName());
+				insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
+			}
+			// Add entry in datatype secondary index.
+			ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
+					dataset.getDatasetName());
+			insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+		} catch (TreeIndexDuplicateKeyException e) {
+			throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
+					+ " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+	}
 
-    @Override
-    public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
-        try {
-            IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
-            ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
-        } catch (TreeIndexDuplicateKeyException e) {
-            throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
-        } catch (Exception e) {
-            throw new MetadataException(e);
-        }
-    }
+	@Override
+	public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
+		try {
+			IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
+			ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
+			insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+		} catch (TreeIndexDuplicateKeyException e) {
+			throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+	}
 
-    @Override
-    public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
-        try {
-            NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
-            ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
-            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
-        } catch (TreeIndexDuplicateKeyException e) {
-            throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
-        } catch (Exception e) {
-            throw new MetadataException(e);
-        }
-    }
+	@Override
+	public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
+		try {
+			NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
+			ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
+			insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
+		} catch (TreeIndexDuplicateKeyException e) {
+			throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+	}
 
     @Override
     public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
@@ -270,14 +283,15 @@
         IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
                 lsmIndex, IndexOperation.INSERT);
 
-        IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+        ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
         ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
-        txnCtx.setTransactionType(TransactionType.READ_WRITE);
-        txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+        txnCtx.setWriteTxn(true);
+        txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+                metadataIndex.isPrimaryIndex());
 
         // TODO: fix exceptions once new BTree exception model is in hyracks.
-        indexAccessor.insert(tuple);
+        indexAccessor.forceInsert(tuple);
 
         indexLifecycleManager.close(resourceID);
     }
@@ -382,47 +396,60 @@
                 // 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.
-                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.
-            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 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.
+				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.
+				}
+			}
+			else
+			{
+				ExternalDatasetDetails id = (ExternalDatasetDetails) 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.
+				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.
+			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(jobId, dataverseName, datasetName);
-                if (datasetIndexes != null) {
-                    for (Index index : datasetIndexes) {
-                        dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
-                    }
-                }
-            }
-        } catch (Exception e) {
-            throw new MetadataException(e);
-        }
-    }
+			// Delete entry(s) from the 'indexes' dataset.
+			List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
+			if (datasetIndexes != null) {
+				for (Index index : datasetIndexes) {
+					dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
+				}
+			}
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+	}
 
     @Override
     public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
@@ -577,13 +604,14 @@
         // prepare a Callback for logging
         IModificationOperationCallback modCallback = createIndexModificationCallback(jobId, resourceID, metadataIndex,
                 lsmIndex, IndexOperation.DELETE);
-        IIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
+        ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
         ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
-        txnCtx.setTransactionType(TransactionType.READ_WRITE);
-        txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+        txnCtx.setWriteTxn(true);
+        txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+                metadataIndex.isPrimaryIndex());
 
-        indexAccessor.delete(tuple);
+        indexAccessor.forceDelete(tuple);
         indexLifecycleManager.close(resourceID);
     }
 
@@ -1132,8 +1160,116 @@
         }
     }
 
-    @Override
-    public int getMostRecentDatasetId() throws MetadataException, RemoteException {
-        return DatasetIdFactory.getMostRecentDatasetId();
+	@Override
+	public List<ExternalFile> getExternalDatasetFiles(JobId jobId,
+			Dataset dataset) throws MetadataException, RemoteException {
+		try {
+			ITupleReference searchKey = createTuple(dataset.getDataverseName(),dataset.getDatasetName());
+			ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
+			IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
+					tupleReaderWriter);
+			List<ExternalFile> results = new ArrayList<ExternalFile>();
+			searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
+			return results;
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+	}
+
+	@SuppressWarnings("unchecked")
+	public ITupleReference createExternalFileSearchTuple(String dataverseName, String datasetName, int fileNumber) throws HyracksDataException {
+		ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
+				.getSerializerDeserializer(BuiltinType.ASTRING);
+		ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+				.getSerializerDeserializer(BuiltinType.AINT32);
+
+		AMutableString aString = new AMutableString("");
+		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(3);
+
+		//dataverse field
+		aString.setValue(dataverseName);
+		stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+		tupleBuilder.addFieldEndOffset();
+
+		//dataset field
+		aString.setValue(datasetName);
+		stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+		tupleBuilder.addFieldEndOffset();
+
+		//file number field
+		intSerde.serialize(new AInt32(fileNumber), tupleBuilder.getDataOutput());
+		tupleBuilder.addFieldEndOffset();
+
+		ArrayTupleReference tuple = new ArrayTupleReference();
+		tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+		return tuple;
+		}
+
+	public ExternalFile getExternalDatasetFile(JobId jobId,String dataverseName, String datasetName,
+			int fileNumber) throws MetadataException, RemoteException {
+		try {
+			//create the search key
+			ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
+			ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
+			IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
+					tupleReaderWriter);
+			List<ExternalFile> results = new ArrayList<ExternalFile>();
+			searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
+			return results.get(0);
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+	}
+	
+	@Override
+	public void dropExternalFile(JobId jobId, String dataverseName,
+			String datasetName, int fileNumber) throws MetadataException,
+			RemoteException {
+		ExternalFile externalFile;
+		try {
+			externalFile = getExternalDatasetFile(jobId, dataverseName, datasetName,fileNumber);
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+		if (externalFile == null) {
+			throw new MetadataException("Cannot drop external file because it doesn't exist.");
+		}
+		try {
+			// Delete entry from the 'ExternalFile' dataset.
+			ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
+			// Searches the index for the tuple to be deleted. Acquires an S
+			// lock on the 'ExternalFile' dataset.
+			ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
+					MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey);
+			deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, datasetTuple);
+
+		} catch (TreeIndexException e) {
+			throw new MetadataException("Couldn't drop externalFile.", e);
+		} catch (Exception e) {
+			throw new MetadataException(e);
+		}
+	}
+	
+	@Override
+    public void addExternalDatasetFile(JobId jobId, ExternalFile externalFile)
+                    throws MetadataException, RemoteException {
+            try {
+                    // Insert into the 'externalFiles' dataset.
+                    ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(true);
+                    ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
+                    insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
+            } catch (TreeIndexDuplicateKeyException e) {
+                    throw new MetadataException("An externalFile with this number " + externalFile.getFileNumber()
+                                    + " already exists in dataset '" + externalFile.getDatasetName() + "' in dataverse '"+externalFile.getDataverseName()+"'.", e);
+            } catch (Exception e) {
+                    throw new MetadataException(e);
+            }
     }
+
+
+	@Override
+	public int getMostRecentDatasetId() throws MetadataException, RemoteException {
+		return DatasetIdFactory.getMostRecentDatasetId();
+	}
 }
+
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 22c5e46..53f72dd 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
@@ -26,6 +26,7 @@
 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.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
@@ -439,6 +440,36 @@
      */
     public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
             throws MetadataException;
+    
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param externalFile
+     *            An instance of type ExternalFile that represents the external file being
+     *            added
+     * @throws MetadataException
+     */
+    public void addExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
+    
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param dataset
+     *            An instance of type Dataset that represents the "external" dataset 
+     * @return A list of external files belonging to the dataset
+     * @throws MetadataException
+     */
+    public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException;
+
+    /**
+     * @param mdTxnCtx
+     *            MetadataTransactionContext of an active metadata transaction.
+     * @param externalFile
+     *            An instance of type ExternalFile that represents the external file being
+     *            dropped
+     * @throws MetadataException
+     */
+    public void dropExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
 
     public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
     
@@ -453,4 +484,5 @@
     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 d1e63e1..206ef8a 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
@@ -28,6 +28,7 @@
 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.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
@@ -471,6 +472,45 @@
      * @throws RemoteException
      */
     public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException;
+    
+    /**
+     * @param jobId
+     *            A globally unique id for an active metadata transaction.
+     * @param externalFile
+     * 			  An object representing the external file entity
+     * @throws MetadataException
+     *             for example, if the file already exists.
+     * @throws RemoteException
+     */
+	public void addExternalDatasetFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException;
+	
+	/**
+     * @param jobId
+     *            A globally unique id for an active metadata transaction.
+     * @param dataset
+     *            A dataset the files belongs to.
+     * @throws MetadataException
+     * @throws RemoteException
+     */
+	public List<ExternalFile> getExternalDatasetFiles(JobId jobId, Dataset dataset
+			) throws MetadataException, RemoteException;
+	
+	/**
+     * Deletes an externalFile , acquiring local locks on behalf of the given
+     * transaction id.
+     * 
+     * @param jobId
+     *            A globally unique id for an active metadata transaction.
+     * @param dataverseName
+     *            dataverse asociated with the external dataset that owns the file to be deleted.
+     * @param datasetName
+     *            Name of dataset owning the file to be deleted.
+     * @param fileNumber
+     * 			  the id number for the file to be deleted
+     * @throws RemoteException
+     */
+    public void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber) throws MetadataException,
+            RemoteException;
 
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
     
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 aa976f8..ed89cfc 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
@@ -33,10 +33,9 @@
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
 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;
@@ -61,7 +60,6 @@
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 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.service.logging.IndexResourceManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
@@ -94,6 +92,7 @@
  */
 public class MetadataBootstrap {
     private static final Logger LOGGER = Logger.getLogger(MetadataBootstrap.class.getName());
+    public static final boolean IS_DEBUG_MODE = false;//true
 
     private static IAsterixAppRuntimeContext runtimeContext;
 
@@ -118,7 +117,7 @@
                 MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
                 MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
                 MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET };
+                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
         secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
@@ -138,15 +137,6 @@
         MetadataSecondaryIndexes.init();
         initLocalIndexArrays();
 
-        TransactionalResourceManagerRepository resourceRepository = runtimeContext.getTransactionSubsystem()
-                .getTransactionalResourceRepository();
-        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()));
-
         AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
         metadataNodeName = metadataProperties.getMetadataNodeName();
         metadataStore = metadataProperties.getMetadataStore();
@@ -201,6 +191,9 @@
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
         } catch (Exception e) {
             try {
+                if (IS_DEBUG_MODE) {
+                    e.printStackTrace();
+                }
                 MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
             } catch (Exception e2) {
                 e.addSuppressed(e2);
@@ -333,18 +326,19 @@
                 + IndexFileNameUtil.prepareFileName(metadataStore + File.separator + index.getFileNameRelativePath(),
                         runtimeContext.getMetaDataIODeviceId());
         FileReference file = new FileReference(new File(filePath));
-        IVirtualBufferCache virtualBufferCache = runtimeContext.getVirtualBufferCache(index.getDatasetId().getId());
+        List<IVirtualBufferCache> virtualBufferCaches = runtimeContext.getVirtualBufferCaches(index.getDatasetId()
+                .getId());
         ITypeTraits[] typeTraits = index.getTypeTraits();
         IBinaryComparatorFactory[] comparatorFactories = index.getKeyBinaryComparatorFactory();
         int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
         LSMBTree lsmBtree = null;
         long resourceID = -1;
-        AsterixRuntimeComponentsProvider rtcProvider = index.isPrimaryIndex() ? AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER
-                : AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER;
+        AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
         ILSMOperationTracker opTracker = index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index
-                .getDatasetId().getId()) : new BaseOperationTracker(LSMBTreeIOOperationCallbackFactory.INSTANCE);
+                .getDatasetId().getId()) : new BaseOperationTracker((DatasetLifecycleManager) indexLifecycleManager,
+                LSMBTreeIOOperationCallbackFactory.INSTANCE, index.getDatasetId().getId());
         if (create) {
-            lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fileMapProvider, typeTraits,
+            lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, runtimeContext.getBloomFilterFalsePositiveRate(),
                     runtimeContext.getLSMMergePolicy(), opTracker, runtimeContext.getLSMIOScheduler(), rtcProvider);
             lsmBtree.create();
@@ -361,11 +355,11 @@
             resourceID = localResourceRepository.getResourceByName(file.getFile().getPath()).getResourceId();
             lsmBtree = (LSMBTree) indexLifecycleManager.getIndex(resourceID);
             if (lsmBtree == null) {
-                lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, bufferCache, fileMapProvider,
+                lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider,
                         typeTraits, comparatorFactories, bloomFilterKeyFields,
                         runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
                         opTracker, runtimeContext.getLSMIOScheduler(),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER);
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
                 indexLifecycleManager.register(resourceID, lsmBtree);
             }
         }
@@ -442,6 +436,9 @@
             }
         } catch (Exception e) {
             try {
+                if (IS_DEBUG_MODE) {
+                    e.printStackTrace();
+                }
                 MetadataManager.INSTANCE.abortTransaction(mdTxnCtx);
             } catch (Exception e2) {
                 e.addSuppressed(e2);
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 8bdd92b..100ec40 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
@@ -41,9 +41,11 @@
     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 EXTERNAL_FILE_DATASET_ID = 9;
     public static final int FIRST_AVAILABLE_USER_DATASET_ID = 100;
 
     public static IMetadataIndex DATASOURCE_ADAPTER_DATASET;
+    public static IMetadataIndex EXTERNAL_FILE_DATASET;
 
     /**
      * Create all metadata primary index descriptors. MetadataRecordTypes must
@@ -92,5 +94,11 @@
                 BuiltinType.ASTRING, BuiltinType.ASTRING }, new String[] { "DataverseName", "Name" }, 0,
                 MetadataRecordTypes.DATASOURCE_ADAPTER_RECORDTYPE, DATASOURCE_ADAPTER_DATASET_ID, true, new int[] { 0,
                         1 });
+        
+        EXTERNAL_FILE_DATASET = new MetadataIndex("ExternalFile", null, 4, new IAType[] {
+                BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32 }, new String[] { "DataverseName", "DatasetName", 
+        		"FileNumber"}, 0,
+                MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE, EXTERNAL_FILE_DATASET_ID, true, new int[] { 0,
+                        1, 2 });
     }
 }
\ 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 50681ee..11f9c91 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
@@ -47,6 +47,7 @@
     public static ARecordType NODEGROUP_RECORDTYPE;
     public static ARecordType FUNCTION_RECORDTYPE;
     public static ARecordType DATASOURCE_ADAPTER_RECORDTYPE;
+    public static ARecordType EXTERNAL_FILE_RECORDTYPE;
 
     /**
      * Create all metadata record types.
@@ -76,6 +77,7 @@
             NODEGROUP_RECORDTYPE = createNodeGroupRecordType();
             FUNCTION_RECORDTYPE = createFunctionRecordType();
             DATASOURCE_ADAPTER_RECORDTYPE = createDatasourceAdapterRecordType();
+            EXTERNAL_FILE_RECORDTYPE = createExternalFileRecordType();
         } catch (AsterixException e) {
             throw new MetadataException(e);
         }
@@ -125,12 +127,13 @@
     // external details.
     public static final int EXTERNAL_DETAILS_ARECORD_DATASOURCE_ADAPTER_FIELD_INDEX = 0;
     public static final int EXTERNAL_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX = 1;
-
+    public static final int EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX = 2;
+    
     private static final ARecordType createExternalDetailsRecordType() throws AsterixException {
 
         AOrderedListType orderedPropertyListType = new AOrderedListType(DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE, null);
-        String[] fieldNames = { "DatasourceAdapter", "Properties" };
-        IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType };
+        String[] fieldNames = { "DatasourceAdapter", "Properties", "GroupName" };
+        IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType, BuiltinType.ASTRING };
         return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
@@ -356,5 +359,19 @@
                 BuiltinType.ASTRING };
         return new ARecordType("DatasourceAdapterRecordType", fieldNames, fieldTypes, true);
     }
+    
+    public static final int EXTERNAL_FILE_ARECORD_DATAVERSENAME_FIELD_INDEX = 0;
+    public static final int EXTERNAL_FILE_ARECORD_DATASET_NAME_FIELD_INDEX = 1;
+    public static final int EXTERNAL_FILE_ARECORD_FILE_NUMBER_FIELD_INDEX = 2;
+    public static final int EXTERNAL_FILE_ARECORD_FILE_NAME_FIELD_INDEX = 3;
+    public static final int EXTERNAL_FILE_ARECORD_FILE_SIZE_FIELD_INDEX = 4;
+    public static final int EXTERNAL_FILE_ARECORD_FILE_MOD_DATE_FIELD_INDEX = 5;
+    
+    private static ARecordType createExternalFileRecordType() throws AsterixException {
+    	String[] fieldNames = { "DataverseName", "DatasetName", "FileNumber", "FileName", "FileSize", "FileModDate"};
+        IAType[] fieldTypes = { BuiltinType.ASTRING, BuiltinType.ASTRING, BuiltinType.AINT32, BuiltinType.ASTRING, BuiltinType.AINT64,
+        		BuiltinType.ADATETIME};
+        return new ARecordType("ExternalFileRecordType", fieldNames, fieldTypes, true);
+    }
 
 }
\ 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 1ade7ef..078b340d 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
@@ -18,11 +18,14 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.logging.Logger;
 
+import org.apache.hadoop.fs.FileStatus;
+
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
@@ -33,21 +36,28 @@
 import edu.uci.ics.asterix.common.dataflow.AsterixLSMInvertedIndexInsertDeleteOperatorDescriptor;
 import edu.uci.ics.asterix.common.dataflow.AsterixLSMTreeInsertDeleteOperatorDescriptor;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
 import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
+import edu.uci.ics.asterix.external.data.operator.ExternalDataAccessByRIDOperatorDescriptor;
+import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
 import edu.uci.ics.asterix.external.data.operator.ExternalDataScanOperatorDescriptor;
 import edu.uci.ics.asterix.external.data.operator.FeedIntakeOperatorDescriptor;
 import edu.uci.ics.asterix.external.data.operator.FeedMessageOperatorDescriptor;
+import edu.uci.ics.asterix.external.dataset.adapter.AbstractDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
 import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
 import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
+import edu.uci.ics.asterix.external.util.ExternalDataFilesMetadataProvider;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -61,6 +71,7 @@
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -81,6 +92,7 @@
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
 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.SecondaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexSearchOperationCallbackFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
@@ -156,6 +168,7 @@
     private boolean asyncResults;
     private ResultSetId resultSetId;
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
+    private static boolean optimizeExternalIndexes = false;
 
     private final Dataverse defaultDataverse;
     private JobId jobId;
@@ -165,22 +178,6 @@
     private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
     private static Scheduler hdfsScheduler;
 
-    public String getPropertyValue(String propertyName) {
-        return config.get(propertyName);
-    }
-
-    public void setConfig(Map<String, String> config) {
-        this.config = config;
-    }
-
-    public Map<String, String[]> getAllStores() {
-        return stores;
-    }
-
-    public Map<String, String> getConfig() {
-        return config;
-    }
-
     public AqlMetadataProvider(Dataverse defaultDataverse) {
         this.defaultDataverse = defaultDataverse;
         this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
@@ -260,6 +257,30 @@
     public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
         return resultSerializerFactoryProvider;
     }
+    
+    public String getPropertyValue(String propertyName) {
+        return config.get(propertyName);
+    }
+
+    public void setConfig(Map<String, String> config) {
+        this.config = config;
+    }
+
+    public Map<String, String[]> getAllStores() {
+        return stores;
+    }
+
+    public Map<String, String> getConfig() {
+        return config;
+    }
+
+    public static boolean isOptimizeExternalIndexes() {
+		return optimizeExternalIndexes;
+	}
+    
+    public static void setOptimizeExternalIndexes(boolean optimizeExternalIndexes) {
+		AqlMetadataProvider.optimizeExternalIndexes = optimizeExternalIndexes;
+	}
 
     @Override
     public AqlDataSource findDataSource(AqlSourceId id) throws AlgebricksException {
@@ -398,6 +419,215 @@
 
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
     }
+    
+    @SuppressWarnings("rawtypes")
+	public Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataIndexingRuntime(
+			JobSpecification jobSpec, IAType itemType, Dataset dataset, IDataFormat format)
+					throws AlgebricksException {
+		IGenericDatasetAdapterFactory adapterFactory;
+		IDatasourceAdapter adapter;
+		String adapterName;
+		DatasourceAdapter adapterEntity;
+		String adapterFactoryClassname;
+		ExternalDatasetDetails datasetDetails = null;
+		try {
+			datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+			adapterName = datasetDetails.getAdapter();
+			adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+					adapterName);
+			if (adapterEntity != null) {
+				adapterFactoryClassname = adapterEntity.getClassname();
+				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+			} else {
+				adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
+				if (adapterFactoryClassname == null) {
+					throw new AlgebricksException(" Unknown adapter :" + adapterName);
+				}
+				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+			}
+
+			adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createIndexingAdapter(
+					wrapProperties(datasetDetails.getProperties()), itemType, null);
+		} catch (AlgebricksException ae) {
+			throw ae;
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw new AlgebricksException("Unable to create adapter " + e);
+		}
+		if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
+				IDatasourceAdapter.AdapterType.READ_WRITE))) {
+			throw new AlgebricksException("external dataset adapter does not support read operation");
+		}
+		ARecordType rt = (ARecordType) itemType;
+		ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
+		RecordDescriptor indexerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
+		ExternalDataIndexingOperatorDescriptor dataIndexScanner = null;
+		List<ExternalFile> files = null;
+		HashMap<String, Integer> filesNumbers = null;
+		if(optimizeExternalIndexes)
+		{
+			try {
+				files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
+			} catch (MetadataException e) {
+				e.printStackTrace();
+				throw new AlgebricksException("Unable to get list of external files from metadata " + e);
+			}
+			
+			filesNumbers = new HashMap<String,Integer>();
+			for(int i=0; i< files.size(); i++)
+			{
+				filesNumbers.put(files.get(i).getFileName(), files.get(i).getFileNumber());
+			}
+			
+			dataIndexScanner = new ExternalDataIndexingOperatorDescriptor(jobSpec,
+					wrapPropertiesEmpty(datasetDetails.getProperties()), rt, indexerDesc, adapterFactory,filesNumbers);
+		}
+		else
+		{
+		dataIndexScanner = new ExternalDataIndexingOperatorDescriptor(jobSpec,
+				wrapPropertiesEmpty(datasetDetails.getProperties()), rt, indexerDesc, adapterFactory,filesNumbers);
+		}
+		AlgebricksPartitionConstraint constraint;
+		try {
+			constraint = adapter.getPartitionConstraint();
+		} catch (Exception e) {
+			throw new AlgebricksException(e);
+		}
+		return new Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint>(dataIndexScanner, constraint);
+	}
+    
+    public ArrayList<ExternalFile> getExternalDatasetFiles(Dataset dataset) throws AlgebricksException
+	{
+    	ArrayList<ExternalFile> files = new ArrayList<ExternalFile>();
+		if(dataset.getDatasetType() != DatasetType.EXTERNAL)
+		{
+			throw new AlgebricksException("Can only get external dataset files");
+		}
+		ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails)dataset.getDatasetDetails();
+		IGenericDatasetAdapterFactory adapterFactory;
+		IDatasourceAdapter adapter;
+		String adapterName;
+		DatasourceAdapter adapterEntity;
+		String adapterFactoryClassname;
+		try {
+			adapterName = datasetDetails.getAdapter();
+			adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+					adapterName);
+			if (adapterEntity != null) {
+				adapterFactoryClassname = adapterEntity.getClassname();
+				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+			} else {
+				adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
+				if (adapterFactoryClassname == null) {
+					throw new AlgebricksException(" Unknown adapter :" + adapterName);
+				}
+				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+			}
+
+			adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
+					wrapProperties(datasetDetails.getProperties()), null);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			throw new AlgebricksException("Unable to create adapter " + e);
+		}
+		
+		try {
+			ArrayList<FileStatus> fileStatuses = ExternalDataFilesMetadataProvider.getHDFSFileStatus((AbstractDatasourceAdapter) adapter);
+			for(int i=0; i<fileStatuses.size(); i++)
+			{
+				files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(), new Date(fileStatuses.get(i).getModificationTime()),
+						fileStatuses.get(i).getLen(),
+						fileStatuses.get(i).getPath().toUri().getPath(),
+						i));
+			}
+			return files;
+		} catch (IOException e) {
+			e.printStackTrace();
+			throw new AlgebricksException("Unable to get list of HDFS files " + e);
+		}
+	}
+
+	@SuppressWarnings("rawtypes")
+	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataAccesByRIDRuntime(
+			JobSpecification jobSpec, Dataset dataset, Index secondaryIndex)
+					throws AlgebricksException {
+		IAType itemType = null;
+		try {
+			itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
+		} catch (MetadataException e) {
+			e.printStackTrace();
+			throw new AlgebricksException("Unable to get item type from metadata " + e);
+		}
+		if (itemType.getTypeTag() != ATypeTag.RECORD) {
+			throw new AlgebricksException("Can only scan datasets of records.");
+		}
+
+		ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails)dataset.getDatasetDetails();
+		IGenericDatasetAdapterFactory adapterFactory;
+		IDatasourceAdapter adapter;
+		String adapterName;
+		DatasourceAdapter adapterEntity;
+		String adapterFactoryClassname;
+		try {
+			adapterName = datasetDetails.getAdapter();
+			adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
+					adapterName);
+			if (adapterEntity != null) {
+				adapterFactoryClassname = adapterEntity.getClassname();
+				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+			} else {
+				adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
+				if (adapterFactoryClassname == null) {
+					throw new AlgebricksException(" Unknown adapter :" + adapterName);
+				}
+				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
+			}
+
+			adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
+					wrapProperties(datasetDetails.getProperties()), itemType);
+		} catch (AlgebricksException ae) {
+			throw ae;
+		} catch (Exception e) {
+			e.printStackTrace();
+			throw new AlgebricksException("Unable to create adapter " + e);
+		}
+
+		if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
+				IDatasourceAdapter.AdapterType.READ_WRITE))) {
+			throw new AlgebricksException("external dataset adapter does not support read operation");
+		}
+		IDataFormat format = NonTaggedDataFormat.INSTANCE;
+		ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
+		RecordDescriptor outRecDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
+
+		ExternalDataAccessByRIDOperatorDescriptor dataAccessOperator = null;
+		if(optimizeExternalIndexes)
+		{
+			//create the hashmap
+			List<ExternalFile> files=null;
+			try {
+				files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
+			} catch (MetadataException e) {
+				e.printStackTrace();
+				throw new AlgebricksException("Couldn't get file names for access by optimized RIDs",e);
+			}
+			HashMap<Integer, String> filesMapping = new HashMap<Integer, String>();
+			for(int i=0; i < files.size(); i++)
+			{
+				filesMapping.put(files.get(i).getFileNumber(), files.get(i).getFileName());
+			}
+			dataAccessOperator = new ExternalDataAccessByRIDOperatorDescriptor(jobSpec, wrapPropertiesEmpty(datasetDetails.getProperties()),
+					itemType, outRecDesc, adapterFactory, filesMapping);
+		}
+		else
+		{
+			dataAccessOperator = new ExternalDataAccessByRIDOperatorDescriptor(jobSpec, wrapPropertiesEmpty(datasetDetails.getProperties()),
+					itemType, outRecDesc, adapterFactory, null);
+		}
+		Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraints = splitProviderAndPartitionConstraintsForExternalDataset(dataset.getDataverseName(),dataset.getDatasetName(),secondaryIndex.getIndexName());
+		return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataAccessOperator, splitsAndConstraints.second);
+	}
 
     @SuppressWarnings("rawtypes")
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildScannerRuntime(JobSpecification jobSpec,
@@ -500,64 +730,111 @@
     }
 
     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, Object implConfig)
-            throws AlgebricksException {
-        boolean isSecondary = true;
-        try {
-            Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-                    dataset.getDatasetName(), dataset.getDatasetName());
-            if (primaryIndex != null) {
-                isSecondary = !indexName.equals(primaryIndex.getIndexName());
-            }
-            int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-            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;
-                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);
+			List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
+			JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] lowKeyFields,
+			int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive, Object implConfig)
+					throws AlgebricksException {
+		boolean isSecondary = true;
+		if(dataset.getDatasetType() == DatasetType.EXTERNAL){
+			try {
+				int numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
+				RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+				int numKeys = numPrimaryKeys;;
+				ITypeTraits[] typeTraits = null;
+				int[] bloomFilterKeyFields;
+				Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+						dataset.getDatasetName(), indexName);
+				int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
+				numKeys += numSecondaryKeys;
+				int keysStartIndex = outputVars.size() - numKeys;
+				typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
+				bloomFilterKeyFields = new int[numSecondaryKeys];
+				for (int i = 0; i < numSecondaryKeys; i++) {
+					bloomFilterKeyFields[i] = i;
+				}
+				IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+						outputVars, keysStartIndex, numKeys, typeEnv, context);
+				IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+				Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
+				try {
+					spPc = splitProviderAndPartitionConstraintsForExternalDataset(dataset.getDataverseName(),
+							dataset.getDatasetName(), indexName);
+				} catch (Exception e) {
+					throw new AlgebricksException(e);
+				}
+				ISearchOperationCallbackFactory searchCallbackFactory = null;
+				searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
+				AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
+				BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
+	                    appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
+	                    typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
+	                    lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
+	                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider,
+	                            isSecondary ? new SecondaryIndexOperationTrackerProvider(
+	                                    LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId())
+	                                    : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
+	                            rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
+	                    searchCallbackFactory);
+				return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+			} catch (MetadataException me) {
+				throw new AlgebricksException(me);
+			}
+		}
+		else
+		{
+			try {
+				Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+						dataset.getDatasetName(), dataset.getDatasetName());
+				if (primaryIndex != null) {
+					isSecondary = !indexName.equals(primaryIndex.getIndexName());
+				}
+				int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+				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;
+					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;
-            try {
-                spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
-                        dataset.getDatasetName(), indexName);
-            } catch (Exception e) {
-                throw new AlgebricksException(e);
-            }
+				IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+				Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
+				try {
+					spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
+							dataset.getDatasetName(), indexName);
+				} 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;
-                }
+				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;
                 ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
@@ -569,25 +846,26 @@
                             primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
                 }
             }
-            AsterixRuntimeComponentsProvider rtcProvider = isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
-                    : AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER;
+            AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
             BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
                     appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
                     typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
                     lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider,
-                            isSecondary ? AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER
+                            isSecondary ? new SecondaryIndexOperationTrackerProvider(
+                                    LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId())
                                     : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
                             rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
                     searchCallbackFactory);
 
-            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+				return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
 
-        } catch (MetadataException me) {
-            throw new AlgebricksException(me);
-        }
-    }
-
+			} catch (MetadataException me) {
+				throw new AlgebricksException(me);
+			}
+		}
+	}
+    
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] keyFields)
@@ -644,10 +922,10 @@
                     typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            new SecondaryIndexOperationTrackerProvider(LSMRTreeIOOperationCallbackFactory.INSTANCE,
+                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
                             storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
@@ -802,10 +1080,10 @@
                     splitsAndConstraint.first, typeTraits, comparatorFactories, bloomFilterKeyFields, fieldPermutation,
                     GlobalConfig.DEFAULT_BTREE_FILL_FACTOR, false, numElementsHint, true,
                     new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -870,10 +1148,10 @@
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
                     new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
                                     .getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory, true);
 
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
@@ -1066,10 +1344,10 @@
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
                     new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, storageProperties
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
                                     .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory,
                     false);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
@@ -1194,10 +1472,11 @@
                     appContext.getIndexLifecycleManagerProvider(), tokenTypeTraits, tokenComparatorFactories,
                     invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
                     new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
-                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER, storageProperties
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            new SecondaryIndexOperationTrackerProvider(
+                                    LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
                                     .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
                     splitsAndConstraint.second);
@@ -1288,10 +1567,10 @@
                     comparatorFactories, null, fieldPermutation, indexOp, new LSMRTreeDataflowHelperFactory(
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            new SecondaryIndexOperationTrackerProvider(LSMRTreeIOOperationCallbackFactory.INSTANCE,
+                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
                             storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
                     modificationCallbackFactory, false);
@@ -1342,9 +1621,20 @@
         }
 
         int numPartitions = 0;
-        InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
-        List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
-                .getNodeNames();
+        List<String> nodeGroup = null;
+        if(dataset.getDatasetType() == DatasetType.EXTERNAL)
+		{
+			ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+			nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
+					.getNodeNames();
+		}
+		else
+		{
+			InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+			nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
+					.getNodeNames();
+		}
+
         for (String nd : nodeGroup) {
             numPartitions += AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
         }
@@ -1362,6 +1652,12 @@
         return splitProviderAndPartitionConstraints(splits);
     }
 
+    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForExternalDataset(
+			String dataverseName, String datasetName, String targetIdxName) throws AlgebricksException {
+		FileSplit[] splits = splitsForExternalDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName);
+		return splitProviderAndPartitionConstraints(splits);
+	}
+    
     public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataverse(
             String dataverse) {
         FileSplit[] splits = splitsForDataverse(mdTxnCtx, dataverse);
@@ -1452,6 +1748,56 @@
         }
     }
 
+    private FileSplit[] splitsForExternalDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
+			String datasetName, String targetIdxName) throws AlgebricksException {
+
+		try {
+			File relPathFile = new File(getRelativePath(dataverseName, datasetName + "_idx_" + targetIdxName));
+			Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
+			if (dataset.getDatasetType() != DatasetType.EXTERNAL) {
+				throw new AlgebricksException("Not an external dataset");
+			}
+			ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
+			List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
+					.getNodeNames();
+			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 {
+					int numIODevices;
+					if (datasetDetails.getNodeGroupName().compareTo(MetadataConstants.METADATA_NODEGROUP_NAME) == 0) {
+						numIODevices = 1;
+					} else {
+						numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
+					}
+					String[] ioDevices = AsterixClusterProperties.INSTANCE.getIODevices(nd);
+					for (int j = 0; j < nodeStores.length; j++) {
+						for (int k = 0; k < numIODevices; k++) {
+							File f = new File(ioDevices[k] + File.separator + nodeStores[j] + File.separator
+									+ relPathFile);
+							splitArray.add(new FileSplit(nd, new FileReference(f), k));
+						}
+					}
+				}
+			}
+			FileSplit[] splits = new FileSplit[splitArray.size()];
+			int i = 0;
+			for (FileSplit fs : splitArray) {
+				splits[i++] = fs;
+			}
+			return splits;
+		} catch (MetadataException me) {
+			throw new AlgebricksException(me);
+		}
+	}
+    
     private static Map<String, String> initializeAdapterFactoryMapping() {
         Map<String, String> adapterFactoryMapping = new HashMap<String, String>();
         adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter",
@@ -1464,6 +1810,8 @@
                 "edu.uci.ics.asterix.external.dataset.adapter..RSSFeedAdapterFactory");
         adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapter",
                 "edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapterFactory");
+        adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter",
+				"edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory");
         return adapterFactoryMapping;
     }
 
@@ -1562,3 +1910,4 @@
     }
 
 }
+
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java
index 18cef340..849c6bd 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalDatasetDetails.java
@@ -40,17 +40,25 @@
 
     private static final long serialVersionUID = 1L;
     private final String adapter;
-    private final Map<String, String> properties;
+    private final Map<String,String> properties;
+    private final String nodeGroupName;
 
-    private final static ARecordType externalRecordType = MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE;
+	private final static ARecordType externalRecordType = MetadataRecordTypes.EXTERNAL_DETAILS_RECORDTYPE;
     private final static ARecordType propertyRecordType = MetadataRecordTypes.DATASOURCE_ADAPTER_PROPERTIES_RECORDTYPE;
 
-    public ExternalDatasetDetails(String adapter, Map<String, String> properties) {
+    public ExternalDatasetDetails(String adapter, Map<String,String> properties, String nodeGroupName) {
         this.properties = properties;
         this.adapter = adapter;
+        this.nodeGroupName = nodeGroupName;
+    }
+    
+    public ExternalDatasetDetails(String adapter, Map<String,String> properties) {
+        this.properties = properties;
+        this.adapter = adapter;
+        this.nodeGroupName = null;
     }
 
-    public String getAdapter() {
+	public String getAdapter() {
         return adapter;
     }
 
@@ -58,6 +66,10 @@
         return properties;
     }
 
+    public String getNodeGroupName() {
+		return nodeGroupName;
+	}
+    
     @Override
     public DatasetType getDatasetType() {
         return DatasetType.EXTERNAL;
@@ -93,6 +105,12 @@
         fieldValue.reset();
         listBuilder.write(fieldValue.getDataOutput(), true);
         externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_PROPERTIES_FIELD_INDEX, fieldValue);
+        
+        //write field 2
+        fieldValue.reset();
+        aString.setValue(getNodeGroupName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        externalRecordBuilder.addField(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX, fieldValue);
 
         try {
             externalRecordBuilder.write(out, true);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java
new file mode 100644
index 0000000..0128783
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java
@@ -0,0 +1,110 @@
+package edu.uci.ics.asterix.metadata.entities;
+
+import java.util.Date;
+
+import edu.uci.ics.asterix.metadata.MetadataCache;
+import edu.uci.ics.asterix.metadata.api.IMetadataEntity;
+
+public class ExternalFile implements IMetadataEntity{
+
+	/**
+	 * A class for metadata entity externalFile
+	 * This class represents an external dataset file and is intended for use with external data indexes
+	 */
+	private static final long serialVersionUID = 1L;
+	
+	private String dataverseName;
+	private String datasetName;
+	private Date lastModefiedTime;
+	private long size;
+	private String fileName;
+	private int fileNumber;
+	
+	
+	public ExternalFile(String dataverseName, String datasetName, Date lastModefiedTime, long size, String fileName,
+			int fileNumber) {
+		this.dataverseName = dataverseName;
+		this.datasetName = datasetName;
+		this.lastModefiedTime = lastModefiedTime;
+		this.size = size;
+		this.fileName = fileName;
+		this.fileNumber = fileNumber;
+	}
+
+	public String getDataverseName() {
+		return dataverseName;
+	}
+
+	public void setDataverseName(String dataverseName) {
+		this.dataverseName = dataverseName;
+	}
+
+	public String getDatasetName() {
+		return datasetName;
+	}
+
+	public void setDatasetName(String datasetName) {
+		this.datasetName = datasetName;
+	}
+	public Date getLastModefiedTime() {
+		return lastModefiedTime;
+	}
+
+	public void setLastModefiedTime(Date lastModefiedTime) {
+		this.lastModefiedTime = lastModefiedTime;
+	}
+
+	public long getSize() {
+		return size;
+	}
+
+	public void setSize(long size) {
+		this.size = size;
+	}
+
+	public String getFileName() {
+		return fileName;
+	}
+
+	public void setFileName(String fileName) {
+		this.fileName = fileName;
+	}
+
+	public int getFileNumber() {
+		return fileNumber;
+	}
+
+	public void setFileNumber(int fileNumber) {
+		this.fileNumber = fileNumber;
+	}
+
+	@Override
+	public Object addToCache(MetadataCache cache) {
+		//return cache.addExternalFileIfNotExists(this);
+		return null;
+	}
+
+	@Override
+	public Object dropFromCache(MetadataCache cache) {
+		//cache.dropExternalFile(this);
+		return null;
+	}
+
+	@Override
+	public boolean equals(Object obj)
+	{
+		if (obj == null)
+            return false;
+        if (obj == this)
+            return true;
+        if (!(obj instanceof ExternalFile))
+            return false;
+        ExternalFile anotherFile = (ExternalFile) obj;
+        if(fileNumber != anotherFile.fileNumber)
+        	return false;
+        if(!fileName.equals(anotherFile.fileName))
+        	return false;
+        return true;
+	}
+	
+}
\ No newline at end of file
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 61f856a..d346a6c 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
@@ -219,7 +219,10 @@
                             .getStringValue();
                     properties.put(key, value);
                 }
-                datasetDetails = new ExternalDatasetDetails(adapter, properties);
+                String groupName = ((AString) datasetDetailsRecord
+                        .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX ))
+                        .getStringValue();
+                datasetDetails = new ExternalDatasetDetails(adapter, properties,groupName);
         }
         
         Map<String, String> hints = getDatasetHints(datasetRecord);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
new file mode 100644
index 0000000..6837c72
--- /dev/null
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
@@ -0,0 +1,158 @@
+package edu.uci.ics.asterix.metadata.entitytupletranslators;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.util.Date;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataPrimaryIndexes;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataRecordTypes;
+import edu.uci.ics.asterix.metadata.entities.ExternalFile;
+import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.AInt64;
+import edu.uci.ics.asterix.om.base.AMutableDateTime;
+import edu.uci.ics.asterix.om.base.AMutableInt32;
+import edu.uci.ics.asterix.om.base.AMutableInt64;
+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;
+
+public class ExternalFileTupleTranslator extends AbstractTupleTranslator<ExternalFile>{
+
+	// Field indexes of serialized ExternalFile in a tuple.
+    // First key field.
+    public static final int EXTERNAL_FILE_DATAVERSENAME_TUPLE_FIELD_INDEX = 0;
+    // Second key field.
+    public static final int EXTERNAL_FILE_DATASETNAME_TUPLE_FIELD_INDEX = 1;
+    // Third key field
+    public static final int EXTERNAL_FILE_NUMBER_TUPLE_FIELD_INDEX = 2;
+    // Payload field containing serialized ExternalFile.
+    public static final int EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX = 3;
+
+    protected AMutableInt32 aInt32 = new AMutableInt32(0);
+    protected AMutableDateTime aDateTime = new AMutableDateTime(0);
+    protected AMutableInt64 aInt64 = new AMutableInt64(0);
+    
+    @SuppressWarnings("unchecked")
+	protected ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT32);
+    @SuppressWarnings("unchecked")
+	protected ISerializerDeserializer<ADateTime> dateTimeSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ADATETIME);
+    @SuppressWarnings("unchecked")
+	protected ISerializerDeserializer<AInt64> longSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AINT64);
+    @SuppressWarnings("unchecked")
+    private ISerializerDeserializer<ARecord> recordSerDes = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE);
+    
+	public ExternalFileTupleTranslator(boolean getTuple) {
+		super(getTuple, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET.getFieldCount());
+	}
+
+	@Override
+	public ExternalFile getMetadataEntytiFromTuple(ITupleReference tuple)
+			throws MetadataException, IOException {
+		byte[] serRecord = tuple.getFieldData(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
+		int recordStartOffset = tuple.getFieldStart(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
+        int recordLength = tuple.getFieldLength(EXTERNAL_FILE_PAYLOAD_TUPLE_FIELD_INDEX);
+        ByteArrayInputStream stream = new ByteArrayInputStream(serRecord, recordStartOffset, recordLength);
+        DataInput in = new DataInputStream(stream);
+        ARecord externalFileRecord = (ARecord) recordSerDes.deserialize(in);
+        return createExternalFileFromARecord(externalFileRecord);
+	}
+
+	private ExternalFile createExternalFileFromARecord(ARecord externalFileRecord) {
+		String dataverseName = ((AString) externalFileRecord
+                .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATAVERSENAME_FIELD_INDEX))
+                .getStringValue();
+        String datasetName = ((AString) externalFileRecord
+                .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATASET_NAME_FIELD_INDEX)).getStringValue();
+        String FileName = ((AString) externalFileRecord
+                .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NAME_FIELD_INDEX)).getStringValue();
+        int fileNumber = ((AInt32) externalFileRecord
+                .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NUMBER_FIELD_INDEX)).getIntegerValue();
+        Date lastMoDifiedDate = new Date(((ADateTime) externalFileRecord
+                .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_MOD_DATE_FIELD_INDEX)).getChrononTime());
+        long fileSize = ((AInt64) externalFileRecord
+                .getValueByPos(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_SIZE_FIELD_INDEX)).getLongValue();
+        
+        return new ExternalFile(dataverseName, datasetName,lastMoDifiedDate,fileSize,FileName,fileNumber);
+	}
+
+	@Override
+	public ITupleReference getTupleFromMetadataEntity(
+			ExternalFile externalFile) throws MetadataException, IOException {
+		// write the key in the first 3 fields of the tuple
+        tupleBuilder.reset();
+        //dataverse name
+        aString.setValue(externalFile.getDataverseName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+        //dataset name
+        aString.setValue(externalFile.getDatasetName());
+        stringSerde.serialize(aString, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+        //file number
+        aInt32.setValue(externalFile.getFileNumber());
+        intSerde.serialize(aInt32, tupleBuilder.getDataOutput());
+        tupleBuilder.addFieldEndOffset();
+        
+        // write the pay-load in the fourth field of the tuple
+        recordBuilder.reset(MetadataRecordTypes.EXTERNAL_FILE_RECORDTYPE);
+
+        // write field 0
+        fieldValue.reset();
+        aString.setValue(externalFile.getDataverseName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATAVERSENAME_FIELD_INDEX, fieldValue);
+
+        // write field 1
+        fieldValue.reset();
+        aString.setValue(externalFile.getDatasetName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_DATASET_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 2
+        fieldValue.reset();
+        aInt32.setValue(externalFile.getFileNumber());
+        intSerde.serialize(aInt32, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NUMBER_FIELD_INDEX, fieldValue);
+
+        // write field 3
+        fieldValue.reset();
+        aString.setValue(externalFile.getFileName());
+        stringSerde.serialize(aString, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_NAME_FIELD_INDEX, fieldValue);
+
+        // write field 4
+        fieldValue.reset();
+        aInt64.setValue(externalFile.getSize());
+        longSerde.serialize(aInt64, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_SIZE_FIELD_INDEX, fieldValue);
+
+        //write field 5 
+        fieldValue.reset();
+        aDateTime.setValue(externalFile.getLastModefiedTime().getTime());
+        dateTimeSerde.serialize(aDateTime, fieldValue.getDataOutput());
+        recordBuilder.addField(MetadataRecordTypes.EXTERNAL_FILE_ARECORD_FILE_MOD_DATE_FIELD_INDEX, fieldValue);
+        
+        // write record
+        try {
+            recordBuilder.write(tupleBuilder.getDataOutput(), true);
+        } catch (AsterixException e) {
+            throw new MetadataException(e);
+        }
+        tupleBuilder.addFieldEndOffset();
+
+        tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
+        return tuple;
+	}
+}
\ No newline at end of file
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 820f277..95d26d9 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
@@ -20,9 +20,12 @@
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 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.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
@@ -32,6 +35,8 @@
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public class DatasetUtils {
+	public static final String KEY_INPUT_FORMAT = "input-format";
+	public static final String INPUT_FORMAT_RC = "rc-input-format";
     public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset,
             ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
             throws AlgebricksException {
@@ -82,6 +87,46 @@
         }
         return bhffs;
     }
+    
+    public static IBinaryHashFunctionFactory[] computeExternalDataKeysBinaryHashFunFactories(Dataset dataset,
+			IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
+    	if (dataset.getDatasetType() != DatasetType.EXTERNAL) {
+            throw new AlgebricksException("not implemented");
+        }
+    	//get dataset details
+    	ExternalDatasetDetails edd = (ExternalDatasetDetails) dataset.getDatasetDetails();	
+    	if (edd.getProperties().get(KEY_INPUT_FORMAT).trim().equals(INPUT_FORMAT_RC))
+    	{
+    		//RID: <String(File name) OR Int32(File number), Int64(Block byte location), Int32(row number)>
+    		IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[3];
+    		if(AqlMetadataProvider.isOptimizeExternalIndexes())
+    		{
+    			bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT32);
+    		}
+    		else
+    		{
+    			bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.ASTRING);
+    		}
+			bhffs[1] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT64);
+			bhffs[2] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT32);
+			return bhffs;
+    	}
+		else
+		{
+			//RID: <String(File name) OR Int32(File number), Int64(Record byte location)>
+			IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[2];
+			if(AqlMetadataProvider.isOptimizeExternalIndexes())
+    		{
+    			bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT32);
+    		}
+    		else
+    		{
+    			bhffs[0] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.ASTRING);
+    		}
+			bhffs[1] = hashFunProvider.getBinaryHashFunctionFactory(BuiltinType.AINT64);
+			return bhffs;
+		}
+	}
 
     public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType)
             throws AlgebricksException {
@@ -112,6 +157,17 @@
         return (((InternalDatasetDetails) dataset.getDatasetDetails())).getNodeGroupName();
     }
 
+    public static int getExternalRIDSize(Dataset dataset) {
+		ExternalDatasetDetails dsd = ((ExternalDatasetDetails) dataset.getDatasetDetails());
+		if (dsd.getProperties().get(KEY_INPUT_FORMAT).equals(INPUT_FORMAT_RC))
+		{
+			return 3;
+		}
+		else{
+			return 2;
+		}
+	}
+    
     public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
         for (int i = 0; i < partitioningKeys.size(); i++) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
index cc3fd04..00b7459 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AInt64.java
@@ -33,6 +33,10 @@
     public long getLongValue() {
         return value;
     }
+    
+    public void setValue(long value) {
+        this.value = value;
+    }
 
     @Override
     public IAType getType() {
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 3c8a0c4..b6bdb4b 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
@@ -243,6 +243,8 @@
 
     public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "index-search", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier EXTERNAL_ACCESS_BY_RID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "external-access-by-rid", FunctionIdentifier.VARARGS);
 
     public final static FunctionIdentifier MAKE_FIELD_INDEX_HANDLE = new FunctionIdentifier(
             FunctionConstants.ASTERIX_NS, "make-field-index-handle", 2);
@@ -672,6 +674,14 @@
                 return BuiltinType.ANY; // TODO
             }
         });
+        addPrivateFunction(EXTERNAL_ACCESS_BY_RID, new IResultTypeComputer() {
+
+            @Override
+            public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+                    IMetadataProvider<?, ?> mp) throws AlgebricksException {
+                return BuiltinType.ANY;
+            }
+        });
         addFunction(INT8_CONSTRUCTOR, OptionalAInt8TypeComputer.INSTANCE);
         addFunction(INT16_CONSTRUCTOR, OptionalAInt16TypeComputer.INSTANCE);
         addFunction(INT32_CONSTRUCTOR, OptionalAInt32TypeComputer.INSTANCE);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
index 6cdaea0..0f836af 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixAppContextInfo.java
@@ -100,11 +100,11 @@
 
     @Override
     public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
 
     @Override
     public IStorageManagerInterface getStorageManagerInterface() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
+        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
     }
 }
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 8092ecd..b44755c 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
@@ -18,7 +18,6 @@
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.hyracks.api.context.IHyracksJobletContext;
@@ -51,8 +50,7 @@
                     ITransactionManager txnManager = ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext()
                             .getApplicationObject()).getTransactionSubsystem().getTransactionManager();
                     ITransactionContext txnContext = txnManager.getTransactionContext(jobId);
-                    txnContext.setTransactionType(transactionalWrite ? TransactionType.READ_WRITE
-                            : TransactionType.READ);
+                    txnContext.setWriteTxn(transactionalWrite);
                     txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
                             !(jobStatus == JobStatus.FAILURE));
                 } catch (ACIDException e) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
index b8c8659..e08c509 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
@@ -92,6 +92,16 @@
             throw new AsterixException(e);
         }
     }
+    
+    @Override
+    public void close() throws IOException{
+    	admLexer.close();
+    }
+    
+    @Override
+    public void reset(){
+    	admLexer.reset();
+    }
 
     protected boolean parseAdmInstance(IAType objectType, boolean datasetRec, DataOutput out) throws AsterixException,
             IOException {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
new file mode 100644
index 0000000..a4f1691
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
@@ -0,0 +1,123 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
+import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
+
+/**
+ * An Abstract class implementation for IControlledTupleParser. It provides common
+ * functionality involved in parsing data in an external format in a pipelined manner and packing
+ * frames with formed tuples.
+ * (DONE)
+ */
+public abstract class AbstractControlledTupleParser extends ControlledTupleParser{
+
+	protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
+	protected transient DataOutput dos;
+	protected final FrameTupleAppender appender;
+	protected final ByteBuffer frame;
+	protected final ARecordType recType;
+	protected final IHyracksTaskContext ctx;
+	protected IDataParser parser;
+	
+	public AbstractControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
+		appender = new FrameTupleAppender(ctx.getFrameSize());
+		frame = ctx.allocateFrame();
+		this.recType = recType;
+		this.ctx = ctx;
+		dos = tb.getDataOutput();
+	}
+
+	public abstract IDataParser getDataParser();
+
+	@Override
+	public void parse(InputStream in, IFrameWriter writer)
+			throws HyracksDataException {
+		//This function when used works as non-pipelined parser
+		//This whole parser interface needs revisiting. 
+		appender.reset(frame, true);
+		parser = getDataParser();
+		try {
+			parser.initialize(in, recType, true);
+			while (true) {
+				tb.reset();
+				if (!parser.parse(tb.getDataOutput())) {
+					parser.reset();
+					break;
+				}
+				tb.addFieldEndOffset();
+				addTupleToFrame(writer);
+			}
+			parser.close();
+			if (appender.getTupleCount() > 0) {
+				FrameUtils.flushFrame(frame, writer);
+			}
+		} catch (Exception e) {
+			throw new HyracksDataException("Failed to initialize data parser");
+		}
+	}
+
+	@Override
+	public void initialize(InputStream in) throws HyracksDataException {
+		appender.reset(frame, true);
+		parser = getDataParser();
+		try {
+			parser.initialize(in, recType, true);
+
+		} catch (Exception e) {
+			throw new HyracksDataException("Failed to initialize data parser");
+		}
+	}
+
+	@Override
+	public void parseNext(IFrameWriter writer) throws HyracksDataException {
+		try {
+			while (true) {
+				tb.reset();
+				if (!parser.parse(tb.getDataOutput())) {
+					parser.reset();
+					break;
+				}
+				tb.addFieldEndOffset();
+				addTupleToFrame(writer);
+			}
+		} catch (AsterixException ae) {
+			throw new HyracksDataException(ae);
+		} catch (IOException ioe) {
+			throw new HyracksDataException(ioe);
+		}
+	}
+
+	@Override
+	public void close(IFrameWriter writer) throws HyracksDataException {
+		try{	
+			parser.close();
+			if (appender.getTupleCount() > 0) {
+				FrameUtils.flushFrame(frame, writer);
+			}
+		} catch (IOException ioe) {
+			throw new HyracksDataException(ioe);
+		}
+	}
+
+	protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
+		if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+			FrameUtils.flushFrame(frame, writer);
+			appender.reset(frame, true);
+			if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
+				throw new IllegalStateException();
+			}
+		}
+
+	}
+
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
index 2322338..78159f5 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
@@ -62,6 +62,7 @@
             while (true) {
                 tb.reset();
                 if (!parser.parse(tb.getDataOutput())) {
+                	parser.close();
                     break;
                 }
                 tb.addFieldEndOffset();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
index 7cb9bb0..34f75e6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
@@ -20,7 +20,7 @@
 
 /**
  * An extension of AbstractTupleParser that provides functionality for
- * parsing delimited files.
+ * parsing adm formatted input files.
  */
 public class AdmTupleParser extends AbstractTupleParser {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
new file mode 100644
index 0000000..aa2d9ba
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+/**
+ * An extension of AbstractControlledTupleParser that provides functionality for
+ * parsing Adm formatted input.
+ */
+public class ControlledADMTupleParser extends AbstractControlledTupleParser{
+
+	public ControlledADMTupleParser(IHyracksTaskContext ctx, ARecordType recType)
+			throws HyracksDataException {
+		super(ctx, recType);
+	}
+
+	@Override
+	public IDataParser getDataParser() {
+		return new ADMDataParser();
+	}
+
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
new file mode 100644
index 0000000..d72366c
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+/**
+ * A Controlled tuple parser factory for creating a tuple parser capable of parsing
+ * ADM data.
+ */
+public class ControlledADMTupleParserFactory implements ITupleParserFactory{
+	private static final long serialVersionUID = 1L;
+
+    protected ARecordType recType;
+    
+    public ControlledADMTupleParserFactory(ARecordType recType){
+    	this.recType = recType;
+    }
+
+	@Override
+	public ITupleParser createTupleParser(IHyracksTaskContext ctx)
+			throws HyracksDataException {
+		return new ControlledADMTupleParser(ctx, recType);
+	}
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
new file mode 100644
index 0000000..15643d4
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
@@ -0,0 +1,23 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+
+public class ControlledDelimitedDataTupleParser extends AbstractControlledTupleParser{
+
+	private final DelimitedDataParser dataParser;
+	
+	public ControlledDelimitedDataTupleParser(IHyracksTaskContext ctx,
+			ARecordType recType,  IValueParserFactory[] valueParserFactories, char fieldDelimter) throws HyracksDataException {
+		super(ctx, recType);
+		dataParser = new DelimitedDataParser(recType, valueParserFactories, fieldDelimter);
+	}
+
+	@Override
+	public IDataParser getDataParser() {
+		return dataParser;
+	}
+
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
new file mode 100644
index 0000000..ced33ef
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
+
+public class ControlledDelimitedDataTupleParserFactory implements ITupleParserFactory{
+	private static final long serialVersionUID = 1L;
+	private IValueParserFactory[] valueParserFactories;
+	private char fieldDelimiter;
+	protected ARecordType recordType;
+
+
+	public ControlledDelimitedDataTupleParserFactory(ARecordType recordType, IValueParserFactory[] fieldParserFactories, char fieldDelimiter) {
+		this.recordType = recordType;
+		this.valueParserFactories = fieldParserFactories;
+		this.fieldDelimiter = fieldDelimiter;
+	}
+	
+
+	@Override
+	public ITupleParser createTupleParser(IHyracksTaskContext ctx)
+			throws HyracksDataException {
+		return new ControlledDelimitedDataTupleParser(ctx, recordType, valueParserFactories, fieldDelimiter);
+	}
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
new file mode 100644
index 0000000..10b09f5
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
@@ -0,0 +1,29 @@
+package edu.uci.ics.asterix.runtime.operators.file;
+
+import java.io.InputStream;
+
+import edu.uci.ics.hyracks.api.comm.IFrameWriter;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
+
+/**
+ * This interface is to be implemented by parsers used in a pipelined hyracks job where input is not ready all at once
+ */
+public abstract class ControlledTupleParser implements ITupleParser{
+
+	/**
+	 * This function associate an input stream with the parser
+	 */
+	public abstract void initialize(InputStream in) throws HyracksDataException;
+	
+	/**
+	 * This function should flush the tuples setting in the frame writer buffer
+	 * and free all resources
+	 */
+	public abstract void close(IFrameWriter writer) throws HyracksDataException;
+
+	/**
+	 * This function is called when there are more data ready for parsing in the input stream
+	 */
+	public abstract void parseNext(IFrameWriter writer) throws HyracksDataException;
+}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
index 5a639dc..22c43ec 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
@@ -100,6 +100,17 @@
     }
 
     @Override
+	public void close() {
+		cursor.close();
+	}
+
+    @Override
+    public void reset()
+    {
+    	cursor.reset();
+    }
+    
+    @Override
     public boolean parse(DataOutput out) throws AsterixException, IOException {
         while (cursor.nextRecord()) {
             recBuilder.reset(recordType);
@@ -165,7 +176,7 @@
         private static final int INITIAL_BUFFER_SIZE = 4096;
         private static final int INCREMENT = 4096;
 
-        private final Reader in;
+        private Reader in;
 
         private char[] buffer;
         private int start;
@@ -182,6 +193,21 @@
             end = 0;
             state = State.INIT;
         }
+        
+        public void close(){
+				try {
+					in.close();
+				} catch (IOException e) {
+					// TODO Auto-generated catch block
+					e.printStackTrace();
+				}
+        }
+        
+        public void reset(){
+        	start = 0;
+            end = 0;
+            state = State.INIT;
+        }
 
         public boolean nextRecord() throws IOException {
             while (true) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
index 46d281b..cd9ae2e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
@@ -52,4 +52,17 @@
      * @throws IOException
      */
     public boolean parse(DataOutput out) throws AsterixException, IOException;
+
+    /**
+     * Close the underlying inputStream object.
+     * 
+     */
+    public void close() throws IOException;
+    
+    /**
+     * Reset the parser before processing a new patch of input in the input stream
+     */
+    public void reset();
 }
+
+
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
index e037ec6..ffd9edd 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
@@ -224,7 +224,8 @@
             while (continueIngestion) {
                 tb.reset();
                 if (!parser.parse(tb.getDataOutput())) {
-                    break;
+                	parser.close();
+                	break;
                 }
                 tb.addFieldEndOffset();
                 if (delayConfigured) {
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
index bf3c086..c576cf8 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
@@ -14,14 +14,17 @@
  */
 package edu.uci.ics.asterix.tools.external.data;
 
+import java.util.HashMap;
 import java.util.Map;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
+import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 
 /**
  * Factory class for creating @see{RateControllerFileSystemBasedAdapter} The
@@ -67,6 +70,12 @@
     public String getName() {
         return "file_feed";
     }
+    
+    @Override
+	public IDatasourceAdapter createIndexingAdapter(
+			Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
+		throw new NotImplementedException("Rate Controlled Indexing Adapter is not implemented for feeds");
+	}
 
     private void checkRequiredArgs(Map<String, Object> configuration) throws Exception {
         if (configuration.get(KEY_FILE_SYSTEM) == null) {
@@ -83,4 +92,10 @@
         }
     }
 
+	@Override
+	public IControlledAdapter createAccessByRIDAdapter(
+			Map<String, Object> configuration, IAType atype, HashMap<Integer,String> files) throws Exception {
+		throw new NotImplementedException("Rate Controlled Access by RID Adapter is not implemented for feeds");
+	}
+
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
new file mode 100644
index 0000000..031a26e
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -0,0 +1,82 @@
+/*
+ * 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.transaction.management.opcallbacks;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+public abstract class AbstractIndexModificationOperationCallback extends AbstractOperationCallback {
+
+    protected final long resourceId;
+    protected final byte resourceType;
+    protected final IndexOperation indexOp;
+    protected final ITransactionSubsystem txnSubsystem;
+    protected final SimpleTupleWriter tupleWriter;
+    protected final ILogRecord logRecord;
+
+    protected AbstractIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+            ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
+            byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager);
+        this.resourceId = resourceId;
+        this.resourceType = resourceType;
+        this.indexOp = indexOp;
+        this.txnSubsystem = txnSubsystem;
+        tupleWriter = new SimpleTupleWriter();
+        logRecord = new LogRecord();
+        logRecord.setTxnCtx(txnCtx);
+        logRecord.setLogType(LogType.UPDATE);
+        logRecord.setJobId(txnCtx.getJobId().getId());
+        logRecord.setDatasetId(datasetId);
+        logRecord.setResourceId(resourceId);
+        logRecord.setResourceType(resourceType);
+        logRecord.setNewOp((byte) (indexOp.ordinal()));
+    }
+
+    protected void log(int PKHash, ITupleReference newValue, IndexOperation oldOp, ITupleReference oldValue)
+            throws ACIDException {
+        logRecord.setPKHashValue(PKHash);
+        logRecord.setPKFields(primaryKeyFields);
+        logRecord.setPKValue(newValue);
+        logRecord.computeAndSetPKValueSize();
+        if (newValue != null) {
+            logRecord.setNewValueSize(tupleWriter.bytesRequired(newValue));
+            logRecord.setNewValue(newValue);
+        } else {
+            logRecord.setNewValueSize(0);
+        }
+        if (resourceType == ResourceType.LSM_BTREE) {
+            logRecord.setOldOp((byte) (oldOp.ordinal()));
+            if (oldValue != null) {
+                logRecord.setOldValueSize(tupleWriter.bytesRequired(oldValue));
+                logRecord.setOldValue(oldValue);
+            } else {
+                logRecord.setOldValueSize(0);
+            }
+        }
+        logRecord.computeAndSetLogSize();
+        txnSubsystem.getLogManager().log(logRecord);
+    }
+}
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
index 0024717..44dcad8 100644
--- 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
@@ -16,12 +16,9 @@
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
 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;
@@ -33,22 +30,13 @@
  * Assumes LSM-BTrees as primary indexes.
  * Performs locking on primary keys, and also logs before/after images.
  */
-public class PrimaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+public class PrimaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
         IModificationOperationCallback {
 
-    protected final long resourceId;
-    protected final byte resourceType;
-    protected final IndexOperation indexOp;
-    protected final ITransactionSubsystem txnSubsystem;
-
     public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
             ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
             IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager);
-        this.resourceId = resourceId;
-        this.resourceType = resourceType;
-        this.indexOp = indexOp;
-        this.txnSubsystem = txnSubsystem;
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
     }
 
     @Override
@@ -64,7 +52,6 @@
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
         try {
-            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
             int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
             LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
             IndexOperation oldOp = IndexOperation.INSERT;
@@ -74,8 +61,7 @@
             if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
                 oldOp = IndexOperation.DELETE;
             }
-            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
-                    indexOp, after, oldOp, before);
+            log(pkHash, 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
index 428a04e..abeec62 100644
--- 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
@@ -62,7 +62,7 @@
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
                     primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
                     indexOp);
-            txnCtx.registerIndexAndCallback(index, (AbstractOperationCallback) modCallback);
+            txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
             return modCallback;
         } 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
index e92f82c..ef5ad21 100644
--- 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
@@ -16,13 +16,10 @@
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
 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;
@@ -34,24 +31,16 @@
  * 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
+public class SecondaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
         IModificationOperationCallback {
 
-    protected final long resourceId;
-    protected final byte resourceType;
-    protected final IndexOperation indexOp;
     protected final IndexOperation oldOp;
-    protected final ITransactionSubsystem txnSubsystem;
 
     public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
             ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
             byte resourceType, IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager);
-        this.resourceId = resourceId;
-        this.resourceType = resourceType;
-        this.indexOp = indexOp;
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
         oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
-        this.txnSubsystem = txnSubsystem;
     }
 
     @Override
@@ -61,9 +50,7 @@
 
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-
         try {
-            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
             int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
             IndexOperation effectiveOldOp;
             if (resourceType == ResourceType.LSM_BTREE) {
@@ -78,8 +65,7 @@
             } else {
                 effectiveOldOp = oldOp;
             }
-            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
-                    indexOp, after, effectiveOldOp, before);
+            this.log(pkHash, after, effectiveOldOp, 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
index 55ca399..d5bc877 100644
--- 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
@@ -17,6 +17,7 @@
 
 import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
@@ -55,8 +56,10 @@
 
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
-            return new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
+            IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
                     txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
+            txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
+            return modCallback;
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
index 3661537..aec378b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
@@ -14,7 +14,9 @@
  */
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
+import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -24,15 +26,19 @@
 
     private static final long serialVersionUID = 1L;
 
+    private final int datasetID;
     private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
 
-    public SecondaryIndexOperationTrackerProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+    public SecondaryIndexOperationTrackerProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory, int datasetID) {
         this.ioOpCallbackFactory = ioOpCallbackFactory;
+        this.datasetID = datasetID;
     }
 
     @Override
     public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
-        return new BaseOperationTracker(ioOpCallbackFactory);
+        DatasetLifecycleManager dslcManager = (DatasetLifecycleManager) ((IAsterixAppRuntimeContext) ctx
+                .getJobletContext().getApplicationContext().getApplicationObject()).getIndexLifecycleManager();
+        return new BaseOperationTracker(dslcManager, ioOpCallbackFactory, datasetID);
     }
 
 }
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
index 8ce0174..d243dd2 100644
--- 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
@@ -15,8 +15,10 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
 import java.io.File;
+import java.util.List;
 
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -49,12 +51,13 @@
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) {
         FileReference file = new FileReference(new File(filePath));
-        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
-        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCache, file, runtimeContextProvider
+        List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
+        LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, runtimeContextProvider
                 .getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories,
                 bloomFilterKeyFields, runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
                         .getLSMMergePolicy(), isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID)
-                        : new BaseOperationTracker(LSMBTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
+                        : new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+                                LSMBTreeIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
                         .getLSMIOScheduler(), runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(isPrimary));
         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
index b3da3ee..8482172 100644
--- 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
@@ -14,7 +14,10 @@
  */
 package edu.uci.ics.asterix.transaction.management.resource;
 
+import java.util.List;
+
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -53,24 +56,26 @@
     @Override
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException {
-        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+        List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
         try {
             if (isPartitioned) {
-                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
+                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCaches, runtimeContextProvider
                         .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
                         tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
                         runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
                                 .getLSMMergePolicy(), new BaseOperationTracker(
-                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
+                                (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
                                 .getLSMIOScheduler(), runtimeContextProvider
                                 .getLSMInvertedIndexIOOperationCallbackProvider());
             } else {
-                return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCache, runtimeContextProvider
+                return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCaches, runtimeContextProvider
                         .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
                         tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
                         runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
                                 .getLSMMergePolicy(), new BaseOperationTracker(
-                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE), runtimeContextProvider
+                                (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
                                 .getLSMIOScheduler(), runtimeContextProvider
                                 .getLSMInvertedIndexIOOperationCallbackProvider());
             }
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
index 9b9faef..bc1e889 100644
--- 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
@@ -15,8 +15,10 @@
 package edu.uci.ics.asterix.transaction.management.resource;
 
 import java.io.File;
+import java.util.List;
 
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -58,14 +60,16 @@
     public ILSMIndex createIndexInstance(IAsterixAppRuntimeContextProvider runtimeContextProvider, String filePath,
             int partition) throws HyracksDataException {
         FileReference file = new FileReference(new File(filePath));
-        IVirtualBufferCache virtualBufferCache = runtimeContextProvider.getVirtualBufferCache(datasetID);
+        List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
         try {
-            return LSMRTreeUtils.createLSMTree(virtualBufferCache, file, runtimeContextProvider.getBufferCache(),
+            return LSMRTreeUtils.createLSMTree(virtualBufferCaches, file, runtimeContextProvider.getBufferCache(),
                     runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
                     valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
                     runtimeContextProvider.getLSMMergePolicy(), new BaseOperationTracker(
-                            LSMRTreeIOOperationCallbackFactory.INSTANCE), runtimeContextProvider.getLSMIOScheduler(),
-                    runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(), linearizeCmpFactory);
+                            (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
+                            LSMRTreeIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
+                            .getLSMIOScheduler(), runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(),
+                    linearizeCmpFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
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 b09cd52..6d86f70 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
@@ -30,10 +30,11 @@
 import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPage;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPageReader;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
@@ -78,38 +79,34 @@
     private DeadlockDetector deadlockDetector;
     private TimeOutDetector toutDetector;
     private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
+    private JobId tempJobIdObj;
 
     private int tryLockDatasetGranuleRevertOperation;
 
     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(txnSubsystem.getTransactionProperties().getLockManagerShrinkTimer());
+        this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties()
+                .getLockManagerShrinkTimer());
         this.lockWaiterManager = new LockWaiterManager();
         this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
         this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
                 entityInfoManager, lockWaiterManager);
         this.toutDetector = new TimeOutDetector(this);
         this.tempDatasetIdObj = new DatasetId(0);
+        this.tempJobIdObj = new JobId(0);
         this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
-
-        this.logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
-
         if (IS_DEBUG_MODE) {
             this.lockRequestTracker = new LockRequestTracker();
         }
     }
-     
+
     public AsterixTransactionProperties getTransactionProperties() {
         return this.txnSubsystem.getTransactionProperties();
     }
@@ -200,7 +197,8 @@
                         if (doEscalate) {
                             throw new IllegalStateException(
                                     "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+                                            + txnSubsystem.getTransactionProperties()
+                                                    .getEntityToDatasetLockEscalationThreshold());
                         }
                     }
                 }
@@ -304,9 +302,9 @@
     }
 
     private void validateJob(ITransactionContext txnContext) throws ACIDException {
-        if (txnContext.getTxnState() == TransactionState.ABORTED) {
+        if (txnContext.getTxnState() == ITransactionManager.ABORTED) {
             throw new ACIDException("" + txnContext.getJobId() + " is in ABORTED state.");
-        } else if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS) {
+        } else if (txnContext.isTimeout()) {
             requestAbort(txnContext);
         }
     }
@@ -640,24 +638,17 @@
     }
 
     @Override
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
-            throws ACIDException {
-        return internalUnlock(datasetId, entityHashValue, txnContext, false, false);
-    }
-
-    @Override
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
-            throws ACIDException {
-        return internalUnlock(datasetId, entityHashValue, txnContext, false, commitFlag);
+    public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext) throws ACIDException {
+        internalUnlock(datasetId, entityHashValue, txnContext, false);
     }
 
     private void instantUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
             throws ACIDException {
-        internalUnlock(datasetId, entityHashValue, txnContext, true, false);
+        internalUnlock(datasetId, entityHashValue, txnContext, true);
     }
 
-    private boolean internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
-            boolean isInstant, boolean commitFlag) throws ACIDException {
+    private void internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
+            boolean isInstant) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int eLockInfo = -1;
         DatasetLockInfo dLockInfo = null;
@@ -665,7 +656,6 @@
         int entityInfo = -1;
         byte datasetLockMode;
 
-        boolean lockCountIsZero = false;
         if (IS_DEBUG_MODE) {
             if (entityHashValue == -1) {
                 throw new UnsupportedOperationException(
@@ -713,28 +703,11 @@
 
             if (entityInfoManager.getEntityLockCount(entityInfo) == 0
                     && entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
-                lockCountIsZero = true;
                 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(ITransactionContext.TransactionType.READ_WRITE)) {
-                        try {
-                            txnSubsystem.getLogManager().log(LogType.ENTITY_COMMIT, txnContext, datasetId.getId(),
-                                    entityHashValue, -1, (byte) 0, 0, null, null, logicalLogLocator);
-                        } catch (ACIDException e) {
-                            requestAbort(txnContext);
-                        }
-                    }
-
-                    txnContext.updateLastLSNForIndexes(logicalLogLocator.getLsn());
-                }
-
                 //1) wake up waiters and remove holder
                 //wake up waiters of dataset-granule lock
                 wakeUpDatasetLockWaiters(dLockInfo);
@@ -760,11 +733,6 @@
                     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);
                 }
             }
@@ -782,7 +750,8 @@
 
             if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
                 if (!isInstant && datasetLockMode == LockMode.IS) {
-                    jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+                    jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties()
+                            .getEntityToDatasetLockEscalationThreshold());
                 }
             }
 
@@ -793,7 +762,6 @@
         } finally {
             unlatchLockTable();
         }
-        return lockCountIsZero;
     }
 
     @Override
@@ -974,8 +942,8 @@
             jobHT.remove(jobId);
 
             if (existWaiter) {
-                txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
-                txnContext.setTxnState(TransactionState.ABORTED);
+                txnContext.setTimeout(true);
+                txnContext.setTxnState(ITransactionManager.ABORTED);
             }
 
             if (IS_DEBUG_MODE) {
@@ -1297,7 +1265,8 @@
                             //We don't want to allow the lock escalation when there is a first lock request on a dataset. 
                             throw new IllegalStateException(
                                     "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+                                            + txnSubsystem.getTransactionProperties()
+                                                    .getEntityToDatasetLockEscalationThreshold());
                         }
                     }
                 }
@@ -1824,7 +1793,7 @@
             //waiter woke up -> remove/deallocate waiter object and abort if timeout
             latchLockTable();
 
-            if (txnContext.getStatus() == ITransactionContext.TIMED_OUT_STATUS || waiter.isVictim()) {
+            if (txnContext.isTimeout() || waiter.isVictim()) {
                 requestAbort(txnContext);
             }
 
@@ -1881,8 +1850,7 @@
     }
 
     private void requestAbort(ITransactionContext txnContext) throws ACIDException {
-        txnContext.setStatus(ITransactionContext.TIMED_OUT_STATUS);
-        txnContext.setStartWaitTime(ITransactionContext.INVALID_TIME);
+        txnContext.setTimeout(true);
         throw new ACIDException("Transaction " + txnContext.getJobId()
                 + " should abort (requested by the Lock Manager)");
     }
@@ -2097,7 +2065,8 @@
         try {
             StringBuilder sb = new StringBuilder();
             sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
-            sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: " + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+            sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: "
+                    + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
             sb.append("\nSHRINK_TIMER_THRESHOLD (entityLockInfoManager): "
                     + entityLockInfoManager.getShrinkTimerThreshold());
             sb.append("\nSHRINK_TIMER_THRESHOLD (entityInfoManager): " + entityInfoManager.getShrinkTimerThreshold());
@@ -2233,6 +2202,31 @@
             }
         }
     }
+
+    public void batchUnlock(LogPage logPage, LogPageReader logPageReader) throws ACIDException {
+        latchLockTable();
+        try {
+            ITransactionContext txnCtx = null;
+            LogRecord logRecord = logPageReader.next();
+            while (logRecord != null) {
+                if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
+                    tempDatasetIdObj.setId(logRecord.getDatasetId());
+                    tempJobIdObj.setId(logRecord.getJobId());
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    unlock(tempDatasetIdObj, logRecord.getPKHashValue(), txnCtx);
+                    txnCtx.notifyOptracker(false);
+                } else if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+                    tempJobIdObj.setId(logRecord.getJobId());
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    txnCtx.notifyOptracker(true);
+                    ((LogPage) logPage).notifyJobCommitter();
+                }
+                logRecord = logPageReader.next();
+            }
+        } finally {
+            unlatchLockTable();
+        }
+    }
 }
 
 class ConsecutiveWakeupContext {
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
index e97c177..e61cb55 100644
--- 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
@@ -14,6 +14,7 @@
  */
 package edu.uci.ics.asterix.transaction.management.service.locking;
 
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.util.ArrayList;
@@ -21,17 +22,15 @@
 import java.util.NoSuchElementException;
 import java.util.Scanner;
 
-import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
-import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
-import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import org.apache.commons.io.FileUtils;
+
 import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -40,6 +39,13 @@
 public class LockManagerDeterministicUnitTest {
 
     public static void main(String args[]) throws ACIDException, IOException, AsterixException {
+        //prepare configuration file
+        File cwd = new File(System.getProperty("user.dir"));
+        File asterixdbDir = cwd.getParentFile();
+        File srcFile = new File(asterixdbDir.getAbsoluteFile(), "asterix-app/src/main/resources/asterix-build-configuration.xml");
+        File destFile = new File(cwd, "target/classes/asterix-configuration.xml");
+        FileUtils.copyFile(srcFile, destFile);
+
         //initialize controller thread
         String requestFileName = new String(
                 "src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockRequestFile");
@@ -61,8 +67,8 @@
     long defaultWaitTime;
 
     public LockRequestController(String requestFileName) throws ACIDException, AsterixException {
-        this.txnProvider = new TransactionSubsystem("LockManagerPredefinedUnitTest", null,
-                new AsterixTransactionProperties(new AsterixPropertiesAccessor()));
+        this.txnProvider = new TransactionSubsystem("nc1", null, new AsterixTransactionProperties(
+                new AsterixPropertiesAccessor()));
         this.workerReadyQueue = new WorkerReadyQueue();
         this.requestList = new ArrayList<LockRequest>();
         this.expectedResultList = new ArrayList<ArrayList<Integer>>();
@@ -428,9 +434,9 @@
             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);
+                if (lockRequest.txnContext.isTimeout()) {
+                    if (lockRequest.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+                        lockRequest.txnContext.setTxnState(ITransactionManager.ABORTED);
                         log("*** " + lockRequest.txnContext.getJobId() + " lock request causing deadlock ***");
                         log("Abort --> Releasing all locks acquired by " + lockRequest.txnContext.getJobId());
                         try {
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
index f73460d..e6f2798 100644
--- 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
@@ -24,7 +24,7 @@
 import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
@@ -287,8 +287,8 @@
         } else {
             try {
                 synchronized (txnContext) {
-                    if (txnContext.getTxnState() != TransactionState.ABORTED) {
-                        txnContext.setTxnState(TransactionState.ABORTED);
+                    if (txnContext.getTxnState() != ITransactionManager.ABORTED) {
+                        txnContext.setTxnState(ITransactionManager.ABORTED);
                         mayRelease = true;
                     }
                 }
@@ -446,9 +446,9 @@
                 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);
+                    if (request.txnContext.isTimeout()) {
+                        if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+                            request.txnContext.setTxnState(ITransactionManager.ABORTED);
                             log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
                             log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
                             try {
@@ -469,9 +469,9 @@
                     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);
+                    if (request.txnContext.isTimeout()) {
+                        if (request.txnContext.getTxnState() != ITransactionManager.ABORTED) {
+                            request.txnContext.setTxnState(ITransactionManager.ABORTED);
                             log("*** " + request.txnContext.getJobId() + " lock request causing deadlock ***");
                             log("Abort --> Releasing all locks acquired by " + request.txnContext.getJobId());
                             try {
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
index fc2a883..c755895 100644
--- 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
@@ -1,20 +1,11 @@
 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
+T3 L	J3 D1 E1 S
+T2 L	J2 D1 E1 X
+T0 CSQ	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
+T3 UL	J3 D1 E1 S
+T0 CST  1 2 3 -1
+T2 RL	J2 D1 E1 X
 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
+T3 END
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
deleted file mode 100644
index 272ef5e..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
-    ByteBuffer buffer;
-
-    public Buffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    protected Buffer() {
-    }
-
-    public void setBuffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    @Override
-    public byte getByte(int offset) {
-        return buffer.get(offset);
-    }
-
-    @Override
-    public byte getByte() {
-        return buffer.get();
-    }
-
-    @Override
-    public void getBytes(byte[] bytes, int offset, int size) {
-        System.arraycopy(buffer.array(), offset, bytes, 0, size);
-    }
-
-    @Override
-    public int getSize() {
-        return buffer.capacity();
-    }
-
-    @Override
-    public int readInt() {
-        return buffer.getInt();
-    }
-
-    @Override
-    public int readInt(int offset) {
-        return buffer.getInt(offset);
-    }
-
-    @Override
-    public long readLong(int offset) {
-        return buffer.getLong(offset);
-    }
-
-    @Override
-    public void put(byte b) {
-        buffer.put(b);
-    }
-
-    @Override
-    public void put(int offset, byte b) {
-        buffer.put(offset, b);
-    }
-
-    @Override
-    public void put(byte[] bytes, int start, int length) {
-        buffer.put(bytes, start, length);
-
-    }
-
-    @Override
-    public void put(byte[] bytes) {
-        buffer.put(bytes);
-    }
-
-    @Override
-    public void writeInt(int value) {
-        buffer.putInt(value);
-    }
-
-    @Override
-    public void writeInt(int offset, int value) {
-        buffer.putInt(offset, value);
-
-    }
-
-    @Override
-    public void writeLong(long value) {
-        buffer.putLong(value);
-    }
-
-    @Override
-    public void writeLong(int offset, long value) {
-        buffer.putLong(offset, value);
-
-    }
-
-    @Override
-    public byte[] getArray() {
-        return buffer.array();
-    }
-
-    @Override
-    public void erase() {
-        Arrays.fill(buffer.array(), (byte) 0);
-    }
-
-    @Override
-    public ByteBuffer getByteBuffer() {
-        return buffer;
-    }
-
-}
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
deleted file mode 100644
index 79c95c4..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.util.zip.CRC32;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * A utility class for doing bit level operations such as forming checksum or
- * converting between Integer and byte array. Used extensively during writing
- * and reading of logs.
- */
-public class DataUtil {
-
-    public static long getChecksum(IBuffer buffer, int offset, int length) {
-        CRC32 checksumEngine = new CRC32();
-        byte[] bytes = new byte[1];
-        for (int i = 0; i < length; i++) {
-            bytes[0] = buffer.getByte(offset + i);
-            checksumEngine.update(bytes, 0, 1);
-        }
-        return checksumEngine.getValue();
-    }
-
-    public static int byteArrayToInt(byte[] bytes, int offset) {
-        return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
-                + ((bytes[offset + 3] & 0xff) << 0);
-    }
-
-    public static byte[] intToByteArray(int value) {
-        byte[] bytes = new byte[4];
-        bytes[0] = (byte) ((value >>> 24) & 0xFF);
-        bytes[1] = (byte) ((value >>> 16) & 0xFF);
-        bytes[2] = (byte) ((value >>> 8) & 0xFF);
-        bytes[3] = (byte) ((value >>> 0) & 0xFF);
-        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/GenericFilter.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
deleted file mode 100644
index 0de43cc..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-
-/*
- *  A generic filter that encompasses one or more filters (see @ILogFiler) that may be applied when selectively retrieving logs.
- *  The contained filters are assumed to form a conjunction.   
- */
-public class GenericFilter implements ILogFilter {
-
-    private final List<ILogFilter> logFilters;
-
-    public GenericFilter() {
-        logFilters = new ArrayList<ILogFilter>();
-    }
-
-    public GenericFilter(List<ILogFilter> logFilters) {
-        this.logFilters = logFilters;
-    }
-
-    public boolean accept(IBuffer fileBuffer, long offset, int length) {
-        boolean satisfies = true;
-        for (ILogFilter logFilter : logFilters) {
-            satisfies = satisfies && logFilter.accept(fileBuffer, offset, length);
-            if (!satisfies) {
-                break;
-            }
-        }
-        return satisfies;
-    }
-
-    public void addFilter(ILogFilter logFilter) {
-        logFilters.add(logFilter);
-    }
-
-    public boolean removeFilter(ILogFilter logFilter) {
-        return logFilters.remove(logFilter);
-    }
-}
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
deleted file mode 100644
index 755e85b..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ICloseable;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-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(ITransactionContext context) {
-        ReusableLogContentObjectRepository txnThreadStateRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
-                .get(context.getJobId());
-        txnThreadStateRepository.remove(Thread.currentThread().getId());
-        jobId2ReusableLogContentObjectRepositoryMap.remove(context.getJobId());
-    }
-
-    public void generateLogRecord(ITransactionSubsystem txnSubsystem, ITransactionContext 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(ITransactionContext 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(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
-            throws ACIDException {
-    }
-
-    @Override
-    public void preLog(ITransactionContext 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);
-        }
-    }
-
-}
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
deleted file mode 100644
index b95b943..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ILoggerRepository;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.MutableResourceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-
-public class IndexLoggerRepository implements ILoggerRepository {
-
-    private final Map<MutableResourceId, ILogger> loggers = new HashMap<MutableResourceId, ILogger>();
-    private final ITransactionSubsystem txnSubsystem;
-    private MutableResourceId mutableResourceId;
-
-    public IndexLoggerRepository(ITransactionSubsystem provider) {
-        this.txnSubsystem = provider;
-        mutableResourceId = new MutableResourceId(0);
-    }
-
-    @Override
-    public synchronized ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException {
-        mutableResourceId.setId(resourceId);
-        ILogger logger = loggers.get(mutableResourceId);
-        if (logger == null) {
-            MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
-            IIndex index;
-            try {
-                index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                        .getIndex(resourceId);
-            } catch (HyracksDataException e) {
-                throw new ACIDException(e);
-            }
-            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
deleted file mode 100644
index 16ffa69..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-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 ITransactionSubsystem txnSubsystem;
-
-    public IndexResourceManager(byte resourceType, ITransactionSubsystem 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);
-
-        IIndex index;
-        try {
-            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(resourceId);
-        } catch (HyracksDataException e1) {
-            throw new ACIDException("Cannot undo: unable to find index");
-        }
-
-        /* 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;
-
-        try {
-            ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
-            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;
-        try {
-            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(resourceId);
-        } catch (HyracksDataException e1) {
-            throw new ACIDException("Cannot redo: unable to find index");
-        }
-
-        /* 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;
-
-        try {
-            ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
-            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/LogActionType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
deleted file mode 100644
index 5f558f3..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-public class LogActionType {
-
-    public static final byte REDO = 0; // used for a log record that contains
-    // just redo information.
-    public static final byte REDO_UNDO = 1; // used for a log record that
-    // contains both redo and undo
-    // information.
-    public static final byte UNDO = 2; // used for a log record that contains
-    // just undo information.
-    public static final byte NO_OP = 3; // used for a log record that does not
-    // require any action.
-
-}
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
deleted file mode 100644
index d3e1ad1..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.io.File;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-public class LogCursor implements ILogCursor {
-
-    private final LogManager logManager;
-    private final ILogFilter logFilter;
-    private final int logPageSize;
-    private IBuffer readOnlyBuffer;
-    private LogicalLogLocator logicalLogLocator = null;
-    private boolean needReloadBuffer = true;
-
-    public LogCursor(final LogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter,
-            int logPageSize) throws IOException, ACIDException {
-        this.logFilter = logFilter;
-        this.logManager = logManager;
-        this.logPageSize = logPageSize;
-        initialize(startingPhysicalLogLocator);
-    }
-
-    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 {
-        int fileId = (int) (lsn / logManager.getLogManagerProperties().getLogPartitionSize());
-        String filePath = LogUtil.getLogFilePath(logManager.getLogManagerProperties(), fileId);
-        File file = new File(filePath);
-        if (file.exists()) {
-            return FileUtil.getFileBasedBuffer(filePath, lsn
-                    % logManager.getLogManagerProperties().getLogPartitionSize(), size, logManager
-                    .getLogManagerProperties().getDiskSectorSize());
-        } else {
-            return null;
-        }
-    }
-
-    /**
-     * Moves the cursor to the next log record that satisfies the configured
-     * filter. The parameter nextLogLocator is set to the point to the next log
-     * record.
-     * 
-     * @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 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 (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
-            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() % logPageSize);
-            long adjustedLSN = logicalLogLocator.getLsn() - offset;
-            readOnlyBuffer = getReadOnlyBuffer(adjustedLSN, logPageSize);
-            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.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
-            integerRead = readOnlyBuffer.readInt(logicalLogLocator.getMemoryOffset());
-            if (integerRead == LogManagerProperties.LOG_MAGIC_NUMBER) {
-                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 (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
-                return next(currentLogLocator); //should read from memory if there is any further log
-            }
-        }
-
-        if (!logRecordBeginPosFound) {
-            // need to reload the buffer
-            // TODO
-            // reduce IO by reading more pages(equal to logBufferSize) at a time.
-            long lsnpos = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
-
-            readOnlyBuffer = getReadOnlyBuffer(lsnpos, logPageSize);
-            if (readOnlyBuffer != null) {
-                logicalLogLocator.setBuffer(readOnlyBuffer);
-                logicalLogLocator.setLsn(lsnpos);
-                logicalLogLocator.setMemoryOffset(0);
-                return next(currentLogLocator);
-            } else {
-                return false;
-            }
-        }
-
-        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);
-            }
-            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, currentLogLocator.getMemoryOffset(), logLength);
-    }
-
-    /**
-     * Returns the filter associated with the cursor.
-     * 
-     * @return ILogFilter
-     */
-    @Override
-    public ILogFilter getLogFilter() {
-        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 (logManager.isMemoryRead(lsn)) {
-
-                //find the magic number to identify the start of the log record
-                //----------------------------------------------------------------
-                int readNumber = -1;
-                int logMagicNumber = LogManagerProperties.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);
-                    lsn = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
-                    logicalLogLocator.setLsn(lsn);
-                    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 5b10144..4f0bb59 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
@@ -15,746 +15,181 @@
 package edu.uci.ics.asterix.transaction.management.service.logging;
 
 import java.io.File;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
 import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 
 public class LogManager implements ILogManager, ILifeCycleComponent {
 
-    public static final boolean IS_DEBUG_MODE = false;//true
+    public static final boolean IS_DEBUG_MODE = false;// true
     private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
-    private final TransactionSubsystem provider;
-    private LogManagerProperties logManagerProperties;
-    private LogPageFlushThread logPageFlusher;
+    private final TransactionSubsystem txnSubsystem;
+    private final LogManagerProperties logManagerProperties;
+    private final long logFileSize;
     private final int logPageSize;
-    private long statLogSize;
-    private long statLogCount;
+    private final int numLogPages;
+    private final String logDir;
+    private final String logFilePrefix;
+    private final MutableLong flushLSN;
+    private LinkedBlockingQueue<LogPage> emptyQ;
+    private LinkedBlockingQueue<LogPage> flushQ;
+    private long appendLSN;
+    private FileChannel appendChannel;
+    private LogPage appendPage;
+    private LogFlusher logFlusher;
+    private Future<Object> futureLogFlusher;
 
-    /*
-     * 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;
-
-    /*
-     * Number of log pages that constitute the in-memory log buffer.
-     */
-    private int numLogPages;
-
-    private AtomicLong lastFlushedLSN = new AtomicLong(-1);
-
-    /*
-     * When the transaction eco-system comes to life, the log manager positions
-     * itself to the end of the last written log. the startingLsn represent the
-     * 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;
-
-    /*
-     * lsn represents the monotonically increasing long value that can be broken
-     * down into a file id and an offset within a log file.
-     */
-    private AtomicLong lsn = new AtomicLong(0);
-
-    private List<HashMap<ITransactionContext, Integer>> activeTxnCountMaps;
-
-    public void addFlushRequest(int pageIndex, long lsn, boolean isSynchronous) {
-        logPageFlusher.requestFlush(pageIndex, lsn, isSynchronous);
-    }
-
-    public AtomicLong getLastFlushedLsn() {
-        return lastFlushedLSN;
-    }
-
-    public AtomicLong getCurrentLsn() {
-        return lsn;
-    }
-
-    public long incrementLastFlushedLsn(long delta) {
-        return lastFlushedLSN.addAndGet(delta);
-    }
-
-    public LogManager(TransactionSubsystem provider) throws ACIDException {
-        this.provider = provider;
-        logManagerProperties = new LogManagerProperties(this.provider.getTransactionProperties(), this.provider.getId());
+    public LogManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+        this.txnSubsystem = txnSubsystem;
+        logManagerProperties = new LogManagerProperties(this.txnSubsystem.getTransactionProperties(),
+                this.txnSubsystem.getId());
+        logFileSize = logManagerProperties.getLogPartitionSize();
         logPageSize = logManagerProperties.getLogPageSize();
-        initLogManager();
-        statLogSize = 0;
-        statLogCount = 0;
-    }
-
-    public LogManager(TransactionSubsystem provider, String nodeId) throws ACIDException {
-        this.provider = provider;
-        logManagerProperties = new LogManagerProperties(provider.getTransactionProperties(), nodeId);
-        logPageSize = logManagerProperties.getLogPageSize();
-        initLogManager();
-        statLogSize = 0;
-        statLogCount = 0;
-    }
-
-    private void initLogManager() throws ACIDException {
-        logRecordHelper = new LogRecordHelper(this);
         numLogPages = logManagerProperties.getNumLogPages();
-        activeTxnCountMaps = new ArrayList<HashMap<ITransactionContext, Integer>>(numLogPages);
+        logDir = logManagerProperties.getLogDir();
+        logFilePrefix = logManagerProperties.getLogFilePrefix();
+        flushLSN = new MutableLong();
+        initializeLogManager();
+    }
 
+    private void initializeLogManager() {
+        emptyQ = new LinkedBlockingQueue<LogPage>(numLogPages);
+        flushQ = new LinkedBlockingQueue<LogPage>(numLogPages);
         for (int i = 0; i < numLogPages; i++) {
-            activeTxnCountMaps.add(new HashMap<ITransactionContext, Integer>());
+            emptyQ.offer(new LogPage((LockManager) txnSubsystem.getLockManager(), logPageSize, flushLSN));
         }
-
-        logPages = new FileBasedBuffer[numLogPages];
-
-        /*
-         * place the log anchor at the end of the last log record written.
-         */
-        initLSN();
-
-        /*
-         * initialize the log pages.
-         */
-        initializeLogPages(startingLSN);
-
-        /*
-         * Instantiate and begin the LogFlusher thread. The Log Flusher thread
-         * is responsible for putting log pages to disk. It is configured as a
-         * daemon thread so that it does not stop the JVM from exiting when all
-         * other threads are done with their work.
-         */
-        logPageFlusher = new LogPageFlushThread(this);
-        logPageFlusher.setDaemon(true);
-        AsterixThreadExecutor.INSTANCE.execute(logPageFlusher);
-    }
-
-    public int getLogPageIndex(long lsnValue) {
-        return (int) (((lsnValue - startingLSN) / logPageSize) % numLogPages);
-    }
-
-    /*
-     * given a lsn, get the file id where the corresponding log record is
-     * located.
-     */
-    public int getLogFileId(long lsnValue) {
-        return (int) ((lsnValue) / logManagerProperties.getLogPartitionSize());
-    }
-
-    /*
-     * given a lsn, get the offset within a log page where the corresponding log
-     * record is (to be) placed.
-     */
-    public int getLogPageOffset(long lsnValue) {
-        return (int) (lsnValue % logPageSize);
-    }
-
-    /*
-     * The method that reserves the space for a transaction to write log record
-     * in the log buffer. Note that the method is not synchronized for
-     * performance reasons as we do not want transactions to be blocked by each
-     * other when writing log records.
-     * 
-     * @param entrySize: the requested space.
-     * 
-     * @param logType: the type of log record.
-     */
-    private long getLsn(int entrySize, byte logType) throws ACIDException {
-
-        while (true) {
-            boolean forwardPage = false;
-            long old = lsn.get();
-
-            // get the log page corresponding to the current lsn value
-            int pageIndex = getLogPageIndex(old);
-            long retVal = old;
-
-            // the lsn value for the next request if the current request is
-            // served.
-            long next = old + entrySize;
-            int prevPage = -1;
-
-            // check if the log record will cross page boundaries, a case that
-            // is not allowed.
-            if ((next - 1) / logPageSize != old / logPageSize || (next % logPageSize == 0)) {
-
-                if ((old != 0 && old % logPageSize == 0)) {
-                    // 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 {
-                    // set the lsn to point to the beginning of the next page.
-                    retVal = ((old / logPageSize) + 1) * logPageSize;
-                }
-
-                next = retVal;
-
-                // 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);
-            }
-
-            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) {
-                logPages[prevPage].acquireReadLatch();
-                // increment the counter as the transaction thread now holds a
-                // space in the log page and hence is an owner.
-                logPages[prevPage].incRefCnt();
-                logPages[prevPage].releaseReadLatch();
-
-                // forward the nextWriteOffset in the log page
-                logPages[prevPage].setBufferNextWriteOffset(logPageSize);
-
-                logPages[prevPage].decRefCnt();
-
-                addFlushRequest(prevPage, old, false);
-
-                // The transaction thread that discovers the need to forward a
-                // page is made to re-acquire a lsn.
-                continue;
-
-            } else {
-                logPages[pageIndex].acquireReadLatch();
-                // increment the counter as the transaction thread now holds a
-                // space in the log page and hence is an owner.
-                logPages[pageIndex].incRefCnt();
-                logPages[pageIndex].releaseReadLatch();
-
-                // 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) {
-                    logPages[pageIndex].decRefCnt();
-                    continue;
-                }
-            }
-
-            return retVal;
-        }
-    }
-
-    @Override
-    public void log(byte logType, ITransactionContext txnCtx, int datasetId, int PKHashValue, long resourceId,
-            byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
-            LogicalLogLocator logicalLogLocator) throws ACIDException {
-
-        HashMap<ITransactionContext, 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 = logRecordHelper.getLogRecordSize(logType, logContentSize);
-
-        // check for the total space requirement to be less than a log page.
-        if (totalLogSize > logPageSize) {
-            throw new ACIDException(
-                    " Maximum Log Content Size is "
-                            + (logPageSize - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
-                                    .getLogChecksumSize()));
-        }
-
-        // all constraints checked and we are good to go and acquire a lsn.
-        long previousLSN = -1;
-
-        // 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);
-        }
-
-        /*
-         * At this point, the transaction thread has obtained reserved space for
-         * writing the log record. In doing so, it has acquired (shared)
-         * ownership of the log page. All subsequent actions are under a try
-         * catch block so that if any exception is encountered, a clean can be
-         * performed correctly that is ownership is released.
-         */
-
-        // indicates if the transaction thread has release ownership of the
-        // page.
-        boolean decremented = false;
-
-        int pageIndex = (int) getLogPageIndex(currentLSN);
-
-        // 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);
-
-            // 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.
-            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 (logContentSize != 0) {
-                logger.preLog(txnCtx, reusableLogContentObject);
-            }
-
-            if (logContentSize != 0) {
-                // call the logger implementation and ask to fill in the log
-                // record content at the allocated space.
-                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 = logicalLogLocator.getMemoryOffset() - logRecordHelper.getLogHeaderSize(logType);
-            int length = totalLogSize - logRecordHelper.getLogChecksumSize();
-            long checksum = DataUtil.getChecksum(logPages[pageIndex], startPosChecksum, length);
-            logPages[pageIndex].writeLong(pageOffset + logRecordHelper.getLogHeaderSize(logType) + logContentSize,
-                    checksum);
-
-            // forward the nextWriteOffset in the log page
-            int bufferNextWriteOffset = (int) ((currentLSN + totalLogSize) % logPageSize);
-            if (bufferNextWriteOffset == 0) {
-                bufferNextWriteOffset = logPageSize;
-            }
-            logPages[pageIndex].setBufferNextWriteOffset(bufferNextWriteOffset);
-
-            if (logType != LogType.ENTITY_COMMIT) {
-                if (logType == LogType.COMMIT) {
-                    txnCtx.setExclusiveJobLevelCommit();
-                    map = activeTxnCountMaps.get(pageIndex);
-                    map.put(txnCtx, 1);
-                }
-                // release the ownership as the log record has been placed in
-                // created space.
-                logPages[pageIndex].decRefCnt();
-
-                // indicating that the transaction thread has released ownership
-                decremented = true;
-            }
-
-            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);
-                }
-                //------------------------------------------------------------------------------
-                // [Notice]
-                // reference count should be decremented 
-                // after activeTxnCount is incremented, but before addFlushRequest() is called. 
-                //------------------------------------------------------------------------------
-                // release the ownership as the log record has been placed in
-                // created space.
-                logPages[pageIndex].decRefCnt();
-
-                // indicating that the transaction thread has released ownership
-                decremented = true;
-
-                addFlushRequest(pageIndex, currentLSN, false);
-            } else if (logType == LogType.COMMIT) {
-
-                addFlushRequest(pageIndex, currentLSN, true);
-                if (IS_DEBUG_MODE) {
-                    System.out.println("Running sum of log size: " + statLogSize + ", log count: " + statLogCount);
-                }
-            }
-
-            if (IS_DEBUG_MODE) {
-                System.out.println("--------------> LSN(" + currentLSN + ") is written");
-            }
-
-            //collect statistics
-            statLogSize += totalLogSize;
-            statLogCount++;
-
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new ACIDException(txnCtx, "Thread: " + Thread.currentThread().getName()
-                    + " logger encountered exception", e);
-        } finally {
-            if (!decremented) {
-                logPages[pageIndex].decRefCnt();
-            }
-        }
-    }
-
-    /*
-     * This method resets the log page and is called by the log flusher thread
-     * after a page has been flushed to disk.
-     */
-    public void resetLogPage(long lsn, long nextWritePosition, int pageIndex) throws IOException {
-
-        String filePath = LogUtil.getLogFilePath(logManagerProperties, getLogFileId(lsn));
-
-        logPages[pageIndex].reset(filePath, LogUtil.getFileOffset(this, nextWritePosition), logPageSize);
-    }
-
-    @Override
-    public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
-            ACIDException {
-        LogCursor cursor = new LogCursor(this, physicalLogLocator, logFilter, logPageSize);
-        return cursor;
-    }
-
-    /*
-     * Read a log that is residing on the disk.
-     */
-    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(logPageSize);
-        RandomAccessFile raf = null;
-        FileChannel fileChannel = null;
-        try {
-            raf = new RandomAccessFile(filePath, "r");
-            fileChannel = raf.getChannel();
-            fileChannel.position(fileOffset);
-            fileChannel.read(buffer);
-            buffer.position(0);
-
-            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());
-            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) {
-            fnfe.printStackTrace();
-            throw new ACIDException(" unable to retrieve log record with lsn " + lsnValue + " from the file system",
-                    fnfe);
-        } finally {
-            try {
-                if (fileChannel != null) {
-                    fileChannel.close();
-                } else if (raf != null) {
-                    raf.close();
-                }
-            } catch (IOException ioe) {
-                ioe.printStackTrace();
-                throw new ACIDException(" exception in closing a file: " + filePath, ioe);
-            }
-        }
-    }
-
-    @Override
-    public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
-        byte[] logRecord = 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 (isMemoryRead(lsnValue)) {
-            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[logPageSize];
-
-            // take a lock on the log page so that the page is not flushed to
-            // disk interim
-            synchronized (logPages[pageIndex]) {
-
-                // need to check again (this thread may have got de-scheduled
-                // and must refresh!)
-
-                if (isMemoryRead(lsnValue)) {
-                    // get the log record length
-                    logPages[pageIndex].getBytes(pageContent, 0, pageContent.length);
-                    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, logRecordSize);
-                    MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
-                    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(logicalLogLocator)) {
-                            throw new ACIDException(" invalid log record at lsn " + lsnValue);
-                        }
-                    } catch (Exception e) {
-                        throw new ACIDException("exception encoutered in validating log record at lsn " + lsnValue, e);
-                    }
-                    return;
-                }
-            }
-        }
-
-        // the log record is residing on the disk, read it from there.
-        readDiskLog(lsnValue, logicalLogLocator);
-    }
-
-    public boolean isMemoryRead(long readLSN) {
-        long flushLSN = lastFlushedLSN.get();
-        if ((flushLSN + 1) == readLSN) {
-            return false;
-        }
-        long logPageBeginOffset = flushLSN - (flushLSN % logPageSize);
-        long logPageEndOffset = logPageBeginOffset + logPageSize;
-        if (readLSN > flushLSN || (readLSN >= logPageBeginOffset && readLSN < logPageEndOffset)) {
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    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 void initLSN() throws ACIDException {
-        PhysicalLogLocator nextPhysicalLsn = LogUtil.initializeLogAnchor(this);
-        startingLSN = nextPhysicalLsn.getLsn();
-        lastFlushedLSN.set(startingLSN - 1);
+        appendLSN = initializeLogAnchor();
+        flushLSN.set(appendLSN);
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" Starting lsn is : " + startingLSN);
+            LOGGER.info("LogManager starts logging in LSN: " + appendLSN);
         }
-        lsn.set(startingLSN);
-    }
-
-    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 * logPageSize, logPageSize);
-            }
-        } catch (Exception e) {
-            throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
-        }
+        appendChannel = getFileChannel(appendLSN, false);
+        getAndInitNewPage();
+        logFlusher = new LogFlusher(this, emptyQ, flushQ);
+        futureLogFlusher = AsterixThreadExecutor.INSTANCE.submit(logFlusher);
     }
 
     @Override
-    public ILogRecordHelper getLogRecordHelper() {
-        return logRecordHelper;
-    }
-
-    /*
-     * Map each log page to cover a physical byte range over a log file. When a
-     * page is flushed, the page contents are put to disk in the corresponding
-     * byte range.
-     */
-    private void initializeLogPages(long beginLsn) throws ACIDException {
-        try {
-            String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, beginLsn));
-            long nextDiskWriteOffset = LogUtil.getFileOffset(this, beginLsn);
-            long nextBufferWriteOffset = nextDiskWriteOffset % logPageSize;
-            long bufferBeginOffset = nextDiskWriteOffset - nextBufferWriteOffset;
-
-            for (int i = 0; i < numLogPages; i++) {
-                logPages[i] = FileUtil.getFileBasedBuffer(filePath, bufferBeginOffset + i * logPageSize, logPageSize,
-                        logManagerProperties.getDiskSectorSize());
-                if (i == 0) {
-                    logPages[i].setBufferLastFlushOffset((int) nextBufferWriteOffset);
-                    logPages[i].setBufferNextWriteOffset((int) nextBufferWriteOffset);
-                    logPages[i].setDiskNextWriteOffset(nextDiskWriteOffset);
+    public void log(ILogRecord logRecord) {
+        if (logRecord.getLogSize() > logPageSize) {
+            throw new IllegalStateException();
+        }
+        syncLog(logRecord);
+        if (logRecord.getLogType() == LogType.JOB_COMMIT && !logRecord.isFlushed()) {
+            synchronized (logRecord) {
+                while (!logRecord.isFlushed()) {
+                    try {
+                        logRecord.wait();
+                    } catch (InterruptedException e) {
+                        //ignore
+                    }
                 }
             }
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
         }
     }
 
-    /*
-     * Pages are sequenced starting with 0 going upto numLogPages-1.
-     */
-    public int getNextPageInSequence(int pageNo) {
-        return (pageNo + 1) % numLogPages;
+    private synchronized void syncLog(ILogRecord logRecord) {
+        ITransactionContext txnCtx = logRecord.getTxnCtx();
+        if (getLogFileOffset(appendLSN) + logRecord.getLogSize() > logFileSize) {
+            prepareNextLogFile();
+            appendPage.isFull(true);
+            getAndInitNewPage();
+        } else if (!appendPage.hasSpace(logRecord.getLogSize(), getLogFileOffset(appendLSN))) {
+            appendPage.isFull(true);
+            getAndInitNewPage();
+        }
+        if (logRecord.getLogType() == LogType.UPDATE) {
+            logRecord.setPrevLSN(txnCtx.getLastLSN());
+        }
+        appendPage.append(logRecord, appendLSN);
+        appendLSN += logRecord.getLogSize();
     }
 
-    public int getPreviousPageInSequence(int pageNo) {
-        return pageNo == 0 ? numLogPages - 1 : pageNo - 1;
+    private void getAndInitNewPage() {
+        appendPage = null;
+        while (appendPage == null) {
+            try {
+                appendPage = emptyQ.take();
+            } catch (InterruptedException e) {
+                //ignore
+            }
+        }
+        appendPage.reset();
+        appendPage.setFileChannel(appendChannel);
+        appendPage.setInitialFlushOffset(getLogFileOffset(appendLSN));
+        flushQ.offer(appendPage);
+    }
+
+    private void prepareNextLogFile() {
+        appendLSN += logFileSize - getLogFileOffset(appendLSN);
+        appendChannel = getFileChannel(appendLSN, true);
+        appendPage.isLastPage(true);
+        //[Notice]
+        //the current log file channel is closed if 
+        //LogPage.flush() completely flush the last page of the file.
     }
 
     @Override
+    public ILogReader getLogReader(boolean isRecoveryMode) {
+        return new LogReader(this, logFileSize, logPageSize, flushLSN, isRecoveryMode);
+    }
+
     public LogManagerProperties getLogManagerProperties() {
         return logManagerProperties;
     }
 
-    public IFileBasedBuffer getLogPage(int pageIndex) {
-        return logPages[pageIndex];
-    }
-
-    public IFileBasedBuffer[] getLogPages() {
-        return logPages;
-    }
-
-    @Override
     public TransactionSubsystem getTransactionSubsystem() {
-        return provider;
+        return txnSubsystem;
     }
 
-    static AtomicInteger t = new AtomicInteger();
-
-    public void decrementActiveTxnCountOnIndexes(int pageIndex) throws HyracksDataException {
-        ITransactionContext ctx = null;
-        int count = 0;
-        int i = 0;
-
-        HashMap<ITransactionContext, Integer> map = activeTxnCountMaps.get(pageIndex);
-        Set<Map.Entry<ITransactionContext, Integer>> entrySet = map.entrySet();
-        if (entrySet != null) {
-            for (Map.Entry<ITransactionContext, 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();
+    public long getAppendLSN() {
+        return appendLSN;
     }
 
     @Override
     public void start() {
-        //no op
+        // no op
     }
 
     @Override
     public void stop(boolean dumpState, OutputStream os) {
+        terminateLogFlusher();
         if (dumpState) {
-            //#. dump Configurable Variables
+            // #. dump Configurable Variables
             dumpConfVars(os);
 
-            //#. dump LSNInfo
+            // #. dump LSNInfo
             dumpLSNInfo(os);
 
             try {
                 os.flush();
             } catch (IOException e) {
-                //ignore
+                // ignore
             }
         }
     }
@@ -767,7 +202,7 @@
             sb.append("\n>>dump_end\t>>----- [ConfVars] -----\n");
             os.write(sb.toString().getBytes());
         } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
+            // ignore exception and continue dumping as much as possible.
             if (IS_DEBUG_MODE) {
                 e.printStackTrace();
             }
@@ -778,191 +213,243 @@
         try {
             StringBuilder sb = new StringBuilder();
             sb.append("\n>>dump_begin\t>>----- [LSNInfo] -----");
-            sb.append("\nstartingLSN: " + startingLSN);
-            sb.append("\ncurrentLSN: " + lsn.get());
-            sb.append("\nlastFlushedLSN: " + lastFlushedLSN.get());
+            sb.append("\nappendLsn: " + appendLSN);
+            sb.append("\nflushLsn: " + flushLSN.get());
             sb.append("\n>>dump_end\t>>----- [LSNInfo] -----\n");
             os.write(sb.toString().getBytes());
         } catch (Exception e) {
-            //ignore exception and continue dumping as much as possible.
+            // ignore exception and continue dumping as much as possible.
             if (IS_DEBUG_MODE) {
                 e.printStackTrace();
             }
         }
     }
+
+    public MutableLong getFlushLSN() {
+        return flushLSN;
+    }
+
+    private long initializeLogAnchor() {
+        long fileId = 0;
+        long offset = 0;
+        File fileLogDir = new File(logDir);
+        try {
+            if (fileLogDir.exists()) {
+                List<Long> logFileIds = getLogFileIds();
+                if (logFileIds == null) {
+                    createFileIfNotExists(getLogFilePath(0));
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("created a log file: " + getLogFilePath(0));
+                    }
+                } else {
+                    fileId = logFileIds.get(logFileIds.size() - 1);
+                    File logFile = new File(getLogFilePath(fileId));
+                    offset = logFile.length();
+                }
+            } else {
+                createNewDirectory(logDir);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
+                }
+                createFileIfNotExists(getLogFilePath(0));
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("created a log file: " + getLogFilePath(0));
+                }
+            }
+        } catch (IOException ioe) {
+            throw new IllegalStateException("Failed to initialize the log anchor", ioe);
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("log file Id: " + fileId + ", offset: " + offset);
+        }
+        return logFileSize * fileId + offset;
+    }
+
+    public void renewLogFiles() {
+        terminateLogFlusher();
+        deleteAllLogFiles();
+        initializeLogManager();
+    }
+
+    private void terminateLogFlusher() {
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Terminating LogFlusher thread ...");
+        }
+        logFlusher.terminate();
+        try {
+            futureLogFlusher.get();
+        } catch (ExecutionException | InterruptedException e) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("---------- warning(begin): LogFlusher thread is terminated abnormally --------");
+                e.printStackTrace();
+                LOGGER.info("---------- warning(end)  : LogFlusher thread is terminated abnormally --------");
+            }
+        }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("LogFlusher thread is terminated.");
+        }
+    }
+
+    private void deleteAllLogFiles() {
+        if (appendChannel != null) {
+            try {
+                appendChannel.close();
+            } catch (IOException e) {
+                throw new IllegalStateException("Failed to close a fileChannel of a log file");
+            }
+        }
+        List<Long> logFileIds = getLogFileIds();
+        for (Long id : logFileIds) {
+            File file = new File(getLogFilePath(id));
+            if (!file.delete()) {
+                throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
+            }
+        }
+    }
+
+    private List<Long> getLogFileIds() {
+        File fileLogDir = new File(logDir);
+        String[] logFileNames = null;
+        List<Long> logFileIds = null;
+        if (fileLogDir.exists()) {
+            logFileNames = fileLogDir.list(new FilenameFilter() {
+                public boolean accept(File dir, String name) {
+                    if (name.startsWith(logFilePrefix)) {
+                        return true;
+                    }
+                    return false;
+                }
+            });
+            if (logFileNames != null && logFileNames.length != 0) {
+                logFileIds = new ArrayList<Long>();
+                for (String fileName : logFileNames) {
+                    logFileIds.add(Long.parseLong(fileName.substring(logFilePrefix.length() + 1)));
+                }
+                Collections.sort(logFileIds, new Comparator<Long>() {
+                    @Override
+                    public int compare(Long arg0, Long arg1) {
+                        return arg0.compareTo(arg1);
+                    }
+                });
+            }
+        }
+        return logFileIds;
+    }
+
+    public String getLogFilePath(long fileId) {
+        return logDir + File.separator + logFilePrefix + "_" + fileId;
+    }
+
+    public long getLogFileOffset(long lsn) {
+        return lsn % logFileSize;
+    }
+
+    public long getLogFileId(long lsn) {
+        return lsn / logFileSize;
+    }
+
+    private boolean createFileIfNotExists(String path) throws IOException {
+        File file = new File(path);
+        File parentFile = file.getParentFile();
+        if (parentFile != null) {
+            parentFile.mkdirs();
+        }
+        return file.createNewFile();
+    }
+
+    private boolean createNewDirectory(String path) throws IOException {
+        return (new File(path)).mkdir();
+    }
+
+    public FileChannel getFileChannel(long lsn, boolean create) {
+        FileChannel newFileChannel = null;
+        try {
+            long fileId = getLogFileId(lsn);
+            String logFilePath = getLogFilePath(fileId);
+            File file = new File(logFilePath);
+            if (create) {
+                if (!file.createNewFile()) {
+                    throw new IllegalStateException();
+                }
+            } else {
+                if (!file.exists()) {
+                    throw new IllegalStateException();
+                }
+            }
+            RandomAccessFile raf = new RandomAccessFile(new File(logFilePath), "rw");
+            newFileChannel = raf.getChannel();
+            newFileChannel.position(getLogFileOffset(lsn));
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        return newFileChannel;
+    }
 }
 
-/*
- * The thread responsible for putting log pages to disk in an ordered manner.
- * The Log Flusher updates the bookkeeping data internal to the log manager and
- * acquires appropriate locks. It also acquires finer level locks on the log
- * page when it is in process of flushing the content to disk.
- */
-class LogPageFlushThread extends Thread {
+class LogFlusher implements Callable<Boolean> {
+    private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_COMMIT_LOG_SIZE, null);
+    private final LogManager logMgr;//for debugging
+    private final LinkedBlockingQueue<LogPage> emptyQ;
+    private final LinkedBlockingQueue<LogPage> flushQ;
+    private LogPage flushPage;
+    private final AtomicBoolean isStarted;
+    private final AtomicBoolean terminateFlag;
 
-    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 flushPageIndex;
-    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.flushPageIndex = 0;
-        groupCommitWaitPeriod = logManager.getLogManagerProperties().getGroupCommitWaitPeriod();
-        isRenewRequest = false;
+    public LogFlusher(LogManager logMgr, LinkedBlockingQueue<LogPage> emptyQ, LinkedBlockingQueue<LogPage> flushQ) {
+        this.logMgr = logMgr;
+        this.emptyQ = emptyQ;
+        this.flushQ = flushQ;
+        flushPage = null;
+        isStarted = new AtomicBoolean(false);
+        terminateFlag = new AtomicBoolean(false);
+        
     }
 
-    public void renew() {
-        isRenewRequest = true;
-        flushPageIndex = 0;
-        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) {
+    public void terminate() {
+        //make sure the LogFlusher thread started before terminating it.
+        synchronized (isStarted) {
+            while(!isStarted.get()) {
                 try {
-                    logManager.getLogPage(pageIndex).wait();
-                    isNotified = true;
+                    isStarted.wait();
                 } catch (InterruptedException e) {
-                    e.printStackTrace();
+                    //ignore
                 }
             }
         }
+        
+        terminateFlag.set(true);
+        if (flushPage != null) {
+            synchronized (flushPage) {
+                flushPage.isStop(true);
+                flushPage.notify();
+            }
+        }
+        //[Notice]
+        //The return value doesn't need to be checked
+        //since terminateFlag will trigger termination if the flushQ is full.
+        flushQ.offer(POISON_PILL);
     }
 
     @Override
-    public void run() {
-        int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
-        int logBufferSize = logManager.getLogManagerProperties().getLogBufferSize();
-        int beforeFlushOffset = 0;
-        int afterFlushOffset = 0;
-        boolean resetFlushPageIndex = false;
-
+    public Boolean call() {
+        synchronized(isStarted) {
+            isStarted.set(true);
+            isStarted.notify();
+        }
         while (true) {
+            flushPage = null;
             try {
-                // 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[flushPageIndex].take();
-                } catch (InterruptedException ie) {
-                    while (isRenewRequest) {
-                        sleep(1);
-                    }
+                flushPage = flushQ.take();
+                if (flushPage == POISON_PILL || terminateFlag.get()) {
+                    return true;
+                }
+            } catch (InterruptedException e) {
+                if (flushPage == null) {
                     continue;
                 }
-
-                //if the log page is already full, don't wait. 
-                if (logManager.getLogPage(flushPageIndex).getBufferNextWriteOffset() < logPageSize
-                        - logManager.getLogRecordHelper().getCommitLogSize()) {
-                    // #. sleep for the groupCommitWaitTime
-                    sleep(groupCommitWaitPeriod);
-                }
-
-                synchronized (logManager.getLogPage(flushPageIndex)) {
-                    logManager.getLogPage(flushPageIndex).acquireWriteLatch();
-                    try {
-
-                        // #. need to wait until the reference count reaches 0
-                        while (logManager.getLogPage(flushPageIndex).getRefCnt() != 0) {
-                            sleep(0);
-                        }
-
-                        beforeFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
-                        // put the content to disk (the thread still has a lock on the log page)
-                        logManager.getLogPage(flushPageIndex).flush();
-
-                        afterFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
-                        // increment the last flushed lsn
-                        logManager.incrementLastFlushedLsn(afterFlushOffset - beforeFlushOffset);
-
-                        // increment currentLSN if currentLSN is less than flushLSN.
-                        if (logManager.getLastFlushedLsn().get() + 1 > logManager.getCurrentLsn().get()) {
-                            logManager.getCurrentLsn().set(logManager.getLastFlushedLsn().get() + 1);
-                        }
-
-                        // Map the log page to a new region in the log file if the flushOffset reached the logPageSize
-                        if (afterFlushOffset == logPageSize) {
-                            long diskNextWriteOffset = logManager.getLogPages()[flushPageIndex]
-                                    .getDiskNextWriteOffset() + logBufferSize;
-                            logManager.resetLogPage(logManager.getLastFlushedLsn().get() + 1 + logBufferSize,
-                                    diskNextWriteOffset, flushPageIndex);
-                            resetFlushPageIndex = true;
-                        }
-
-                        // decrement activeTxnCountOnIndexes
-                        logManager.decrementActiveTxnCountOnIndexes(flushPageIndex);
-
-                    } finally {
-                        logManager.getLogPage(flushPageIndex).releaseWriteLatch();
-                    }
-
-                    // #. 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[flushPageIndex].peek() != null) {
-                        flushRequestQueue[flushPageIndex].take();
-                    }
-
-                    // notify all waiting (transaction) threads.
-                    logManager.getLogPage(flushPageIndex).notifyAll();
-
-                    if (resetFlushPageIndex) {
-                        flushPageIndex = logManager.getNextPageInSequence(flushPageIndex);
-                        resetFlushPageIndex = false;
-                    }
-                }
-            } catch (IOException ioe) {
-                ioe.printStackTrace();
-                throw new Error(" exception in flushing log page", ioe);
-            } catch (InterruptedException e) {
-                e.printStackTrace();
-                break;
             }
+            flushPage.flush();
+            emptyQ.offer(flushPage);
         }
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
new file mode 100644
index 0000000..edfec69
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
@@ -0,0 +1,217 @@
+/*
+ * 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.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogPage;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
+
+public class LogPage implements ILogPage {
+
+    public static final boolean IS_DEBUG_MODE = false;//true
+    private static final Logger LOGGER = Logger.getLogger(LogPage.class.getName());
+    private final LockManager lockMgr;
+    private final LogPageReader logPageReader;
+    private final int logPageSize;
+    private final MutableLong flushLSN;
+    private final AtomicBoolean full;
+    private int appendOffset;
+    private int flushOffset;
+    private final ByteBuffer appendBuffer;
+    private final ByteBuffer flushBuffer;
+    private final ByteBuffer unlockBuffer;
+    private boolean isLastPage;
+    private final LinkedBlockingQueue<ILogRecord> syncCommitQ;
+    private FileChannel fileChannel;
+    private boolean stop;
+
+    public LogPage(LockManager lockMgr, int logPageSize, MutableLong flushLSN) {
+        this.lockMgr = lockMgr;
+        this.logPageSize = logPageSize;
+        this.flushLSN = flushLSN;
+        appendBuffer = ByteBuffer.allocate(logPageSize);
+        flushBuffer = appendBuffer.duplicate();
+        unlockBuffer = appendBuffer.duplicate();
+        logPageReader = getLogPageReader();
+        full = new AtomicBoolean(false);
+        appendOffset = 0;
+        flushOffset = 0;
+        isLastPage = false;
+        syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_COMMIT_LOG_SIZE);
+    }
+
+    ////////////////////////////////////
+    // LogAppender Methods
+    ////////////////////////////////////
+
+    @Override
+    public void append(ILogRecord logRecord, long appendLSN) {
+        logRecord.writeLogRecord(appendBuffer);
+        if (logRecord.getLogType() == LogType.UPDATE) {
+            logRecord.getTxnCtx().setLastLSN(logRecord.getResourceId(), appendLSN);
+        }
+        synchronized (this) {
+            appendOffset += logRecord.getLogSize();
+            if (IS_DEBUG_MODE) {
+                LOGGER.info("append()| appendOffset: " + appendOffset);
+            }
+            if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+                logRecord.isFlushed(false);
+                syncCommitQ.offer(logRecord);
+            }
+            this.notify();
+        }
+    }
+
+    public void setFileChannel(FileChannel fileChannel) {
+        this.fileChannel = fileChannel;
+    }
+
+    public void setInitialFlushOffset(long offset) {
+        try {
+            fileChannel.position(offset);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    public synchronized void isFull(boolean full) {
+        this.full.set(full);
+        this.notify();
+    }
+
+    public void isLastPage(boolean isLastPage) {
+        this.isLastPage = isLastPage;
+    }
+
+    public boolean hasSpace(int logSize, long logFileOffset) {
+        return appendOffset + logSize <= logPageSize;
+    }
+
+    public void reset() {
+        appendBuffer.position(0);
+        appendBuffer.limit(logPageSize);
+        flushBuffer.position(0);
+        flushBuffer.limit(logPageSize);
+        unlockBuffer.position(0);
+        unlockBuffer.limit(logPageSize);
+        full.set(false);
+        appendOffset = 0;
+        flushOffset = 0;
+        isLastPage = false;
+    }
+
+    ////////////////////////////////////
+    // LogFlusher Methods
+    ////////////////////////////////////
+
+    @Override
+    public void flush() {
+        try {
+            int endOffset;
+            while (!full.get()) {
+                synchronized (this) {
+                    if (appendOffset - flushOffset == 0 && !full.get()) {
+                        try {
+                            if (IS_DEBUG_MODE) {
+                                LOGGER.info("flush()| appendOffset: " + appendOffset + ", flushOffset: " + flushOffset
+                                        + ", full: " + full.get());
+                            }
+                            if (stop) {
+                                fileChannel.close();
+                                break;
+                            }
+                            this.wait();
+                        } catch (InterruptedException e) {
+                            continue;
+                        }
+                    }
+                    endOffset = appendOffset;
+                }
+                internalFlush(flushOffset, endOffset);
+            }
+            internalFlush(flushOffset, appendOffset);
+            if (isLastPage) {
+                fileChannel.close();
+            }
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    private void internalFlush(int beginOffset, int endOffset) {
+        try {
+            if (endOffset > beginOffset) {
+                flushBuffer.limit(endOffset);
+                fileChannel.write(flushBuffer);
+                fileChannel.force(false);
+                flushOffset = endOffset;
+                synchronized (flushLSN) {
+                    flushLSN.set(flushLSN.get() + (endOffset - beginOffset));
+                    flushLSN.notifyAll(); //notify to LogReaders if any
+                }
+                if (IS_DEBUG_MODE) {
+                    LOGGER.info("internalFlush()| flushOffset: " + flushOffset + ", flushLSN: " + flushLSN.get());
+                }
+                batchUnlock(beginOffset, endOffset);
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    private LogPageReader getLogPageReader() {
+        return new LogPageReader(unlockBuffer);
+    }
+
+    private void batchUnlock(int beginOffset, int endOffset) throws ACIDException {
+        if (endOffset > beginOffset) {
+            logPageReader.initializeScan(beginOffset, endOffset);
+            lockMgr.batchUnlock(this, logPageReader);
+        }
+    }
+
+    public void notifyJobCommitter() {
+        ILogRecord logRecord = null;
+        while (logRecord == null) {
+            try {
+                logRecord = syncCommitQ.take();
+            } catch (InterruptedException e) {
+                //ignore
+            }
+        }
+        synchronized (logRecord) {
+            logRecord.isFlushed(true);
+            logRecord.notifyAll();
+        }
+    }
+
+    public boolean isStop() {
+        return stop;
+    }
+
+    public void isStop(boolean stop) {
+        this.stop = stop;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
new file mode 100644
index 0000000..9e54abc
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
@@ -0,0 +1,44 @@
+/*
+ * 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.transaction.management.service.logging;
+
+import java.nio.ByteBuffer;
+
+public class LogPageReader {
+
+    private final ByteBuffer buffer;
+    private final LogRecord logRecord;
+    private int endOffset;
+
+    public LogPageReader(ByteBuffer buffer) {
+        this.buffer = buffer;
+        logRecord = new LogRecord();
+    }
+
+    public void initializeScan(int beginOffset, int endOffset) {
+        this.endOffset = endOffset;
+        buffer.position(beginOffset);
+    }
+
+    public LogRecord next() {
+        if (buffer.position() == endOffset) {
+            return null;
+        }
+        if (!logRecord.readLogRecord(buffer)) {
+            throw new IllegalStateException();
+        }
+        return logRecord;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
new file mode 100644
index 0000000..9dc966c
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
@@ -0,0 +1,188 @@
+/*
+ * 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.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+
+public class LogReader implements ILogReader {
+
+    public static final boolean IS_DEBUG_MODE = false;//true
+    private static final Logger LOGGER = Logger.getLogger(LogReader.class.getName());
+    private final LogManager logMgr;
+    private final long logFileSize;
+    private final int logPageSize;
+    private final MutableLong flushLSN;
+    private final boolean isRecoveryMode;
+    private final ByteBuffer readBuffer;
+    private final ILogRecord logRecord;
+    private long readLSN;
+    private long bufferBeginLSN;
+    private long fileBeginLSN;
+    private FileChannel fileChannel;
+    
+    private enum ReturnState {
+        FLUSH,
+        EOF
+    };
+
+    public LogReader(LogManager logMgr, long logFileSize, int logPageSize, MutableLong flushLSN, boolean isRecoveryMode) {
+        this.logMgr = logMgr;
+        this.logFileSize = logFileSize;
+        this.logPageSize = logPageSize;
+        this.flushLSN = flushLSN;
+        this.isRecoveryMode = isRecoveryMode;
+        this.readBuffer = ByteBuffer.allocate(logPageSize);
+        this.logRecord = new LogRecord();
+    }
+
+    @Override
+    public void initializeScan(long beginLSN) throws ACIDException {
+        readLSN = beginLSN;
+        if (waitForFlushOrReturnIfEOF() == ReturnState.EOF) {
+            return;
+        }
+        getFileChannel();
+        readPage();
+    }
+    
+    //for scanning
+    @Override
+    public ILogRecord next() throws ACIDException {
+        if (waitForFlushOrReturnIfEOF() == ReturnState.EOF) {
+            return null;
+        }
+        if (readBuffer.position() == readBuffer.limit() || !logRecord.readLogRecord(readBuffer)) {
+            readNextPage();
+            if (!logRecord.readLogRecord(readBuffer)) {
+                throw new IllegalStateException();
+            }
+        }
+        logRecord.setLSN(readLSN);
+        readLSN += logRecord.getLogSize();
+        return logRecord;
+    }
+    
+    private ReturnState waitForFlushOrReturnIfEOF() {
+        synchronized (flushLSN) {
+            while (readLSN >= flushLSN.get()) {
+                if (isRecoveryMode) {
+                    return ReturnState.EOF;
+                }
+                try {
+                    if (IS_DEBUG_MODE) {
+                        LOGGER.info("waitForFlushOrReturnIfEOF()| flushLSN: " + flushLSN.get() + ", readLSN: "
+                                + readLSN);
+                    }
+                    flushLSN.wait();
+                } catch (InterruptedException e) {
+                    //ignore
+                }
+            }
+            return ReturnState.FLUSH;
+        }
+    }
+
+    private void readNextPage() throws ACIDException {
+        try {
+            if (readLSN % logFileSize == fileChannel.size()) {
+                fileChannel.close();
+                readLSN += logFileSize - (readLSN % logFileSize);
+                getFileChannel();
+            }
+            readPage();
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+    }
+
+    private void readPage() throws ACIDException {
+        int size;
+        readBuffer.position(0);
+        readBuffer.limit(logPageSize);
+        try {
+            fileChannel.position(readLSN % logFileSize);
+            size = fileChannel.read(readBuffer);
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+        readBuffer.position(0);
+        readBuffer.limit(size);
+        bufferBeginLSN = readLSN;
+    }
+
+    //for random reading
+    @Override
+    public ILogRecord read(long LSN) throws ACIDException {
+        readLSN = LSN;
+        synchronized (flushLSN) {
+            while (readLSN >= flushLSN.get()) {
+                try {
+                    flushLSN.wait();
+                } catch (InterruptedException e) {
+                    //ignore
+                }
+            }
+        }
+        try {
+            if (fileChannel == null) {
+                getFileChannel();
+                readPage();
+            } else if (readLSN < fileBeginLSN || readLSN >= fileBeginLSN + fileChannel.size()) {
+                fileChannel.close();
+                getFileChannel();
+                readPage();
+            } else if (readLSN < bufferBeginLSN || readLSN >= bufferBeginLSN + readBuffer.limit()) {
+                readPage();
+            } else {
+                readBuffer.position((int) (readLSN - bufferBeginLSN));
+            }
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+        if (!logRecord.readLogRecord(readBuffer)) {
+            readNextPage();
+            if (!logRecord.readLogRecord(readBuffer)) {
+                throw new IllegalStateException();
+            }
+        }
+        logRecord.setLSN(readLSN);
+        readLSN += logRecord.getLogSize();
+        return logRecord;
+    }
+
+    private void getFileChannel() throws ACIDException {
+        fileChannel = logMgr.getFileChannel(readLSN, false);
+        fileBeginLSN = readLSN;
+    }
+
+    @Override
+    public void close() throws ACIDException {
+        try {
+            if (fileChannel != null) {
+                fileChannel.close();
+            }
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
new file mode 100644
index 0000000..4b0e1f2
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
@@ -0,0 +1,520 @@
+/*
+ * 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.transaction.management.service.logging;
+
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.zip.CRC32;
+
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+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;
+
+/*
+ * == LogRecordFormat ==
+ * ---------------------------
+ * [Header1] (5 bytes) : for all log types
+ * LogType(1)
+ * JobId(4)
+ * ---------------------------
+ * [Header2] (16 bytes + PKValueSize) : for entity_commit and update log types 
+ * DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * PKHashValue(4)
+ * PKFieldCnt(4)
+ * PKValueSize(4)
+ * PKValue(PKValueSize)
+ * ---------------------------
+ * [Header3] (21 bytes) : only for update log type
+ * PrevLSN(8)
+ * ResourceId(8) //stored in .metadata of the corresponding index in NC node
+ * ResourceType(1)
+ * LogRecordSize(4)
+ * ---------------------------
+ * [Body] (Variable size) : only for update log type
+ * FieldCnt(4)
+ * NewOp(1)
+ * NewValueSize(4)
+ * NewValue(NewValueSize)
+ * OldOp(1)
+ * OldValueSize(4)
+ * OldValue(OldValueSize)
+ * ---------------------------
+ * [Tail] (8 bytes) : for all log types
+ * Checksum(8)
+ * ---------------------------
+ * = LogSize =
+ * 1) JOB_COMMIT_LOG_SIZE: 13 bytes (5 + 8)
+ * 2) ENTITY_COMMIT: 29 + PKSize (5 + 16 + PKSize + 8)
+ *    --> ENTITY_COMMIT_LOG_BASE_SIZE = 29
+ * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 16 + PKSize + 21 + 14 + New/OldValueSize + 8)
+ *    --> UPDATE_LOG_BASE_SIZE = 64
+ */
+public class LogRecord implements ILogRecord {
+
+    //------------- fields in a log record (begin) ------------//
+    private byte logType;
+    private int jobId;
+    private int datasetId;
+    private int PKHashValue;
+    private int PKFieldCnt;
+    private int PKValueSize;
+    private ITupleReference PKValue;
+    private long prevLSN;
+    private long resourceId;
+    private byte resourceType;
+    private int logSize;
+    private int fieldCnt;
+    private byte newOp;
+    private int newValueSize;
+    private ITupleReference newValue;
+    private byte oldOp;
+    private int oldValueSize;
+    private ITupleReference oldValue;
+    private long checksum;
+    //------------- fields in a log record (end) --------------//
+
+    private static final int CHECKSUM_SIZE = 8;
+    private ITransactionContext txnCtx;
+    private long LSN;
+    private final AtomicBoolean isFlushed;
+    private final SimpleTupleWriter tupleWriter;
+    private final SimpleTupleReference readPKValue;
+    private final SimpleTupleReference readNewValue;
+    private final SimpleTupleReference readOldValue;
+    private final CRC32 checksumGen;
+    private int[] PKFields;
+
+    public LogRecord() {
+        isFlushed = new AtomicBoolean(false);
+        tupleWriter = new SimpleTupleWriter();
+        readPKValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+        readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+        readOldValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+        checksumGen = new CRC32();
+    }
+
+    @Override
+    public void writeLogRecord(ByteBuffer buffer) {
+        int beginOffset = buffer.position();
+        buffer.put(logType);
+        buffer.putInt(jobId);
+        if (logType != LogType.JOB_COMMIT) {
+            buffer.putInt(datasetId);
+            buffer.putInt(PKHashValue);
+            buffer.putInt(PKFieldCnt);
+            if (PKValueSize <= 0) {
+                throw new IllegalStateException("Primary Key Size is less than or equal to 0");
+            }
+            buffer.putInt(PKValueSize);
+            writePKValue(buffer);
+        }
+        if (logType == LogType.UPDATE) {
+            buffer.putLong(prevLSN);
+            buffer.putLong(resourceId);
+            buffer.put(resourceType);
+            buffer.putInt(logSize);
+            buffer.putInt(fieldCnt);
+            buffer.put(newOp);
+            buffer.putInt(newValueSize);
+            writeTuple(buffer, newValue, newValueSize);
+            if (resourceType == ResourceType.LSM_BTREE) {
+                buffer.put(oldOp);
+                if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+                    buffer.putInt(oldValueSize);
+                    if (oldValueSize > 0) {
+                        writeTuple(buffer, oldValue, oldValueSize);
+                    }
+                }
+            }
+        }
+        checksum = generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE);
+        buffer.putLong(checksum);
+    }
+
+    private void writePKValue(ByteBuffer buffer) {
+        int i;
+        for (i = 0; i < PKFieldCnt; i++) {
+            buffer.put(PKValue.getFieldData(0), PKValue.getFieldStart(PKFields[i]), PKValue.getFieldLength(PKFields[i]));
+        }
+    }
+
+    private void writeTuple(ByteBuffer buffer, ITupleReference tuple, int size) {
+        tupleWriter.writeTuple(tuple, buffer.array(), buffer.position());
+        //writeTuple() doesn't change the position of the buffer. 
+        buffer.position(buffer.position() + size);
+    }
+
+    private long generateChecksum(ByteBuffer buffer, int offset, int len) {
+        checksumGen.reset();
+        checksumGen.update(buffer.array(), offset, len);
+        return checksumGen.getValue();
+    }
+
+    @Override
+    public boolean readLogRecord(ByteBuffer buffer) {
+        int beginOffset = buffer.position();
+        try {
+            logType = buffer.get();
+            jobId = buffer.getInt();
+            if (logType == LogType.JOB_COMMIT) {
+                datasetId = -1;
+                PKHashValue = -1;
+            } else {
+                datasetId = buffer.getInt();    
+                PKHashValue = buffer.getInt();
+                PKFieldCnt = buffer.getInt();
+                PKValueSize = buffer.getInt();
+                if (PKValueSize <= 0) {
+                    throw new IllegalStateException("Primary Key Size is less than or equal to 0");
+                }
+                PKValue = readPKValue(buffer);
+            }
+            if (logType == LogType.UPDATE) {
+                prevLSN = buffer.getLong();
+                resourceId = buffer.getLong();
+                resourceType = buffer.get();
+                logSize = buffer.getInt();
+                fieldCnt = buffer.getInt();
+                newOp = buffer.get();
+                newValueSize = buffer.getInt();
+                newValue = readTuple(buffer, readNewValue, fieldCnt, newValueSize);
+                if (resourceType == ResourceType.LSM_BTREE) {
+                    oldOp = buffer.get();
+                    if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+                        oldValueSize = buffer.getInt();
+                        if (oldValueSize > 0) {
+                            oldValue = readTuple(buffer, readOldValue, fieldCnt, oldValueSize);
+                        }
+                    }
+                }
+            } else {
+                computeAndSetLogSize();
+            }
+            checksum = buffer.getLong();
+            if (checksum != generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE)) {
+                throw new IllegalStateException();
+            }
+        } catch (BufferUnderflowException e) {
+            buffer.position(beginOffset);
+            return false;
+        }
+        return true;
+    }
+    
+    private ITupleReference readPKValue(ByteBuffer buffer) {
+        return readTuple(buffer, readPKValue, PKFieldCnt, PKValueSize);
+    }
+
+    private ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt, int size) {
+        destTuple.setFieldCount(fieldCnt);
+        destTuple.resetByTupleOffset(srcBuffer, srcBuffer.position());
+        srcBuffer.position(srcBuffer.position() + size);
+        return destTuple;
+    }
+
+    @Override
+    public void formJobCommitLogRecord(ITransactionContext txnCtx) {
+        this.txnCtx = txnCtx;
+        this.logType = LogType.JOB_COMMIT;
+        this.jobId = txnCtx.getJobId().getId();
+        this.datasetId = -1;
+        this.PKHashValue = -1;
+        computeAndSetLogSize();
+    }
+
+    @Override
+    public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
+            ITupleReference PKValue, int[] PKFields) {
+        this.txnCtx = txnCtx;
+        this.logType = LogType.ENTITY_COMMIT;
+        this.jobId = txnCtx.getJobId().getId();
+        this.datasetId = datasetId;
+        this.PKHashValue = PKHashValue;
+        this.PKFieldCnt = PKFields.length;
+        this.PKValue = PKValue;
+        this.PKFields = PKFields;
+        computeAndSetPKValueSize();
+        computeAndSetLogSize();
+    }
+
+    @Override
+    public void computeAndSetPKValueSize() {
+        int i;
+        PKValueSize = 0;
+        for (i = 0; i < PKFieldCnt; i++) {
+            PKValueSize += PKValue.getFieldLength(PKFields[i]);
+        }
+    }
+
+    private void setUpdateLogSize() {
+        logSize = UPDATE_LOG_BASE_SIZE + PKValueSize + newValueSize + oldValueSize;
+        if (resourceType != ResourceType.LSM_BTREE) {
+            logSize -= 5; //oldOp(byte: 1) + oldValueLength(int: 4)
+        } else {
+            if (oldOp == (byte) (IndexOperation.NOOP.ordinal())) {
+                logSize -= 4; //oldValueLength(int: 4)
+            }
+        }
+    }
+
+    @Override
+    public void computeAndSetLogSize() {
+        switch (logType) {
+            case LogType.UPDATE:
+                setUpdateLogSize();
+                break;
+            case LogType.JOB_COMMIT:
+                logSize = JOB_COMMIT_LOG_SIZE;
+                break;
+            case LogType.ENTITY_COMMIT:
+                logSize = ENTITY_COMMIT_LOG_BASE_SIZE + PKValueSize;
+                break;
+            default:
+                throw new IllegalStateException("Unsupported Log Type");
+        }
+    }
+
+    @Override
+    public String getLogRecordForDisplay() {
+        StringBuilder builder = new StringBuilder();
+        builder.append(" LSN : ").append(LSN);
+        builder.append(" LogType : ").append(LogType.toString(logType));
+        builder.append(" LogSize : ").append(logSize);
+        builder.append(" JobId : ").append(jobId);
+        if (logType != LogType.JOB_COMMIT) {
+            builder.append(" DatasetId : ").append(datasetId);
+            builder.append(" PKHashValue : ").append(PKHashValue);
+            builder.append(" PKFieldCnt : ").append(PKFieldCnt);
+            builder.append(" PKSize: ").append(PKValueSize);
+        }
+        if (logType == LogType.UPDATE) {
+            builder.append(" PrevLSN : ").append(prevLSN);
+            builder.append(" ResourceId : ").append(resourceId);
+            builder.append(" ResourceType : ").append(resourceType);
+        }
+        return builder.toString();
+    }
+
+    ////////////////////////////////////////////
+    // getter and setter methods
+    ////////////////////////////////////////////
+
+    @Override
+    public ITransactionContext getTxnCtx() {
+        return txnCtx;
+    }
+
+    @Override
+    public void setTxnCtx(ITransactionContext txnCtx) {
+        this.txnCtx = txnCtx;
+    }
+
+    @Override
+    public boolean isFlushed() {
+        return isFlushed.get();
+    }
+
+    @Override
+    public void isFlushed(boolean isFlushed) {
+        this.isFlushed.set(isFlushed);
+    }
+
+    @Override
+    public byte getLogType() {
+        return logType;
+    }
+
+    @Override
+    public void setLogType(byte logType) {
+        this.logType = logType;
+    }
+
+    @Override
+    public int getJobId() {
+        return jobId;
+    }
+
+    @Override
+    public void setJobId(int jobId) {
+        this.jobId = jobId;
+    }
+
+    @Override
+    public int getDatasetId() {
+        return datasetId;
+    }
+
+    @Override
+    public void setDatasetId(int datasetId) {
+        this.datasetId = datasetId;
+    }
+
+    @Override
+    public int getPKHashValue() {
+        return PKHashValue;
+    }
+
+    @Override
+    public void setPKHashValue(int PKHashValue) {
+        this.PKHashValue = PKHashValue;
+    }
+
+    @Override
+    public long getPrevLSN() {
+        return prevLSN;
+    }
+
+    @Override
+    public void setPrevLSN(long prevLSN) {
+        this.prevLSN = prevLSN;
+    }
+
+    @Override
+    public long getResourceId() {
+        return resourceId;
+    }
+
+    @Override
+    public void setResourceId(long resourceId) {
+        this.resourceId = resourceId;
+    }
+
+    @Override
+    public byte getResourceType() {
+        return resourceType;
+    }
+
+    @Override
+    public void setResourceType(byte resourceType) {
+        this.resourceType = resourceType;
+    }
+
+    @Override
+    public int getLogSize() {
+        return logSize;
+    }
+
+    @Override
+    public void setLogSize(int logSize) {
+        this.logSize = logSize;
+    }
+
+    @Override
+    public byte getNewOp() {
+        return newOp;
+    }
+
+    @Override
+    public void setNewOp(byte newOp) {
+        this.newOp = newOp;
+    }
+
+    @Override
+    public int getNewValueSize() {
+        return newValueSize;
+    }
+
+    @Override
+    public void setNewValueSize(int newValueSize) {
+        this.newValueSize = newValueSize;
+    }
+
+    @Override
+    public ITupleReference getNewValue() {
+        return newValue;
+    }
+
+    @Override
+    public void setNewValue(ITupleReference newValue) {
+        this.newValue = newValue;
+        this.fieldCnt = newValue.getFieldCount();
+    }
+
+    @Override
+    public byte getOldOp() {
+        return oldOp;
+    }
+
+    @Override
+    public void setOldOp(byte oldOp) {
+        this.oldOp = oldOp;
+    }
+
+    @Override
+    public int getOldValueSize() {
+        return oldValueSize;
+    }
+
+    @Override
+    public void setOldValueSize(int oldValueSize) {
+        this.oldValueSize = oldValueSize;
+    }
+
+    @Override
+    public ITupleReference getOldValue() {
+        return oldValue;
+    }
+
+    @Override
+    public void setOldValue(ITupleReference oldValue) {
+        this.oldValue = oldValue;
+    }
+
+    @Override
+    public long getChecksum() {
+        return checksum;
+    }
+
+    @Override
+    public void setChecksum(long checksum) {
+        this.checksum = checksum;
+    }
+
+    @Override
+    public long getLSN() {
+        return LSN;
+    }
+
+    @Override
+    public void setLSN(long LSN) {
+        this.LSN = LSN;
+    }
+    
+    @Override
+    public int getPKValueSize() {
+        return PKValueSize;
+    }
+    
+    @Override
+    public ITupleReference getPKValue() {
+        return PKValue;
+    }
+    
+    @Override
+    public void setPKFields(int[] primaryKeyFields) {
+        PKFields = primaryKeyFields;
+        PKFieldCnt = PKFields.length;
+    }
+
+    @Override
+    public void setPKValue(ITupleReference PKValue) {
+        this.PKValue = PKValue;
+    }
+}
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
deleted file mode 100644
index d46ade8..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * An implementation of the @see ILogRecordHelper interface that provides API
- * 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 LOG_CHECKSUM_SIZE = 8;
-    private final int LOG_HEADER_PART1_SIZE = 17;
-    private final int LOG_HEADER_PART2_SIZE = 21;
-    private final int COMMIT_LOG_SIZE = LOG_HEADER_PART1_SIZE + LOG_CHECKSUM_SIZE;
-
-    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;
-
-    public LogRecordHelper(ILogManager logManager) {
-        this.logManager = logManager;
-    }
-
-    @Override
-    public byte getLogType(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS);
-    }
-
-    @Override
-    public int getJobId(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS);
-    }
-
-    @Override
-    public int getDatasetId(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS);
-    }
-
-    @Override
-    public int getPKHashValue(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_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;
-    }
-
-    @Override
-    public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator) {
-        long prevLsnValue = (logicalLogLocator.getBuffer())
-                .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
-        if (prevLsnValue == -1) {
-            return false;
-        }
-        physicalLogLocator.setLsn(prevLsnValue);
-        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) {
-        if (getLogType(logicalLogLocater) == LogType.COMMIT || getLogType(logicalLogLocater) == LogType.ENTITY_COMMIT) {
-            return 0;
-        } else {
-            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));
-        String logTypeDisplay = null;
-        switch (logType) {
-            case LogType.COMMIT:
-                logTypeDisplay = "COMMIT";
-                break;
-            case LogType.UPDATE:
-                logTypeDisplay = "UPDATE";
-                break;
-            case LogType.ENTITY_COMMIT:
-                logTypeDisplay = "ENTITY_COMMIT";
-                break;
-        }
-        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 void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
-            int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
-            int logRecordSize) {
-
-        /* magic no */
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + MAGIC_NO_POS,
-                LogManagerProperties.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 - 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 LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE + LOG_CHECKSUM_SIZE + logBodySize;
-        } else {
-            return COMMIT_LOG_SIZE;
-        }
-    }
-
-    @Override
-    public int getLogHeaderSize(byte logType) {
-        if (logType == LogType.UPDATE) {
-            return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE;
-        } else {
-            return LOG_HEADER_PART1_SIZE;
-        }
-    }
-
-    @Override
-    public int getLogChecksumSize() {
-        return LOG_CHECKSUM_SIZE;
-    }
-
-    public int getCommitLogSize() {
-        return COMMIT_LOG_SIZE;
-    }
-}
\ No newline at end of file
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 99c0fef..823c8d3 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
@@ -17,7 +17,24 @@
 public class LogType {
 
     public static final byte UPDATE = 0;
-    public static final byte COMMIT = 1;
+    public static final byte JOB_COMMIT = 1;
     public static final byte ENTITY_COMMIT = 2;
+    private static final String STRING_UPDATE = "UPDATE";
+    private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
+    private static final String STRING_ENTITY_COMMIT = "ENTITY_COMMIT";
+    private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
+
+    public static String toString(byte logType) {
+        switch (logType) {
+            case LogType.UPDATE:
+                return STRING_UPDATE;
+            case LogType.JOB_COMMIT:
+                return STRING_JOB_COMMIT;
+            case LogType.ENTITY_COMMIT:
+                return STRING_ENTITY_COMMIT;
+            default:
+                return STRING_INVALID_LOG_TYPE;
+        }
+    }
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
deleted file mode 100644
index d63ce1c..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * 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.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a memory-only buffer that is not associated with and physical
- * file.
- */
-public class MemBasedBuffer extends Buffer {
-
-    public MemBasedBuffer(byte[] content) {
-        super(ByteBuffer.wrap(content));
-    }
-
-    public MemBasedBuffer(ByteBuffer buffer) {
-        super(buffer);
-    }
-
-}
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
index 110ddee..f2cb22a 100644
--- 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
@@ -20,24 +20,24 @@
 
     private static final long serialVersionUID = 1L;
  
-    private final long checkpointLSN;
-    private final long minMCTFirstLSN;
+    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;
+    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 getCheckpointLsn() {
+        return checkpointLsn;
     }
 
-    public long getMinMCTFirstLSN() {
-        return minMCTFirstLSN;
+    public long getMinMCTFirstLsn() {
+        return minMCTFirstLsn;
     }
 
     public int getMaxJobId() {
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
index a558969..6f6da4a 100644
--- 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
@@ -19,6 +19,7 @@
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 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;
@@ -56,7 +57,7 @@
                 try {
                     recoveryMgr.checkpoint(false);
                     lastMinMCTFirstLSN = currentMinMCTFirstLSN;
-                } catch (ACIDException e) {
+                } catch (ACIDException | HyracksDataException e) {
                     throw new Error("failed to checkpoint", e);
                 }
             }
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 ad1db1f..2ad3055 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
@@ -27,6 +27,7 @@
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.List;
@@ -39,20 +40,12 @@
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-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.transaction.TransactionManagementConstants;
@@ -60,23 +53,14 @@
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
+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.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.common.ophelpers.IndexOperation;
 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;
 
@@ -91,6 +75,7 @@
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
     private final TransactionSubsystem txnSubsystem;
+    private final LogManager logMgr;
     private final int checkpointHistory;
 
     /**
@@ -100,8 +85,9 @@
     private static final String CHECKPOINT_FILENAME_PREFIX = "checkpoint_";
     private SystemState state;
 
-    public RecoveryManager(TransactionSubsystem TransactionProvider) throws ACIDException {
-        this.txnSubsystem = TransactionProvider;
+    public RecoveryManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+        this.txnSubsystem = txnSubsystem;
+        this.logMgr = (LogManager) txnSubsystem.getLogManager();
         this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
     }
 
@@ -132,12 +118,15 @@
         //#. 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()) {
+        if ((checkpointObject.getMinMCTFirstLsn() == -2 && logMgr.getAppendLSN() == 0)
+                || (checkpointObject.getMinMCTFirstLsn() == -1 && checkpointObject.getCheckpointLsn() == logMgr
+                        .getAppendLSN())) {
             state = SystemState.HEALTHY;
             return state;
         } else {
+            if (logMgr.getAppendLSN() == 0) {
+                throw new IllegalStateException("Transaction log files are lost.");
+            }
             state = SystemState.CORRUPTED;
             return state;
         }
@@ -146,151 +135,133 @@
     public void startRecovery(boolean synchronous) throws IOException, ACIDException {
 
         int updateLogCount = 0;
-        int commitLogCount = 0;
+        int entityCommitLogCount = 0;
+        int jobCommitLogCount = 0;
         int redoCount = 0;
+        int jobId = -1;
 
         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;
+        Set<Integer> winnerJobSet = new HashSet<Integer>();
+        Map<Integer, Set<TxnId>> jobId2WinnerEntitiesMap = new HashMap<Integer, Set<TxnId>>();
+        //winnerEntity is used to add pairs, <committed TxnId, the most recent commit Lsn of the TxnId>
+        Set<TxnId> winnerEntitySet = null;
+        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
+        TxnId winnerEntity = null;
 
         //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
         CheckpointObject checkpointObject = readCheckpoint();
-        long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLSN();
-        if (lowWaterMarkLSN == -1) {
-            lowWaterMarkLSN = 0;
+        long lowWaterMarkLsn = checkpointObject.getMinMCTFirstLsn();
+        if (lowWaterMarkLsn == -1 || lowWaterMarkLsn == -2) {
+            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.
+        //  - collect all committed Lsn 
         //-------------------------------------------------------------------------
         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));
+        //#. set log reader to the lowWaterMarkLsn
+        ILogReader logReader = logMgr.getLogReader(true);
+        logReader.initializeScan(lowWaterMarkLsn);
+        ILogRecord logRecord = logReader.next();
+        while (logRecord != null) {
+            if (IS_DEBUG_MODE) {
+                System.out.println(logRecord.getLogRecordForDisplay());
             }
-
-            logType = logRecordHelper.getLogType(currentLogLocator);
-
             //update max jobId
-            currentJobId = logRecordHelper.getJobId(currentLogLocator);
-            if (currentJobId > maxJobId) {
-                maxJobId = currentJobId;
+            if (logRecord.getJobId() > maxJobId) {
+                maxJobId = logRecord.getJobId();
             }
-
-            TxnId commitTxnId = null;
-            switch (logType) {
+            switch (logRecord.getLogType()) {
                 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());
+                case LogType.JOB_COMMIT:
+                    winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
+                    jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
                     if (IS_DEBUG_MODE) {
-                        commitLogCount++;
+                        jobCommitLogCount++;
                     }
                     break;
-
+                case LogType.ENTITY_COMMIT:
+                    jobId = logRecord.getJobId();
+                    winnerEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                            logRecord.getPKValue(), logRecord.getPKValueSize(), true);
+                    if (!jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+                        winnerEntitySet = new HashSet<TxnId>();
+                        jobId2WinnerEntitiesMap.put(Integer.valueOf(jobId), winnerEntitySet);
+                    } else {
+                        winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+                    }
+                    winnerEntitySet.add(winnerEntity);
+                    if (IS_DEBUG_MODE) {
+                        entityCommitLogCount++;
+                    }
+                    break;
                 default:
-                    throw new ACIDException("Unsupported LogType: " + logType);
+                    throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
+            logRecord = logReader.next();
         }
 
         //-------------------------------------------------------------------------
         //  [ redo phase ]
         //  - redo if
-        //    1) The TxnId is committed --> gurantee durability
-        //      &&  
-        //    2) the currentLSN > maxDiskLastLSN of the index --> guarantee idempotance
+        //    1) The TxnId is committed && --> guarantee durability
+        //    2) lsn > maxDiskLastLsn of the index --> guarantee idempotence
         //-------------------------------------------------------------------------
         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;
-            }
-        });
-        currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-
         long resourceId;
-        byte resourceMgrId;
-        long maxDiskLastLSN;
-        long currentLSN = -1;
-        int resourceType;
+        long maxDiskLastLsn;
+        long LSN = -1;
         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;
+        boolean foundWinner = false;
 
         //#. get indexLifeCycleManager 
         IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
         IIndexLifecycleManager indexLifecycleManager = appRuntimeContext.getIndexLifecycleManager();
         ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
 
-        //#. redo
-        while (logCursor.next(currentLogLocator)) {
-            foundWinnerTxn = false;
+        //#. set log reader to the lowWaterMarkLsn again.
+        logReader.initializeScan(lowWaterMarkLsn);
+        logRecord = logReader.next();
+        while (logRecord != null) {
             if (LogManager.IS_DEBUG_MODE) {
-                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+                System.out.println(logRecord.getLogRecordForDisplay());
             }
-
-            logType = logRecordHelper.getLogType(currentLogLocator);
-
-            switch (logType) {
+            LSN = logRecord.getLSN();
+            jobId = logRecord.getJobId();
+            foundWinner = false;
+            switch (logRecord.getLogType()) {
                 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 (winnerJobSet.contains(Integer.valueOf(jobId))) {
+                        foundWinner = true;
+                    } else if (jobId2WinnerEntitiesMap.containsKey(Integer.valueOf(jobId))) {
+                        winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
+                        tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                                logRecord.getPKValue(), logRecord.getPKValueSize());
+                        if (winnerEntitySet.contains(tempKeyTxnId)) {
+                            foundWinner = true;
+                        }
                     }
-
-                    if (foundWinnerTxn) {
-                        resourceId = logRecordHelper.getResourceId(currentLogLocator);
+                    if (foundWinner) {
+                        resourceId = logRecord.getResourceId();
                         localResource = localResourceRepository.getResourceById(resourceId);
 
                         //get index instance from IndexLifeCycleManager
@@ -325,59 +296,20 @@
                             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");
-                            }
+                            ILSMIndex lsmIndex = (ILSMIndex) index;
+                            BaseOperationTracker indexOpTracker = (BaseOperationTracker) lsmIndex.getOperationTracker();
+                            AbstractLSMIOOperationCallback abstractLSMIOCallback = (AbstractLSMIOOperationCallback) indexOpTracker
+                                    .getIOOperationCallback();
+                            maxDiskLastLsn = abstractLSMIOCallback.getComponentLSN(index.getImmutableComponents());
 
                             //#. set resourceId and maxDiskLastLSN to the map
-                            resourceId2MaxLSNMap.put(resourceId, maxDiskLastLSN);
+                            resourceId2MaxLSNMap.put(Long.valueOf(resourceId), Long.valueOf(maxDiskLastLsn));
                         } else {
-                            maxDiskLastLSN = resourceId2MaxLSNMap.get(resourceId);
+                            maxDiskLastLsn = resourceId2MaxLSNMap.get(Long.valueOf(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 (LSN > maxDiskLastLsn) {
+                            redo(logRecord);
                             if (IS_DEBUG_MODE) {
                                 redoCount++;
                             }
@@ -385,14 +317,15 @@
                     }
                     break;
 
-                case LogType.COMMIT:
+                case LogType.JOB_COMMIT:
                 case LogType.ENTITY_COMMIT:
                     //do nothing
                     break;
 
                 default:
-                    throw new ACIDException("Unsupported LogType: " + logType);
+                    throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
+            logRecord = logReader.next();
         }
 
         //close all indexes
@@ -401,42 +334,25 @@
             indexLifecycleManager.close(r);
         }
 
+        logReader.close();
+
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("[RecoveryMgr] recovery is completed.");
         }
         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);
+            System.out.println("[RecoveryMgr] Count: Update/EntityCommit/JobCommit/Redo = " + updateLogCount + "/"
+                    + entityCommitLogCount + "/" + jobCommitLogCount + "/" + redoCount);
         }
     }
 
     @Override
-    public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException {
+    public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException, HyracksDataException {
 
+        long minMCTFirstLSN;
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting sharp checkpoint ... ");
         }
 
-        LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
         TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
         String logDir = logMgr.getLogManagerProperties().getLogDir();
 
@@ -447,10 +363,14 @@
         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) {
+            ///////////////////////////////////////////////
+            //TODO : change the code inside the if statement into indexLifeCycleManager.flushAllDatasets()
+            //indexLifeCycleManager.flushAllDatasets();
+            ///////////////////////////////////////////////
+            List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
             for (IIndex index : openIndexList) {
                 ILSMIndex lsmIndex = (ILSMIndex) index;
                 ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(NoOpOperationCallback.INSTANCE,
@@ -473,21 +393,20 @@
                     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 = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
-                minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
-            }
+            minMCTFirstLSN = -2;
         } else {
-            minMCTFirstLSN = -1;
+            long firstLSN;
+            minMCTFirstLSN = Long.MAX_VALUE;
+            if (openIndexList.size() > 0) {
+                for (IIndex index : openIndexList) {
+                    firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                    minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
+                }
+            } else {
+                minMCTFirstLSN = -1;
+            }
         }
-
-        CheckpointObject checkpointObject = new CheckpointObject(logMgr.getCurrentLsn().get(), minMCTFirstLSN,
+        CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
                 txnMgr.getMaxJobId(), System.currentTimeMillis());
 
         FileOutputStream fos = null;
@@ -585,7 +504,7 @@
     }
 
     private File[] getPreviousCheckpointFiles() {
-        String logDir = txnSubsystem.getLogManager().getLogManagerProperties().getLogDir();
+        String logDir = ((LogManager) txnSubsystem.getLogManager()).getLogManagerProperties().getLogDir();
 
         File parentDir = new File(logDir);
 
@@ -621,24 +540,25 @@
      */
     @Override
     public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
-        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);
+        TxnId tempKeyTxnId = new TxnId(-1, -1, -1, null, -1, false);
 
         int updateLogCount = 0;
-        int commitLogCount = 0;
+        int entityCommitLogCount = 0;
+        int jobId = -1;
+        int abortedJobId = txnContext.getJobId().getId();
+        long currentLSN = -1;
+        TxnId loserEntity = null;
 
-        // Obtain the first log record written by the Job
-        PhysicalLogLocator firstLSNLogLocator = txnContext.getFirstLogLocator();
-        PhysicalLogLocator lastLSNLogLocator = txnContext.getLastLogLocator();
+        // Obtain the first/last log record LSNs written by the Job
+        long firstLSN = txnContext.getFirstLSN();
+        long lastLSN = txnContext.getLastLSN();
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" rollbacking transaction log records from " + firstLSNLogLocator.getLsn() + " to "
-                    + lastLSNLogLocator.getLsn());
+            LOGGER.info(" rollbacking transaction log records from " + firstLSN + " to " + lastLSN);
         }
 
         // check if the transaction actually wrote some logs.
-        if (firstLSNLogLocator.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
+        if (firstLSN == 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.getJobId());
@@ -646,147 +566,107 @@
             return;
         }
 
-        // 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);
-        }
-
-        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-        boolean valid;
-        byte logType;
-        List<Long> undoLSNSet = null;
-
+        // While reading log records from firstLsn to lastLsn, collect uncommitted txn's Lsns
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" collecting loser transaction's LSNs from " + firstLSNLogLocator.getLsn() + " to "
-                    + +lastLSNLogLocator.getLsn());
+            LOGGER.info(" collecting loser transaction's LSNs from " + firstLSN + " to " + lastLSN);
         }
-
-        while (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
-            try {
-                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
+        List<Long> undoLSNSet = null;
+        ILogReader logReader = logMgr.getLogReader(false);
+        logReader.initializeScan(firstLSN);
+        ILogRecord logRecord = null;
+        while (currentLSN < lastLSN) {
+            logRecord = logReader.next();
+            if (logRecord == null) {
+                break;
+            } else {
+                if (IS_DEBUG_MODE) {
+                    System.out.println(logRecord.getLogRecordForDisplay());
                 }
+                currentLSN = logRecord.getLSN();
             }
-
-            if (IS_DEBUG_MODE) {
-                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+            jobId = logRecord.getJobId();
+            if (jobId != abortedJobId) {
+                continue;
             }
-
-            tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
-                    logRecordHelper.getDatasetId(currentLogLocator), logRecordHelper.getPKHashValue(currentLogLocator));
-            logType = logRecordHelper.getLogType(currentLogLocator);
-
-            switch (logType) {
+            tempKeyTxnId.setTxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(), logRecord.getPKValue(),
+                    logRecord.getPKValueSize());
+            switch (logRecord.getLogType()) {
                 case LogType.UPDATE:
                     undoLSNSet = loserTxnTable.get(tempKeyTxnId);
                     if (undoLSNSet == null) {
-                        TxnId txnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
-                                logRecordHelper.getDatasetId(currentLogLocator),
-                                logRecordHelper.getPKHashValue(currentLogLocator));
+                        loserEntity = new TxnId(jobId, logRecord.getDatasetId(), logRecord.getPKHashValue(),
+                                logRecord.getPKValue(), logRecord.getPKValueSize(), true);
                         undoLSNSet = new LinkedList<Long>();
-                        loserTxnTable.put(txnId, undoLSNSet);
+                        loserTxnTable.put(loserEntity, undoLSNSet);
                     }
-                    undoLSNSet.add(currentLogLocator.getLsn());
+                    undoLSNSet.add(Long.valueOf(currentLSN));
                     if (IS_DEBUG_MODE) {
                         updateLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> update["
-                                + currentLogLocator.getLsn() + "]:" + tempKeyTxnId);
+                        System.out.println("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+                                + tempKeyTxnId);
                     }
                     break;
 
-                case LogType.COMMIT:
+                case LogType.JOB_COMMIT:
+                    throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
+
                 case LogType.ENTITY_COMMIT:
-                    undoLSNSet = loserTxnTable.get(tempKeyTxnId);
-                    if (undoLSNSet != null) {
-                        loserTxnTable.remove(tempKeyTxnId);
-                    }
+                    loserTxnTable.remove(tempKeyTxnId);
                     if (IS_DEBUG_MODE) {
-                        commitLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> commit["
-                                + currentLogLocator.getLsn() + "]" + tempKeyTxnId);
+                        entityCommitLogCount++;
+                        System.out.println("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
+                                + tempKeyTxnId);
                     }
                     break;
 
                 default:
-                    throw new ACIDException("Unsupported LogType: " + logType);
+                    throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
         }
+        if (currentLSN != lastLSN) {
+            throw new ACIDException("LastLSN mismatch: lastLSN(" + lastLSN + ") vs currentLSN(" + currentLSN
+                    + ") during abort( " + txnContext.getJobId() + ")");
+        }
 
         //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.
-
+                //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);
-
+                logRecord = logReader.read(undoLSN);
+                if (logRecord == null) {
+                    throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
+                }
                 if (IS_DEBUG_MODE) {
-                    System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+                    System.out.println(logRecord.getLogRecordForDisplay());
                 }
-
-                // 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);
-
+                undo(logRecord);
                 if (IS_DEBUG_MODE) {
                     undoCount++;
                 }
             }
         }
 
+        logReader.close();
+
         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);
+            System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + entityCommitLogCount
+                    + "/" + undoCount);
         }
     }
 
@@ -799,39 +679,103 @@
     public void stop(boolean dumpState, OutputStream os) {
         //no op
     }
+
+    private void undo(ILogRecord logRecord) {
+        try {
+            ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(logRecord.getResourceId());
+            ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            if (logRecord.getResourceType() == ResourceType.LSM_BTREE) {
+                if (logRecord.getOldOp() != IndexOperation.NOOP.ordinal()) {
+                    if (logRecord.getOldOp() == IndexOperation.DELETE.ordinal()) {
+                        indexAccessor.forceDelete(logRecord.getOldValue());
+                    } else {
+                        indexAccessor.forceInsert(logRecord.getOldValue());
+                    }
+                } else {
+                    indexAccessor.forcePhysicalDelete(logRecord.getNewValue());
+                }
+            } else {
+                if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+                    indexAccessor.forceInsert(logRecord.getNewValue());
+                } else {
+                    indexAccessor.forceDelete(logRecord.getNewValue());
+                }
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException("Failed to undo", e);
+        }
+    }
+
+    private void redo(ILogRecord logRecord) {
+        try {
+            ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(logRecord.getResourceId());
+            ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
+                indexAccessor.forceInsert(logRecord.getNewValue());
+            } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+                indexAccessor.forceDelete(logRecord.getNewValue());
+            } else {
+                throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
+            }
+            ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(logRecord.getLSN());
+        } catch (Exception e) {
+            throw new IllegalStateException("Failed to redo", e);
+        }
+    }
 }
 
 class TxnId {
+    public boolean isByteArrayPKValue;
     public int jobId;
     public int datasetId;
-    public int pkHashVal;
+    public int pkHashValue;
+    public int pkSize;
+    public byte[] byteArrayPKValue;
+    public ITupleReference tupleReferencePKValue;
 
-    public TxnId(int jobId, int datasetId, int pkHashVal) {
+    public TxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize,
+            boolean isByteArrayPKValue) {
         this.jobId = jobId;
         this.datasetId = datasetId;
-        this.pkHashVal = pkHashVal;
+        this.pkHashValue = pkHashValue;
+        this.pkSize = pkSize;
+        this.isByteArrayPKValue = isByteArrayPKValue;
+        if (isByteArrayPKValue) {
+            this.byteArrayPKValue = new byte[pkSize];
+            readPKValueIntoByteArray(pkValue, pkSize, byteArrayPKValue);
+        } else {
+            this.tupleReferencePKValue = pkValue;
+        }
     }
 
-    public void setTxnId(int jobId, int datasetId, int pkHashVal) {
+    private void readPKValueIntoByteArray(ITupleReference pkValue, int pkSize, byte[] byteArrayPKValue) {
+        int readOffset = pkValue.getFieldStart(0);
+        byte[] readBuffer = pkValue.getFieldData(0);
+        for (int i = 0; i < pkSize; i++) {
+            byteArrayPKValue[i] = readBuffer[readOffset + i];
+        }
+    }
+
+    public void setTxnId(int jobId, int datasetId, int pkHashValue, ITupleReference pkValue, int pkSize) {
         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;
+        this.pkHashValue = pkHashValue;
+        this.tupleReferencePKValue = pkValue;
+        isByteArrayPKValue = false;
     }
 
     @Override
     public String toString() {
-        return "[" + jobId + "," + datasetId + "," + pkHashVal + "]";
+        return "[" + jobId + "," + datasetId + "," + pkHashValue + "," + pkSize + "]";
     }
 
     @Override
     public int hashCode() {
-        return pkHashVal;
+        return pkHashValue;
     }
 
     @Override
@@ -843,7 +787,52 @@
             return false;
         }
         TxnId txnId = (TxnId) o;
+        return (txnId.pkHashValue == pkHashValue && txnId.datasetId == datasetId && txnId.jobId == jobId
+                && pkSize == txnId.pkSize && isEqualTo(txnId));
+    }
 
-        return (txnId.pkHashVal == pkHashVal && txnId.datasetId == datasetId && txnId.jobId == jobId);
+    private boolean isEqualTo(TxnId txnId) {
+        if (isByteArrayPKValue && txnId.isByteArrayPKValue) {
+            return isEqual(byteArrayPKValue, txnId.byteArrayPKValue, pkSize);
+        } else if (isByteArrayPKValue && (!txnId.isByteArrayPKValue)) {
+            return isEqual(byteArrayPKValue, txnId.tupleReferencePKValue, pkSize);
+        } else if ((!isByteArrayPKValue) && txnId.isByteArrayPKValue) {
+            return isEqual(txnId.byteArrayPKValue, tupleReferencePKValue, pkSize);
+        } else {
+            return isEqual(tupleReferencePKValue, txnId.tupleReferencePKValue, pkSize);
+        }
+    }
+
+    private boolean isEqual(byte[] a, byte[] b, int size) {
+        for (int i = 0; i < size; i++) {
+            if (a[i] != b[i]) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private boolean isEqual(byte[] a, ITupleReference b, int size) {
+        int readOffset = b.getFieldStart(0);
+        byte[] readBuffer = b.getFieldData(0);
+        for (int i = 0; i < size; i++) {
+            if (a[i] != readBuffer[readOffset + i]) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    private boolean isEqual(ITupleReference a, ITupleReference b, int size) {
+        int aOffset = a.getFieldStart(0);
+        byte[] aBuffer = a.getFieldData(0);
+        int bOffset = b.getFieldStart(0);
+        byte[] bBuffer = b.getFieldData(0);
+        for (int i = 0; i < size; i++) {
+            if (aBuffer[aOffset + i] != bBuffer[bOffset + i]) {
+                return false;
+            }
+        }
+        return true;
     }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
deleted file mode 100644
index e64ec01..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.transaction.management.service.recovery;
-
-import edu.uci.ics.asterix.common.transactions.ITransactionManager;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * Represents a bookkeeping data-structure that is populated duing the analysis
- * phase of recovery. It contains for each transaction, the transaction state,
- * the LSN corresponding to the last log record written by the transaction and
- * the log record written by the transaction that needs to be undone.
- */
-public class TransactionTableEntry {
-
-    private long transactionId;
-    private ITransactionManager.TransactionState transactionState;
-    private PhysicalLogLocator lastLSN;
-    private PhysicalLogLocator undoNextLSN;
-
-    public TransactionTableEntry(long transactionId, ITransactionManager.TransactionState transactionState,
-            PhysicalLogLocator lastLSN, PhysicalLogLocator undoNextLSN) {
-        this.transactionId = transactionId;
-        this.transactionState = transactionState;
-        this.lastLSN = lastLSN;
-        this.undoNextLSN = undoNextLSN;
-    }
-
-    public long getTransactionId() {
-        return transactionId;
-    }
-
-    public void setTransactionId(long transactionId) {
-        this.transactionId = transactionId;
-    }
-
-    public ITransactionManager.TransactionState getTransactionState() {
-        return transactionState;
-    }
-
-    public void setTransactionState(ITransactionManager.TransactionState transactionState) {
-        this.transactionState = transactionState;
-    }
-
-    public PhysicalLogLocator getLastLSN() {
-        return lastLSN;
-    }
-
-    public void setLastLSN(PhysicalLogLocator lastLSN) {
-        this.lastLSN = lastLSN;
-    }
-
-    public PhysicalLogLocator getUndoNextLSN() {
-        return undoNextLSN;
-    }
-
-    public void setUndoNextLSN(PhysicalLogLocator undoNextLSN) {
-        this.undoNextLSN = undoNextLSN;
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
index 97f2477..59a8363 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
@@ -16,22 +16,16 @@
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 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.ILSMOperationTrackerProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.file.IFileMapProvider;
@@ -39,33 +33,13 @@
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
 public class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
-        ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider, ILSMOperationTrackerProvider,
+        ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider,
         ILSMIOOperationCallbackProvider {
     private static final long serialVersionUID = 1L;
 
-    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
-    private final boolean isSecondary;
-
-    public static final AsterixRuntimeComponentsProvider LSMBTREE_PRIMARY_PROVIDER = new AsterixRuntimeComponentsProvider(
-            LSMBTreeIOOperationCallbackFactory.INSTANCE, false);
-    public static final AsterixRuntimeComponentsProvider LSMBTREE_SECONDARY_PROVIDER = new AsterixRuntimeComponentsProvider(
-            LSMBTreeIOOperationCallbackFactory.INSTANCE, true);
-    public static final AsterixRuntimeComponentsProvider LSMRTREE_PROVIDER = new AsterixRuntimeComponentsProvider(
-            LSMRTreeIOOperationCallbackFactory.INSTANCE, true);
-    public static final AsterixRuntimeComponentsProvider LSMINVERTEDINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(
-            LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, true);
-    public static final AsterixRuntimeComponentsProvider NOINDEX_PROVIDER = new AsterixRuntimeComponentsProvider(null,
-            false);
-
-    private AsterixRuntimeComponentsProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory, boolean isSecondary) {
-        this.ioOpCallbackFactory = ioOpCallbackFactory;
-        this.isSecondary = isSecondary;
-    }
-
-    @Override
-    public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
-        assert isSecondary;
-        return new BaseOperationTracker(ioOpCallbackFactory);
+    public static final AsterixRuntimeComponentsProvider RUNTIME_PROVIDER = new AsterixRuntimeComponentsProvider();
+    
+    private AsterixRuntimeComponentsProvider() {
     }
 
     @Override
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 33522e3..678956b 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,176 +15,176 @@
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
 import java.io.Serializable;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.PrimaryIndexOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
-import edu.uci.ics.asterix.common.transactions.ICloseable;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallback;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
 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
- * transaction. A TransactionContext instance can be used as a token and
- * provided to Transaction sub-systems (Log/Lock/Recovery/Transaction)Manager to
- * initiate an operation on the behalf of the transaction associated with the
- * context.
+/*
+ * An object of TransactionContext is created and accessed(read/written) by multiple threads which work for
+ * a single job identified by a jobId. Thus, the member variables in the object can be read/written
+ * concurrently. Please see each variable declaration to know which one is accessed concurrently and
+ * which one is not. 
  */
 public class TransactionContext implements ITransactionContext, Serializable {
 
     private static final long serialVersionUID = -6105616785783310111L;
     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;
-    private boolean exlusiveJobLevelCommit;
 
-    // List of indexes on which operations were performed on behalf of this transaction.
-    private final Set<ILSMIndex> indexes = new HashSet<ILSMIndex>();
+    //jobId is set once and read concurrently.
+    private final JobId jobId;
 
-    // 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 Set<AbstractOperationCallback> callbacks = new HashSet<AbstractOperationCallback>();
+    //There are no concurrent writers on both firstLSN and lastLSN 
+    //since both values are updated by serialized log appenders. 
+    //But readers and writers can be different threads, 
+    //so both LSNs are atomic variables in order to be read and written atomically.
+    private AtomicLong firstLSN;
+    private AtomicLong lastLSN;
 
+    //txnState is read and written concurrently.
+    private AtomicInteger txnState;
+
+    //isTimeout is read and written under the lockMgr's tableLatch
+    //Thus, no other synchronization is required separately.
+    private boolean isTimeout;
+
+    //isWriteTxn can be set concurrently by multiple threads. 
+    private AtomicBoolean isWriteTxn;
+
+    //isMetadataTxn is accessed by a single thread since the metadata is not partitioned
+    private boolean isMetadataTxn;
+
+    //indexMap is concurrently accessed by multiple threads, 
+    //so those threads are synchronized on indexMap object itself
+    private Map<MutableLong, BaseOperationTracker> indexMap;
+
+    //TODO: fix ComponentLSNs' issues. 
+    //primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be modified accordingly
+    //when the issues of componentLSNs are fixed.  
+    private ILSMIndex primaryIndex;
+    private PrimaryIndexModificationOperationCallback primaryIndexCallback;
+    private PrimaryIndexOperationTracker primaryIndexOpTracker;
+
+    //The following three variables are used as temporary variables in order to avoid object creations.
+    //Those are used in synchronized methods. 
+    private MutableLong tempResourceIdForRegister;
+    private MutableLong tempResourceIdForSetLSN;
+    private LogRecord logRecord;
+
+    //TODO: implement transactionContext pool in order to avoid object creations.
+    //      also, the pool can throttle the number of concurrent active jobs at every moment. 
     public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
         this.jobId = jobId;
         this.transactionSubsystem = transactionSubsystem;
-        init();
+        firstLSN = new AtomicLong(-1);
+        lastLSN = new AtomicLong(-1);
+        txnState = new AtomicInteger(ITransactionManager.ACTIVE);
+        isTimeout = false;
+        isWriteTxn = new AtomicBoolean(false);
+        isMetadataTxn = false;
+        indexMap = new HashMap<MutableLong, BaseOperationTracker>();
+        primaryIndex = null;
+        tempResourceIdForRegister = new MutableLong();
+        tempResourceIdForSetLSN = new MutableLong();
+        logRecord = new LogRecord();
     }
 
-    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) {
-                ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(lastLSN);
+    public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
+            boolean isPrimaryIndex) {
+        synchronized (indexMap) {
+            if (isPrimaryIndex && primaryIndex == null) {
+                primaryIndex = index;
+                primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
+                primaryIndexOpTracker = (PrimaryIndexOperationTracker) index.getOperationTracker();
+            }
+            tempResourceIdForRegister.set(resourceId);
+            if (!indexMap.containsKey(tempResourceIdForRegister)) {
+                indexMap.put(new MutableLong(resourceId), ((BaseOperationTracker) index.getOperationTracker()));
             }
         }
     }
 
-    public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException {
-        synchronized (indexes) {
-            Set<BaseOperationTracker> opTrackers = new HashSet<BaseOperationTracker>();
-            Iterator<ILSMIndex> indexIt = indexes.iterator();
-            Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
-            while (indexIt.hasNext()) {
-                ILSMIndex index = indexIt.next();
-                opTrackers.add((BaseOperationTracker) index.getOperationTracker());
-                assert cbIt.hasNext();
-            }
-            Iterator<BaseOperationTracker> trackerIt = opTrackers.iterator();
-            while (trackerIt.hasNext()) {
-                IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
-                BaseOperationTracker opTracker = (BaseOperationTracker) trackerIt.next();
-                if (exlusiveJobLevelCommit) {
-                    opTracker.exclusiveJobCommitted();
-                } else {
-                    opTracker.completeOperation(null, LSMOperationType.MODIFICATION, null, modificationCallback);
-                }
-            }
+    //[Notice] 
+    //This method is called sequentially by the LogAppender threads. 
+    //However, the indexMap is concurrently read and modified through this method and registerIndexAndCallback()
+    //TODO: fix issues - 591, 609, 612, and 614.
+    @Override
+    public void setLastLSN(long resourceId, long LSN) {
+        synchronized (indexMap) {
+            firstLSN.compareAndSet(-1, LSN);
+            lastLSN.set(Math.max(lastLSN.get(), LSN));
+            tempResourceIdForSetLSN.set(resourceId);
+            //TODO; create version number tracker and keep LSNs there. 
+            BaseOperationTracker opTracker = indexMap.get(tempResourceIdForSetLSN);
+            opTracker.updateLastLSN(LSN);
         }
     }
 
     @Override
-    public int getActiveOperationCountOnIndexes() throws HyracksDataException {
-        synchronized (indexes) {
-            int count = 0;
-            Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
-            while (cbIt.hasNext()) {
-                IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
-                count += ((AbstractOperationCallback) modificationCallback).getLocalNumActiveOperations();
+    public void notifyOptracker(boolean isJobLevelCommit) {
+        try {
+            if (isJobLevelCommit && isMetadataTxn) {
+                primaryIndexOpTracker.exclusiveJobCommitted();
+            } else if (!isJobLevelCommit) {
+                primaryIndexOpTracker
+                        .completeOperation(null, LSMOperationType.MODIFICATION, null, primaryIndexCallback);
             }
-            return count;
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
         }
     }
 
-    public void setTransactionType(TransactionType transactionType) {
-        this.transactionType = transactionType;
+    public void setWriteTxn(boolean isWriteTxn) {
+        this.isWriteTxn.set(isWriteTxn);
     }
 
-    public TransactionType getTransactionType() {
-        return transactionType;
+    public boolean isWriteTxn() {
+        return isWriteTxn.get();
     }
 
-    public void addCloseableResource(ICloseable resource) {
-        resources.add(resource);
+    @Override
+    public long getFirstLSN() {
+        return firstLSN.get();
     }
 
-    public LogicalLogLocator getFirstLogLocator() {
-        return firstLogLocator;
-    }
-
-    public LogicalLogLocator getLastLogLocator() {
-        return lastLogLocator;
-    }
-
-    public void setLastLSN(long lsn) {
-        if (firstLogLocator.getLsn() == -1) {
-            firstLogLocator.setLsn(lsn);
-        }
-        lastLogLocator.setLsn(lsn);
+    @Override
+    public long getLastLSN() {
+        return lastLSN.get();
     }
 
     public JobId getJobId() {
         return jobId;
     }
 
-    public void setStartWaitTime(long time) {
-        this.startWaitTime = time;
+    public void setTimeout(boolean isTimeout) {
+        this.isTimeout = isTimeout;
     }
 
-    public long getStartWaitTime() {
-        return startWaitTime;
+    public boolean isTimeout() {
+        return isTimeout;
     }
 
-    public void setStatus(int status) {
-        this.status = status;
+    public void setTxnState(int txnState) {
+        this.txnState.set(txnState);
     }
 
-    public int getStatus() {
-        return status;
-    }
-
-    public void setTxnState(TransactionState txnState) {
-        this.txnState = txnState;
-    }
-
-    public TransactionState getTxnState() {
-        return txnState;
-    }
-
-    public void releaseResources() throws ACIDException {
-        for (ICloseable closeable : resources) {
-            closeable.close(this);
-        }
+    public int getTxnState() {
+        return txnState.get();
     }
 
     @Override
@@ -198,19 +198,27 @@
     }
 
     @Override
-    public void setExclusiveJobLevelCommit() {
-        exlusiveJobLevelCommit = true;
+    public void setMetadataTransaction(boolean isMetadataTxn) {
+        this.isMetadataTxn = isMetadataTxn;
+    }
+
+    @Override
+    public boolean isMetadataTransaction() {
+        return isMetadataTxn;
     }
 
     public String prettyPrint() {
         StringBuilder sb = new StringBuilder();
         sb.append("\n" + jobId + "\n");
-        sb.append("transactionType: " + transactionType);
-        sb.append("firstLogLocator: " + firstLogLocator.getLsn() + "\n");
-        sb.append("lastLogLocator: " + lastLogLocator.getLsn() + "\n");
+        sb.append("isWriteTxn: " + isWriteTxn + "\n");
+        sb.append("firstLSN: " + firstLSN.get() + "\n");
+        sb.append("lastLSN: " + lastLSN.get() + "\n");
         sb.append("TransactionState: " + txnState + "\n");
-        sb.append("startWaitTime: " + startWaitTime + "\n");
-        sb.append("status: " + status + "\n");
+        sb.append("isTimeout: " + isTimeout + "\n");
         return sb.toString();
     }
+
+    public LogRecord getLogRecord() {
+        return logRecord;
+    }
 }
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 54ec8a1..01b38c2 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,9 +16,9 @@
 
 import java.io.IOException;
 import java.io.OutputStream;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -28,8 +28,7 @@
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 
 /**
@@ -40,115 +39,73 @@
 
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
-    private final TransactionSubsystem transactionProvider;
-    private Map<JobId, ITransactionContext> transactionContextRepository = new HashMap<JobId, ITransactionContext>();
+    private final TransactionSubsystem txnSubsystem;
+    private Map<JobId, ITransactionContext> transactionContextRepository = new ConcurrentHashMap<JobId, ITransactionContext>();
     private AtomicInteger maxJobId = new AtomicInteger(0);
 
     public TransactionManager(TransactionSubsystem provider) {
-        this.transactionProvider = provider;
+        this.txnSubsystem = provider;
     }
 
     @Override
-    public void abortTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
-            throws ACIDException {
-        synchronized (txnContext) {
-            if (txnContext.getTxnState().equals(TransactionState.ABORTED)) {
-                return;
+    public void abortTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+        if (txnCtx.getTxnState() != ITransactionManager.ABORTED) {
+            txnCtx.setTxnState(ITransactionManager.ABORTED);
+        }
+        try {
+            txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+        } catch (Exception ae) {
+            String msg = "Could not complete rollback! System is in an inconsistent state";
+            if (LOGGER.isLoggable(Level.SEVERE)) {
+                LOGGER.severe(msg);
             }
-
-            try {
-                transactionProvider.getRecoveryManager().rollbackTransaction(txnContext);
-            } catch (Exception ae) {
-                String msg = "Could not complete rollback! System is in an inconsistent state";
-                if (LOGGER.isLoggable(Level.SEVERE)) {
-                    LOGGER.severe(msg);
-                }
-                ae.printStackTrace();
-                throw new Error(msg);
-            } finally {
-                txnContext.releaseResources();
-                transactionProvider.getLockManager().releaseLocks(txnContext);
-                transactionContextRepository.remove(txnContext.getJobId());
-                txnContext.setTxnState(TransactionState.ABORTED);
-            }
+            ae.printStackTrace();
+            throw new ACIDException(msg, ae);
+        } finally {
+            txnSubsystem.getLockManager().releaseLocks(txnCtx);
+            transactionContextRepository.remove(txnCtx.getJobId());
         }
     }
 
     @Override
     public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
-        setMaxJobId(jobId.getId());
-        ITransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
-        synchronized (this) {
-            transactionContextRepository.put(jobId, txnContext);
-        }
-        return txnContext;
+        return getTransactionContext(jobId);
     }
 
     @Override
     public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException {
         setMaxJobId(jobId.getId());
-        synchronized (transactionContextRepository) {
-
-            ITransactionContext context = transactionContextRepository.get(jobId);
-            if (context == null) {
-                context = transactionContextRepository.get(jobId);
-                context = new TransactionContext(jobId, transactionProvider);
-                transactionContextRepository.put(jobId, context);
+        ITransactionContext txnCtx = transactionContextRepository.get(jobId);
+        if (txnCtx == null) {
+            synchronized (this) {
+                txnCtx = transactionContextRepository.get(jobId);
+                if (txnCtx == null) {
+                    txnCtx = new TransactionContext(jobId, txnSubsystem);
+                    transactionContextRepository.put(jobId, txnCtx);
+                }
             }
-            return context;
         }
+        return txnCtx;
     }
 
     @Override
-    public void commitTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
-            throws ACIDException {
-        synchronized (txnContext) {
-            if ((txnContext.getTxnState().equals(TransactionState.COMMITTED))) {
-                return;
+    public void commitTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+        //Only job-level commits call this method. 
+        try {
+            if (txnCtx.isWriteTxn()) {
+                LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
+                logRecord.formJobCommitLogRecord(txnCtx);
+                txnSubsystem.getLogManager().log(logRecord);
             }
-
-            //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) {
-                boolean countIsZero = transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext,
-                        true);
-                if (!countIsZero) {
-                    // Lock count != 0 for a particular entity implies that the entity has been locked 
-                    // more than once (probably due to a hash collision in our current model).
-                    // It is safe to decrease the active transaction count on indexes since,  
-                    // by virtue of the counter not being zero, there is another transaction 
-                    // that has increased the transaction count. Thus, decreasing it will not 
-                    // allow the data to be flushed (yet). The flush will occur when the log page
-                    // flush thread decides to decrease the count for the last time.
-                    try {
-                        //decrease the transaction reference count on index
-                        txnContext.decreaseActiveTransactionCountOnIndexes();
-                    } catch (HyracksDataException e) {
-                        throw new ACIDException("failed to complete index operation", e);
-                    }
-                }
-                return;
+        } catch (Exception ae) {
+            if (LOGGER.isLoggable(Level.SEVERE)) {
+                LOGGER.severe(" caused exception in commit !" + txnCtx.getJobId());
             }
-
-            //for job-level commit
-            try {
-                if (txnContext.getTransactionType().equals(ITransactionContext.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.getJobId());
-                }
-                throw ae;
-            } finally {
-                txnContext.releaseResources();
-                transactionProvider.getLockManager().releaseLocks(txnContext); // release
-                transactionContextRepository.remove(txnContext.getJobId());
-                txnContext.setTxnState(TransactionState.COMMITTED);
-            }
+            throw ae;
+        } finally {
+            txnSubsystem.getLockManager().releaseLocks(txnCtx); // release
+            transactionContextRepository.remove(txnCtx.getJobId());
+            txnCtx.setTxnState(ITransactionManager.COMMITTED);
         }
     }
 
@@ -164,11 +121,14 @@
 
     @Override
     public TransactionSubsystem getTransactionProvider() {
-        return transactionProvider;
+        return txnSubsystem;
     }
 
     public void setMaxJobId(int jobId) {
-        maxJobId.set(Math.max(maxJobId.get(), jobId));
+        int maxId = maxJobId.get();
+        if (jobId > maxId) {
+            maxJobId.compareAndSet(maxId, jobId);
+        }
     }
 
     public int getMaxJobId() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index b76cf11..aceeb82 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -22,9 +22,7 @@
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
 import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
-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.recovery.CheckpointThread;
 import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
@@ -39,8 +37,6 @@
     private final ILockManager lockManager;
     private final ITransactionManager transactionManager;
     private final IRecoveryManager recoveryManager;
-    private final TransactionalResourceManagerRepository resourceRepository;
-    private final IndexLoggerRepository loggerRepository;
     private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
     private final CheckpointThread checkpointThread;
     private final AsterixTransactionProperties txnProperties;
@@ -50,11 +46,9 @@
         this.id = id;
         this.txnProperties = txnProperties;
         this.transactionManager = new TransactionManager(this);
-        this.logManager = new LogManager(this);
         this.lockManager = new LockManager(this);
+        this.logManager = new LogManager(this);
         this.recoveryManager = new RecoveryManager(this);
-        this.loggerRepository = new IndexLoggerRepository(this);
-        this.resourceRepository = new TransactionalResourceManagerRepository();
         this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
         if (asterixAppRuntimeContextProvider != null) {
             this.checkpointThread = new CheckpointThread(recoveryManager,
@@ -81,14 +75,6 @@
         return recoveryManager;
     }
 
-    public TransactionalResourceManagerRepository getTransactionalResourceRepository() {
-        return resourceRepository;
-    }
-
-    public IndexLoggerRepository getTreeLoggerRepository() {
-        return loggerRepository;
-    }
-
     public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider() {
         return asterixAppRuntimeContextProvider;
     }
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
deleted file mode 100644
index 06379b4..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.transaction.management.logging;
-
-import java.util.Random;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-
-public class BasicLogger implements ILogger {
-
-    private static long averageContentCreationTime = 0;
-    private static long count = 0;
-
-    public void log(ITransactionContext context, LogicalLogLocator wMemLSN, int length,
-            ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-
-        byte[] logContent = getRandomBytes(length);
-        try {
-            long startTime2 = System.nanoTime();
-
-            IBuffer buffer = (IBuffer) (wMemLSN.getBuffer());
-
-            /*
-             * synchronized(buffer){ buffer.position(wMemLSN.getMemoryOffset());
-             * buffer.put(logContent); }
-             */
-
-            byte[] logPageStorage = buffer.getArray();
-            System.arraycopy(logContent, 0, logPageStorage, wMemLSN.getMemoryOffset(), logContent.length);
-
-            /*
-             * for(int i=0;i<logContent.length;i++){
-             * ((IFileBasedBuffer)(wMemLSN.
-             * getBuffer())).put(wMemLSN.getMemoryOffset() + i, logContent[i]);
-             * }
-             */
-            long endTime2 = System.nanoTime();
-            averageContentCreationTime = ((averageContentCreationTime * count) + (endTime2 - startTime2)) / (++count);
-        } catch (Exception e) {
-            throw new ACIDException("", e);
-        }
-    }
-
-    public static long getAverageContentCreationTime() {
-        return averageContentCreationTime;
-    }
-
-    public static long getNumLogs() {
-        return averageContentCreationTime;
-    }
-
-    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    private static byte[] getRandomBytes(int size) {
-        byte[] b = new byte[size];
-        Random random = new Random();
-        int num = random.nextInt(30);
-        Integer number = (new Integer(num + 65));
-        byte numByte = number.byteValue();
-        for (int i = 0; i < size; i++) {
-            b[i] = numByte;
-        }
-        return b;
-    }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
deleted file mode 100644
index 8b9c970..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * 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.transaction.management.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogger;
-
-public interface IResource {
-
-    public byte[] getId();
-
-    public ILogger getLogger();
-}
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
deleted file mode 100644
index e4a8e45..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * 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.transaction.management.logging.test;
-
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class RecoverySimulator {
-
-    private static IRecoveryManager recoveryManager;
-
-    public static void startRecovery() throws IOException, ACIDException {
-        recoveryManager.startRecovery(true);
-    }
-
-    public static void main(String args[]) throws IOException, ACIDException, AsterixException {
-        String id = "nc1";
-        try {
-            TransactionSubsystem factory = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
-                    new AsterixPropertiesAccessor()));
-            IRecoveryManager recoveryManager = factory.getRecoveryManager();
-            recoveryManager.startRecovery(true);
-        } catch (ACIDException acide) {
-            acide.printStackTrace();
-            throw 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
deleted file mode 100644
index a74b079..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * 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.transaction.management.logging.test;
-
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.BasicLogger;
-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.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class TransactionWorkloadSimulator {
-
-    public static ILogManager logManager;
-    public static ILockManager lockManager;
-    TransactionSubsystem provider;
-
-    public static WorkloadProperties workload;
-    Transaction[] transactions;
-
-    public TransactionWorkloadSimulator(WorkloadProperties workload) {
-        this.workload = workload;
-        transactions = new Transaction[workload.numActiveThreads];
-    }
-
-    public void beginWorkload() throws ACIDException, AsterixException {
-        provider = new TransactionSubsystem("nc1", null, new AsterixTransactionProperties(
-                new AsterixPropertiesAccessor()));
-        logManager = provider.getLogManager();
-        lockManager = provider.getLockManager();
-        provider.getTransactionalResourceRepository().registerTransactionalResourceManager(DummyResourceMgr.id,
-                new DummyResourceMgr());
-        Transaction[] transactions = new Transaction[workload.numActiveThreads];
-        long startTime = System.nanoTime();
-        for (int i = 0; i < workload.numActiveThreads; i++) {
-            transactions[i] = new Transaction(provider, "Transaction " + (i + 1), workload.singleTransaction);
-            transactions[i].start();
-        }
-        for (int i = 0; i < workload.numActiveThreads; i++) {
-            try {
-                transactions[i].join();
-            } catch (InterruptedException ignore) {
-            }
-        }
-
-        for (int i = 0; i < workload.numActiveThreads; i++) {
-            provider.getTransactionManager().commitTransaction(transactions[i].getContext(), new DatasetId(-1), -1);
-        }
-
-        long endTime = System.nanoTime();
-        int totalLogs = Transaction.logCount.get();
-        System.out.println(" Total logs :" + totalLogs);
-        long timeTaken = ((endTime - startTime) / 1000000);
-        System.out.println(" total time :" + timeTaken);
-        System.out.println(" throughput :" + totalLogs * 1000 / timeTaken + " logs/sec");
-        long totalBytesWritten = Transaction.logByteCount.get();
-        System.out.println(" bytes written :" + totalBytesWritten);
-        System.out.println(" IO throughput " + totalBytesWritten * 1000 / timeTaken + " bytes/sec");
-        System.out.println(" Avg Content Creation time :" + BasicLogger.getAverageContentCreationTime());
-    }
-
-    public static void main(String args[]) throws AsterixException {
-        WorkloadProperties workload = new WorkloadProperties();
-        TransactionWorkloadSimulator simulator = new TransactionWorkloadSimulator(workload);
-        try {
-            simulator.beginWorkload();
-        } catch (ACIDException acide) {
-            acide.printStackTrace();
-        }
-
-    }
-}
-
-class SingleTransactionContextFactory {
-    private static TransactionContext context;
-
-    public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
-        if (context == null) {
-            context = new TransactionContext(JobIdFactory.generateJobId(), provider);
-        }
-        return context;
-    }
-}
-
-class MultipleTransactionContextFactory {
-
-    public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
-        return new TransactionContext(JobIdFactory.generateJobId(), provider);
-    }
-}
-
-class Transaction extends Thread {
-
-    public static AtomicInteger logCount = new AtomicInteger(0);
-    public static AtomicLong logByteCount = new AtomicLong(0);
-    Random random = new Random();
-    BasicLogger logger = new BasicLogger();
-    LogicalLogLocator memLSN;
-    String name;
-    TransactionContext context;
-    //private byte[] resourceID = new byte[1];
-    private int resourceID;
-    private int myLogCount = 0;
-    private TransactionSubsystem transactionProvider;
-    private ILogManager logManager;
-    private DatasetId tempDatasetId = new DatasetId(-1);
-
-    public Transaction(TransactionSubsystem provider, String name, boolean singleTransaction) throws ACIDException {
-        this.name = name;
-        this.transactionProvider = provider;
-        if (singleTransaction) {
-            context = SingleTransactionContextFactory.getContext(transactionProvider);
-        } else {
-            context = MultipleTransactionContextFactory.getContext(transactionProvider);
-        }
-        memLSN = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
-        logManager = transactionProvider.getLogManager();
-    }
-
-    public TransactionContext getContext() {
-        return context;
-    }
-
-    @Override
-    public void run() {
-        if (TransactionWorkloadSimulator.workload.minLogsPerTransactionThread == TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread) {
-            TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread++;
-        }
-        int numLogs = TransactionWorkloadSimulator.workload.minLogsPerTransactionThread
-                + random.nextInt(TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread
-                        - TransactionWorkloadSimulator.workload.minLogsPerTransactionThread);
-        int total = 0;
-        LogicalLogLocator memLSN = LogUtil.getDummyLogicalLogLocator(logManager);
-        if (TransactionWorkloadSimulator.workload.maxLogSize == TransactionWorkloadSimulator.workload.minLogSize) {
-            TransactionWorkloadSimulator.workload.maxLogSize++;
-        }
-        if (TransactionWorkloadSimulator.workload.singleResource) {
-            int choice = random.nextInt(2);
-            resourceID = (byte) (choice % 2);
-        } else {
-            random.nextInt(resourceID);
-        }
-        boolean retry = false;
-        byte lockMode = -1;
-        try {
-            for (int i = 0; i < numLogs - 1; i++) {
-                int logSize = TransactionWorkloadSimulator.workload.minLogSize
-                        + random.nextInt(TransactionWorkloadSimulator.workload.maxLogSize
-                                - TransactionWorkloadSimulator.workload.minLogSize);
-                total += logSize;
-
-                byte logType = LogType.UPDATE;
-                byte logActionType = LogActionType.REDO_UNDO;
-                long pageId = 0;
-                if (!retry) {
-                    lockMode = (byte) (random.nextInt(2));
-                }
-                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.getLogRecordHelper().getLogHeaderSize(logType)
-                        + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogChecksumSize());
-                myLogCount++;
-            }
-        } catch (ACIDException acide) {
-            acide.printStackTrace();
-        } catch (Exception ie) {
-            ie.printStackTrace();
-        }
-    }
-
-}
-
-class WorkloadProperties {
-    public int numActiveThreads = 200;
-    public long thinkTime = 0; // (in mesecs)
-    public int minLogsPerTransactionThread = 5;
-    public int maxLogsPerTransactionThread = 5;
-    public int minLogSize = 1024 - 51;
-    public int maxLogSize = 1024 - 51;
-    public float commitFraction = 0.5f;
-    public float rollbackFraction = 0.1f;
-    public boolean singleTransaction = false;
-    public boolean singleResource = true;
-}
-
-class ResourceMgrInfo {
-    public static final byte BTreeResourceMgrId = 1;
-    public static final byte MetadataResourceMgrId = 2;
-}
-
-class DummyResourceMgr implements IResourceManager {
-
-    public static final byte id = 1;
-
-    @Override
-    public void redo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void undo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public byte getResourceManagerId() {
-        // TODO Auto-generated method stub
-        return 1;
-    }
-
-}
\ No newline at end of file
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
deleted file mode 100644
index 2eea43e..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * 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.transaction.management.test;
-
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileLogger implements ILogger {
-
-    IResource resource;
-    String logRecordContent;
-
-    public FileLogger(IResource resource) {
-        this.resource = resource;
-    }
-
-    public int generateLogRecordContent(int currentValue, int finalValue) {
-        StringBuilder builder = new StringBuilder();
-        builder.append("" + currentValue + " " + finalValue);
-        logRecordContent = new String(builder);
-        return resource.getId().length + logRecordContent.length();
-    }
-
-    @Override
-    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void log(ITransactionContext 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++) {
-            buffer[memLSN.getMemoryOffset() + i] = resource.getId()[i];
-        }
-        for (int i = 0; i < content.length; i++) {
-            buffer[memLSN.getMemoryOffset() + resource.getId().length + i] = content[i];
-        }
-    }
-
-    @Override
-    public void postLog(ITransactionContext 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/FileResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
deleted file mode 100644
index 072738f..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.transaction.management.test;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResource implements IResource {
-
-    private byte[] resourceId = new byte[] { 1 };
-    private ILogger logger;
-
-    private File file;
-    private int memCounter = 0;
-    private int diskCounter = 0;
-
-    public int getMemoryCounter() {
-        return memCounter;
-    }
-
-    public int getDiskCounter() {
-        return diskCounter;
-    }
-
-    public static enum CounterOperation {
-        INCREMENT,
-        DECREMENT,
-    };
-
-    public FileResource(String fileDir, String fileName) throws IOException {
-        File dirFile = new File(fileDir);
-        if (!dirFile.exists()) {
-            FileUtil.createNewDirectory(fileDir);
-        }
-        file = new File(fileDir + "/" + fileName);
-        if (!file.exists()) {
-            FileUtil.createFileIfNotExists(file.getAbsolutePath());
-            BufferedWriter writer = new BufferedWriter(new FileWriter(file));
-            writer.write("0");
-            writer.flush();
-        } else {
-            FileReader fileReader = new FileReader(file);
-            BufferedReader bufferedReader = new BufferedReader(fileReader);
-            String content = bufferedReader.readLine();
-            diskCounter = Integer.parseInt(content);
-        }
-        logger = new FileLogger(this);
-    }
-
-    public synchronized void increment() {
-        memCounter++;
-    }
-
-    public synchronized void decrement() {
-        memCounter--;
-    }
-
-    public synchronized void setValue(int value) {
-        memCounter = value;
-    }
-
-    public synchronized void sync() throws IOException {
-        BufferedWriter writer = new BufferedWriter(new FileWriter(file));
-        writer.write("" + memCounter);
-        writer.flush();
-    }
-
-    public synchronized boolean checkIfValueInSync(int expectedValue) throws IOException {
-        FileReader fileReader = new FileReader(file);
-        BufferedReader bufferedReader = new BufferedReader(fileReader);
-        String content = bufferedReader.readLine();
-        return content.equals("" + expectedValue);
-    }
-
-    @Override
-    public byte[] getId() {
-        return resourceId;
-    }
-
-    @Override
-    public ILogger getLogger() {
-        return logger;
-    }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
deleted file mode 100644
index 5a6e408..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * 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.transaction.management.test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResourceManager implements IResourceManager {
-
-    public static final byte id = 100;
-    private Map<String, FileResource> transactionalResources = new HashMap<String, FileResource>();
-
-    public void registerTransactionalResource(IResource resource) throws ACIDException {
-        if (resource instanceof FileResource) {
-            if (transactionalResources.get(new String(resource.getId())) == null) {
-                transactionalResources.put(new String(resource.getId()), (FileResource) resource);
-            }
-        } else {
-            throw new ACIDException(" invalid resource type :" + resource);
-        }
-    }
-
-    @Override
-    public byte getResourceManagerId() {
-        return id;
-    }
-
-    @Override
-    public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
-        LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
-        FileResource fileManager = transactionalResources.get(logRecordInfo.getResourceId());
-        if (fileManager == null) {
-            throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
-        }
-        fileManager.setValue(logRecordInfo.getBeforeValue());
-    }
-
-    @Override
-    public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
-        LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
-        FileResource fileManager = transactionalResources.get(new String(logRecordInfo.getResourceId()));
-        if (fileManager == null) {
-            throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
-        }
-        fileManager.setValue(logRecordInfo.getAfterValue());
-    }
-
-}
-
-class LogRecordInfo {
-
-    byte[] resourceId;
-    int beforeValue;
-    int afterValue;
-
-    public LogRecordInfo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
-        int logContentBeginPos = logParser.getLogContentBeginPos(memLSN);
-        int logContentEndPos = logParser.getLogContentEndPos(memLSN);
-        byte[] bufferContent = memLSN.getBuffer().getArray();
-        resourceId = new byte[] { bufferContent[logContentBeginPos] };
-        String content = new String(bufferContent, logContentBeginPos + resourceId.length, logContentEndPos
-                - (logContentBeginPos + resourceId.length));
-        beforeValue = Integer.parseInt(content.split(" ")[0]);
-        afterValue = Integer.parseInt(content.split(" ")[1]);
-    }
-
-    public byte[] getResourceId() {
-        return resourceId;
-    }
-
-    public void setResourceId(byte[] resourceId) {
-        this.resourceId = resourceId;
-    }
-
-    public int getAfterValue() {
-        return afterValue;
-    }
-
-    public void setFinalValue(int afterValue) {
-        this.afterValue = afterValue;
-    }
-
-    public int getBeforeValue() {
-        return beforeValue;
-    }
-
-    public void setBeforeValue(int beforeValue) {
-        this.beforeValue = beforeValue;
-    }
-
-}
\ No newline at end of file
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
deleted file mode 100644
index 8e13356..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.transaction.management.test;
-
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class LogRecordReader {
-
-    ILogManager logManager;
-
-    public LogRecordReader(TransactionSubsystem factory) throws ACIDException {
-        logManager = factory.getLogManager();
-    }
-
-    public LogRecordReader(ILogManager logManager) {
-        this.logManager = logManager;
-    }
-
-    public void readLogs(long startingLsn) throws IOException, ACIDException {
-        ILogRecordHelper parser = logManager.getLogRecordHelper();
-        PhysicalLogLocator lsn = new PhysicalLogLocator(startingLsn, logManager);
-        ILogCursor logCursor = logManager.readLog(lsn, new ILogFilter() {
-            @Override
-            public boolean accept(IBuffer buffer, long startOffset, int length) {
-                return true;
-            }
-        });
-        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-        int logCount = 0;
-        while (true) {
-            boolean logValidity = logCursor.next(currentLogLocator);
-            if (logValidity) {
-                System.out.println(++logCount + parser.getLogRecordForDisplay(currentLogLocator));
-            } else {
-                break;
-            }
-        }
-    }
-
-    public void readLogRecord(long lsnValue) throws IOException, ACIDException {
-        LogicalLogLocator memLSN = null;
-        logManager.readLog(lsnValue, memLSN);
-        System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(memLSN));
-    }
-
-    /**
-     * @param args
-     */
-    public static void main(String[] args) throws ACIDException, Exception {
-        LogManager logManager = new LogManager(null, "nc1");
-        LogRecordReader logReader = new LogRecordReader(logManager);
-        logReader.readLogs(0);
-    }
-
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
deleted file mode 100644
index 5195658..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * 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.transaction.management.test;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-public class TransactionRecoverySimulator {
-
-    public boolean recoverSystem() throws ACIDException {
-        return true;
-    }
-
-    public static void main(String args[]) {
-        TransactionRecoverySimulator recoverySimulator = new TransactionRecoverySimulator();
-    }
-}
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
deleted file mode 100644
index 4bebbc4..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * 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.transaction.management.test;
-
-import java.io.IOException;
-import java.util.Random;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager;
-import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class TransactionSimulator {
-
-    private ITransactionManager transactionManager;
-    private ILogManager logManager;
-    private ILockManager lockManager;
-    private IRecoveryManager recoveryManager;
-    private IResourceManager resourceMgr;
-    private ILogger logger;
-    private IResource resource;
-    private LogicalLogLocator memLSN;
-    private TransactionSubsystem transactionProvider;
-
-    public TransactionSimulator(IResource resource, IResourceManager resourceMgr) throws ACIDException, AsterixException {
-        String id = "nc1";
-        transactionProvider = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
-                new AsterixPropertiesAccessor()));
-        transactionManager = transactionProvider.getTransactionManager();
-        logManager = transactionProvider.getLogManager();
-        lockManager = transactionProvider.getLockManager();
-        recoveryManager = transactionProvider.getRecoveryManager();
-        transactionProvider.getTransactionalResourceRepository().registerTransactionalResourceManager(
-                resourceMgr.getResourceManagerId(), resourceMgr);
-        this.resourceMgr = resourceMgr;
-        this.logger = resource.getLogger();
-        this.resource = resource;
-        memLSN = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
-    }
-
-    public ITransactionContext beginTransaction() throws ACIDException {
-        JobId jobId = JobIdFactory.generateJobId();
-        return transactionManager.beginTransaction(jobId);
-    }
-
-    public void executeTransactionOperation(ITransactionContext txnContext, FileResource.CounterOperation operation)
-            throws ACIDException {
-        // lockManager.lock(txnContext, resourceId, 0);
-        ILogManager logManager = transactionProvider.getLogManager();
-        int currentValue = ((FileResource) resource).getMemoryCounter();
-        int finalValue;
-        switch (operation) {
-            case INCREMENT:
-                finalValue = currentValue + 1;
-                int logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
-                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(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
-                ((FileResource) resource).decrement();
-                break;
-        }
-
-    }
-
-    public void commitTransaction(ITransactionContext context) throws ACIDException {
-        transactionManager.commitTransaction(context, new DatasetId(-1), -1);
-    }
-
-    public void recover() throws ACIDException, IOException {
-        recoveryManager.startRecovery(true);
-        ((FileResource) resource).sync();
-    }
-
-    /**
-     * @param args
-     */
-    public static void main(String[] args) throws IOException, ACIDException, AsterixException {
-        String fileDir = "testdata";
-        String fileName = "counterFile";
-        IResource resource = new FileResource(fileDir, fileName);
-        FileResourceManager resourceMgr = new FileResourceManager();
-        resourceMgr.registerTransactionalResource(resource);
-        int existingValue = ((FileResource) resource).getDiskCounter();
-
-        TransactionSimulator txnSimulator = new TransactionSimulator(((FileResource) resource), resourceMgr);
-        int numTransactions = 2;
-        Schedule schedule = new Schedule(numTransactions);
-
-        for (int i = 0; i < numTransactions; i++) {
-            ITransactionContext context = txnSimulator.beginTransaction();
-            txnSimulator.executeTransactionOperation(context, schedule.getOperations()[i]);
-            if (schedule.getWillCommit()[i]) {
-                txnSimulator.commitTransaction(context);
-            }
-        }
-
-        int finalExpectedValue = existingValue + schedule.getDeltaChange();
-        txnSimulator.recover();
-        boolean isCorrect = ((FileResource) resource).checkIfValueInSync(finalExpectedValue);
-        System.out.println(" Did recovery happen correctly " + isCorrect);
-    }
-
-}
-
-class ResourceMgrIds {
-
-    public static final byte FileResourceMgrId = 1;
-
-}
-
-class Schedule {
-
-    private int numCommittedIncrements;
-    private int numCommittedDecrements;
-
-    private FileResource.CounterOperation[] operations;
-    private Boolean[] willCommit;
-
-    public Boolean[] getWillCommit() {
-        return willCommit;
-    }
-
-    private Random random = new Random();
-
-    public int getDeltaChange() {
-        return numCommittedIncrements - numCommittedDecrements;
-    }
-
-    public Schedule(int numTransactions) {
-        operations = new FileResource.CounterOperation[numTransactions];
-        willCommit = new Boolean[numTransactions];
-        for (int i = 0; i < numTransactions; i++) {
-            willCommit[i] = random.nextBoolean();
-            int nextOp = random.nextInt(2);
-            FileResource.CounterOperation op = nextOp == 0 ? FileResource.CounterOperation.INCREMENT
-                    : FileResource.CounterOperation.DECREMENT;
-            operations[i] = op;
-            if (willCommit[i]) {
-                if (op.equals(FileResource.CounterOperation.INCREMENT)) {
-                    numCommittedIncrements++;
-                } else {
-                    numCommittedDecrements++;
-                }
-            }
-        }
-
-    }
-
-    public String toString() {
-        StringBuilder builder = new StringBuilder();
-        for (int i = 0; i < operations.length; i++) {
-            builder.append(" operation " + operations[i]);
-            if (willCommit[i]) {
-                builder.append(" commit ");
-            } else {
-                builder.append(" abort ");
-            }
-        }
-
-        builder.append(" number of committed increments " + numCommittedIncrements);
-        builder.append(" number of committed decrements " + numCommittedDecrements);
-        return new String(builder);
-    }
-
-    public FileResource.CounterOperation[] getOperations() {
-        return operations;
-    }
-
-}