checkpoint towards fixing diskComponents' LSN
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 693a19e..ba940d6 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,7 +39,6 @@
 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;
@@ -388,13 +387,6 @@
     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 {
@@ -446,5 +438,4 @@
     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 6dd11bd..32bfa58 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
@@ -72,7 +72,7 @@
     @Override
     public void open() throws HyracksDataException {
         try {
-            transactionContext = transactionManager.getTransactionContext(jobId);
+            transactionContext = transactionManager.getTransactionContext(jobId, false);
             transactionContext.setWriteTxn(isWriteTransaction);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
@@ -89,7 +89,11 @@
             pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
             logRecord.formEntityCommitLogRecord(transactionContext, datasetId, pkHash, frameTupleReference,
                     primaryKeyFields);
-            logMgr.log(logRecord);
+            try {
+                logMgr.log(logRecord);
+            } catch (ACIDException e) {
+                throw new HyracksDataException(e);
+            }
         }
     }
 
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
deleted file mode 100644
index 447555e..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
+++ /dev/null
@@ -1,110 +0,0 @@
-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/optimizer/rules/am/AccessMethodUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
index 95d29e3..ab0fd79 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,12 +22,8 @@
 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;
@@ -38,7 +34,6 @@
 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;
@@ -57,7 +52,6 @@
 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;
@@ -66,384 +60,290 @@
  * 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 void appendExternalRecTypes(Dataset dataset, IAType itemType, List<Object> target){
-		target.add(itemType);
-	}
+    public static ConstantExpression createStringConstant(String str) {
+        return new ConstantExpression(new AsterixConstantValue(new AString(str)));
+    }
 
-	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 createInt32Constant(int i) {
+        return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
+    }
 
-	public static ConstantExpression createStringConstant(String str) {
-		return new ConstantExpression(new AsterixConstantValue(new AString(str)));
-	}
+    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 createInt32Constant(int i) {
-		return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
-	}
+    public static String getStringConstant(Mutable<ILogicalExpression> expr) {
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((AString) obj).getStringValue();
+    }
 
-	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 int getInt32Constant(Mutable<ILogicalExpression> expr) {
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((AInt32) obj).getIntegerValue();
+    }
 
-	public static String getStringConstant(Mutable<ILogicalExpression> expr) {
-		IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
-		return ((AString) obj).getStringValue();
-	}
+    public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((ABoolean) obj).getBoolean();
+    }
 
-	public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
-		IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
-		return ((AInt32) obj).getIntegerValue();
-	}
+    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 boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
-		IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
-		return ((ABoolean) obj).getBoolean();
-	}
+    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 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 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 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;
-	}
+    /**
+     * 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 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 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());
+        }
+    }
 
-	/**
-	 * 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);
-				}
-			}
-		}
-	}
+    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;
+    }
 
-	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 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 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;
-	}
+    /**
+     * 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);
+    }
 
-	/**
-	 * 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 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 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;
-	}
+    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 ce2a1f7..ddcf768 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeAccessMethod.java
@@ -27,7 +27,6 @@
 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;
@@ -52,7 +51,6 @@
 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;
 
@@ -418,16 +416,10 @@
         // 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(dataset.getDatasetType() == DatasetType.EXTERNAL)
-    	{
-        	ExternalDataAccessByRIDOperator externalDataAccessOp = AccessMethodUtils.createExternalDataAccessByRIDUnnestMap(dataSourceScan, dataset, 
-    				recordType, secondaryIndexUnnestOp, context, chosenIndex);
-    		indexSubTree.dataSourceScanRef.setValue(externalDataAccessOp);
-    		return externalDataAccessOp;
-    	}
-        else if (!isPrimaryIndex) {
+        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 ea2adad..28aee7a 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,8 +114,7 @@
         if (dataset == null) {
             throw new AlgebricksException("No metadata for dataset " + datasetName);
         }
-        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED
-        		&& dataset.getDatasetType() != DatasetType.EXTERNAL	) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
             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 8af4ac1..b8125aa 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,7 +21,6 @@
 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;
@@ -45,7 +44,6 @@
 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;
 
@@ -198,19 +196,10 @@
         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.
-        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);
+        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/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index 9c3a853..f22d2fb 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
@@ -414,8 +414,7 @@
                 case EXTERNAL: {
                     String adapter = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getAdapter();
                     Map<String, String> properties = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getProperties();
-                    String ngName = ((ExternalDetailsDecl) dd.getDatasetDetailsDecl()).getNodegroupName().getValue();
-                    datasetDetails = new ExternalDatasetDetails(adapter, properties,ngName);
+                    datasetDetails = new ExternalDatasetDetails(adapter, properties);
                     break;
                 }
                 case FEED: {
@@ -591,18 +590,6 @@
             //#. 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);
@@ -667,8 +654,6 @@
                     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 {
@@ -745,7 +730,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 || dsType == DatasetType.EXTERNAL) {
+                if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
 
                     List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName,
                             datasetName);
@@ -858,7 +843,7 @@
                 }
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED || ds.getDatasetType() == DatasetType.EXTERNAL) {
+            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
 
                 //#. prepare jobs to drop the datatset and the indexes in NC
                 List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
@@ -959,7 +944,7 @@
                         + dataverseName);
             }
 
-            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED || ds.getDatasetType() == DatasetType.EXTERNAL) {
+            if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
                 indexName = stmtIndexDrop.getIndexName().getValue();
                 Index index = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
                 if (index == null) {
@@ -997,8 +982,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);
 
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 e090c2c..bf31db7 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,18 +14,14 @@
  */
 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;
@@ -60,16 +56,6 @@
         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 {
         String dataverseName = indexDropStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
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 55cccbc..3ae5f6d 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
@@ -16,31 +16,21 @@
 
 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.LSMBTreeIOOperationCallbackFactory;
-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.opcallbacks.SecondaryIndexOperationTrackerProvider;
 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;
@@ -62,6 +52,7 @@
     @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(
@@ -69,6 +60,7 @@
                 dataset.getDatasetId());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
+
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
@@ -87,110 +79,49 @@
 
     @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);
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
-                //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");
-            }
+        // Create dummy key provider for feeding the primary index scan. 
+        AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
 
-            // 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 primary index scan op.
+        BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
 
-            // 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.RUNTIME_PROVIDER,
-                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
-                                    .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
-            IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(
-                    dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
+        // Assign op.
+        AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
 
-            //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(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMBTreeIOOperationCallbackFactory.INSTANCE, 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;
+        // 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(
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE, 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;
     }
 }
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 1418c32..9f80568 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
@@ -29,9 +29,6 @@
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.JobId;
-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.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -39,17 +36,14 @@
 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.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
 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.formats.NonTaggedDataFormat;
 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;
@@ -109,11 +103,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;
@@ -122,6 +116,7 @@
     protected int[] secondaryBloomFilterKeyFields;
     protected RecordDescriptor secondaryRecDesc;
     protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
+
     protected IAsterixPropertiesProvider propertiesProvider;
 
     // Prevent public construction. Should be created via createIndexCreator().
@@ -176,55 +171,25 @@
             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);
+            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 {
@@ -309,196 +274,6 @@
         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;
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 a3261e6..72d0d70 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
@@ -296,10 +296,4 @@
                     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 ebfee66..4d8118d 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,47 +18,36 @@
 
 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;
@@ -160,159 +149,47 @@
     }
 
     @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 dummy key provider for feeding the primary index scan. 
+        AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
 
-            // 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(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMRTreeIOOperationCallbackFactory.INSTANCE, 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;
+        // Create primary index scan op.
+        BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
 
-            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 {
+        // Assign op.
+        AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp,
+                numNestedSecondaryKeyFields);
 
-            // 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,
-                    numNestedSecondaryKeyFields);
-
-            // If any of the secondary fields are nullable, then add a select op that filters nulls.
-            AlgebricksMetaOperatorDescriptor selectOp = null;
-            if (anySecondaryKeyIsNullable) {
-                selectOp = createFilterNullsSelectOp(spec, numNestedSecondaryKeyFields);
-            }
-
-            AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
-            // Create secondary RTree bulk load op.
-            TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
-                    spec,
-                    numNestedSecondaryKeyFields,
-                    new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                            primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
-                                    secondaryComparatorFactories.length), 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, secondaryBulkLoadOp, 0);
-            } else {
-                spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, secondaryBulkLoadOp, 0);
-            }
-            spec.addRoot(secondaryBulkLoadOp);
-            spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-            return spec;
+        // If any of the secondary fields are nullable, then add a select op that filters nulls.
+        AlgebricksMetaOperatorDescriptor selectOp = null;
+        if (anySecondaryKeyIsNullable) {
+            selectOp = createFilterNullsSelectOp(spec, numNestedSecondaryKeyFields);
         }
+
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+        // Create secondary RTree bulk load op.
+        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec,
+                numNestedSecondaryKeyFields, new LSMRTreeDataflowHelperFactory(valueProviderFactories,
+                        RTreePolicyType.RTREE, primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+                        AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+                        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, secondaryBulkLoadOp, 0);
+        } else {
+            spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, secondaryBulkLoadOp, 0);
+        }
+        spec.addRoot(secondaryBulkLoadOp);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
     }
 }
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 8935d5d..b56fe7c 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,9 +1,8 @@
-{ "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 }
+{ "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" }
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 a44cbc0..d193dd1 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,63 +1,62 @@
-{ "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
+{ "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" }
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 6037036..607bfd1 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,12 +1,11 @@
-{ "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 }
+{ "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" }
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 8935d5d..0078603 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": "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 }
+{ "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" }
+
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 a44cbc0..cecdb85 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,63 +1,62 @@
-{ "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
+{ "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" }
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 6037036..607bfd1 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,12 +1,11 @@
-{ "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 }
+{ "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" }
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 445122f..7be9c57 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" } ], "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 }
+{ "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" }
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 2d97f0a..4233225 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,20 +16,10 @@
 
 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;
     }
@@ -45,12 +35,4 @@
     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 8dd7bfb..cb6336b 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(properties,
-        adapterName,nodeGroupName != null? new Identifier(nodeGroupName): null);
+        ExternalDetailsDecl edd = new ExternalDetailsDecl();
+        edd.setAdapter(adapterName);
+        edd.setProperties(properties);
         dsetDecl = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    new Identifier(typeName),
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 92a33cd..5adb361 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,6 +18,7 @@
 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;
@@ -41,7 +42,7 @@
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
-            numActiveOperations.incrementAndGet();
+            incrementNumActiveOperations(modificationCallback);
         } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
             datasetLifecycleManager.declareActiveIOOperation(datasetID);
         }
@@ -60,14 +61,11 @@
     public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
-            numActiveOperations.decrementAndGet();
+            decrementNumActiveOperations(modificationCallback);
+            flushIfFull();
         } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
             datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
         }
-
-        if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
-            flushIfFull();
-        }
     }
 
     private void flushIfFull() throws HyracksDataException {
@@ -100,4 +98,25 @@
     public int getNumActiveOperations() {
         return numActiveOperations.get();
     }
+
+    private void incrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+        if (modificationCallback != null && modificationCallback != NoOpOperationCallback.INSTANCE) {
+            numActiveOperations.incrementAndGet();
+            ((AbstractOperationCallback) modificationCallback).incrementLocalNumActiveOperations();
+        }
+    }
+
+    private void decrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+        if (modificationCallback != null && modificationCallback != NoOpOperationCallback.INSTANCE) {
+            numActiveOperations.decrementAndGet();
+            ((AbstractOperationCallback) modificationCallback).decrementLocalNumActiveOperations();
+        }
+    }
+
+    public void cleanupNumActiveOperationsForAbortedJob(AbstractOperationCallback callback) {
+        int delta = callback.getLocalNumActiveOperations() * -1;
+        numActiveOperations.getAndAdd(delta);
+        callback.resetLocalNumActiveOperations();
+    }
+
 }
\ No newline at end of file
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 d2eb5ba..87c497c 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
@@ -17,7 +17,6 @@
 
 import java.util.List;
 
-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.LSMBTreeDiskComponent;
@@ -32,7 +31,7 @@
     @Override
     public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
-        if (oldComponents != null && newComponent != null) {
+        if (newComponent != null) {
             LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) newComponent;
             putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
         }
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 d4b26f7..c549e7d 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,6 +15,8 @@
 
 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;
 
@@ -27,6 +29,7 @@
     protected final ITransactionContext txnCtx;
     protected final ILockManager lockManager;
     protected final long[] longHashes;
+    protected final AtomicInteger transactorLocalNumActiveOperations;
 
     public AbstractOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
             ILockManager lockManager) {
@@ -34,6 +37,7 @@
         this.primaryKeyFields = primaryKeyFields;
         this.txnCtx = txnCtx;
         this.lockManager = lockManager;
+        this.transactorLocalNumActiveOperations = new AtomicInteger(0);
         this.longHashes = new long[2];
     }
 
@@ -41,4 +45,21 @@
         MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
         return Math.abs((int) longHashes[0]);
     }
+    
+    public void resetLocalNumActiveOperations() {
+        transactorLocalNumActiveOperations.set(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/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 8913f8a..27a91a4 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,9 +14,11 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+
 public interface ILogManager {
 
-    public void log(ILogRecord logRecord);
+    public void log(ILogRecord logRecord) throws ACIDException;
 
     public ILogReader getLogReader(boolean isRecoveryMode);
 
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
index d13ef6c..3068867 100644
--- 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
@@ -20,15 +20,15 @@
 
 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 static final int JOB_TERMINATE_LOG_SIZE = 13; //JOB_COMMIT or ABORT log type
+    public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 25;
+    public static final int UPDATE_LOG_BASE_SIZE = 60;
 
     public boolean readLogRecord(ByteBuffer buffer);
 
     public void writeLogRecord(ByteBuffer buffer);
 
-    public void formJobCommitLogRecord(ITransactionContext txnCtx);
+    public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit);
 
     public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
             ITupleReference tupleReference, int[] primaryKeyFields);
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 77e960b..ffd4cc2 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
@@ -52,10 +52,11 @@
      * 
      * @param jobId
      *            a unique value for the transaction id.
+     * @param createIfNotExist TODO
      * @return
      * @throws ACIDException
      */
-    public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException;
+    public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException;
 
     /**
      * Commits a transaction.
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index ffc0eac..f8d5ea2 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -153,17 +153,6 @@
 			<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 e9d2175..4ca3d72 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,21 +20,12 @@
 
 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;
@@ -44,190 +35,76 @@
  */
 @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 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";
+    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";
 
-	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");
-		formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
-		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");
+        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);
+            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;
+    }
 
-		//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));
-		}
+    @Override
+    public String getName() {
+        return HDFS_ADAPTER_NAME;
+    }
 
-		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);
-	}
-
+    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;
+    }
 
 }
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 64c8153..409eb7a 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,20 +18,15 @@
 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;
@@ -41,185 +36,109 @@
  */
 @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 INPUT_FORMAT_RC = "rc-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 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 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 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");
-		formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
-		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");
+        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 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;
+            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
 
-		//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;
-	}
+            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+            inputSplitsFactory = new InputSplitsFactory(inputSplits);
 
-	@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);
+            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+            readSchedule = scheduler.getLocationConstraints(inputSplits);
+            executed = new boolean[readSchedule.length];
+            Arrays.fill(executed, false);
 
-			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+            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;
+    }
 
-			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-			inputSplitsFactory = new InputSplitsFactory(inputSplits);
+    @Override
+    public String getName() {
+        return "hive";
+    }
 
-			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-			readSchedule = scheduler.getLocationConstraints(inputSplits);
-			executed = new boolean[readSchedule.length];
-			Arrays.fill(executed, false);
+    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+        JobConf conf = new JobConf();
 
-			setup = true;
-		}
-		JobConf conf = confFactory.getConf();
-		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-		HiveIndexingAdapter hiveIndexingAdapter = new HiveIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
+        /** 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 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));
-		}
+        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");
+        }
 
-		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
+        /** 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;
+    }
+}
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 f046f88..0a178a7 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,10 +14,8 @@
  */
 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;
 
@@ -42,29 +40,4 @@
      */
     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 4fae7e7..e680232 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,14 +14,11 @@
  */
 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
@@ -29,31 +26,18 @@
  * 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 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");
-	}
+    @Override
+    public String getName() {
+        return NC_FILE_SYSTEM_ADAPTER_NAME;
+    }
 }
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
deleted file mode 100644
index aa91a56..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
+++ /dev/null
@@ -1,78 +0,0 @@
-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
deleted file mode 100644
index 9ff1f06..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
+++ /dev/null
@@ -1,64 +0,0 @@
-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
deleted file mode 100644
index 86a060c..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
+++ /dev/null
@@ -1,1170 +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.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 2b355ff..f8b381b 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,7 +18,6 @@
 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;
@@ -27,8 +26,6 @@
 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;
@@ -37,348 +34,202 @@
 
 /**
  * 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 {
 
-		if(conf.getInputFormat() instanceof RCFileInputFormat)
-		{
-			//if hdfs input format is rc-input-format, we return a different InputStream
-			return new InputStream() {
+        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 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;
-								}
-							}
+            @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();
-							return true;
-						}
-					}
-					return false;
-				}
+                        /**
+                         * 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;
-						}
-					}
+            @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
-						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;
-					}
+                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;
+                }
 
-					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;
-				}
+                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;
+            }
 
-				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;
-					}
-				}
+            @Override
+            public int read() throws IOException {
+                throw new NotImplementedException("Use read(byte[], int, int");
+            }
 
-				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;
-				}
+            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 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
-		{
-			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;
-	}
+    @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
deleted file mode 100644
index 59b39c5..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
+++ /dev/null
@@ -1,1208 +0,0 @@
-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
deleted file mode 100644
index 178b106..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
+++ /dev/null
@@ -1,62 +0,0 @@
-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
deleted file mode 100644
index 253f675..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
+++ /dev/null
@@ -1,17 +0,0 @@
-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
deleted file mode 100644
index 47550a4..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java
+++ /dev/null
@@ -1,38 +0,0 @@
-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
deleted file mode 100644
index 29fcfb1..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
+++ /dev/null
@@ -1,94 +0,0 @@
-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-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 92d1c8e..dae1fb1 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,235 +19,215 @@
 
 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]
-	};
+    // 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;
 
-	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;
+// ================================================================================
+//  Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
+// ================================================================================
 
-	// ================================================================================
-	//  Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
-	// ================================================================================
+    [LEXER_AUXFUNCTIONS]
 
-	[LEXER_AUXFUNCTIONS]
+// ================================================================================
+//  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;
 
-			// ================================================================================
-			//  Main method. Return a TOKEN_CONSTANT
-			// ================================================================================            
+        [LEXER_LOGIC]
+    }
 
-			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 interface
+// ================================================================================
+    
+    public [LEXER_NAME](java.io.Reader stream) throws IOException{
+        reInit(stream);
+    }
 
-		[LEXER_LOGIC]
-	}
+    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();
+    }
 
-	//used when done with stream, must be called exiplicitly now.
-	public void close()throws IOException
-	{
-		inputStream.close();
-	}
+    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 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;
-	}
+    public void done(){
+        buffer = null;
+    }
 
-	// ================================================================================
-	//  Public interface
-	// ================================================================================
+// ================================================================================
+//  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 [LEXER_NAME](java.io.Reader stream) throws IOException{
-		reInit(stream);
-	}
+    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 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 char readNextChar() throws IOException {
+        if (++bufpos >= endOf_USED_Buffer)
+            fillBuff();
+        char c = buffer[bufpos];
+        updateLineColumn(c);
+        return c;
+    }
 
-	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 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;
+    }
 
 
-	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 2de5d78..8f0eedb 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,7 +31,6 @@
 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;
@@ -269,39 +268,6 @@
         }
         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 81ce4f8..ee0791b 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
@@ -41,8 +41,6 @@
 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;
@@ -52,7 +50,6 @@
 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;
@@ -62,7 +59,6 @@
 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;
@@ -119,14 +115,15 @@
 
     @Override
     public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
     }
 
     @Override
     public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
         try {
-            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId,
+                    false);
             transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
         } catch (ACIDException e) {
             e.printStackTrace();
@@ -136,13 +133,13 @@
 
     @Override
     public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
     }
 
     @Override
     public void unlock(JobId jobId) throws ACIDException, RemoteException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
     }
 
@@ -173,56 +170,48 @@
                 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);
-			}
-			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);
-		}
-	}
+                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);
+        }
+    }
 
-	@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 {
@@ -285,7 +274,7 @@
 
         ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         txnCtx.setWriteTxn(true);
         txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                 metadataIndex.isPrimaryIndex());
@@ -298,7 +287,7 @@
 
     private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
             IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
 
         if (metadataIndex.isPrimaryIndex()) {
             return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
@@ -396,60 +385,47 @@
                 // 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 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(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);
-		}
-	}
+            // 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);
+        }
+    }
 
     @Override
     public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
@@ -606,7 +582,7 @@
                 lsmIndex, IndexOperation.DELETE);
         ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         txnCtx.setWriteTxn(true);
         txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                 metadataIndex.isPrimaryIndex());
@@ -1160,116 +1136,8 @@
         }
     }
 
-	@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();
     }
-
-
-	@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 53f72dd..22c5e46 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,7 +26,6 @@
 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;
@@ -440,36 +439,6 @@
      */
     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;
     
@@ -484,5 +453,4 @@
     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 206ef8a..d1e63e1 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,7 +28,6 @@
 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;
@@ -472,45 +471,6 @@
      * @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 0c64497..7be4e8e 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
@@ -60,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.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -117,7 +116,7 @@
                 MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
                 MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
                 MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
+                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET };
         secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
@@ -333,7 +332,6 @@
         int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
         LSMBTree lsmBtree = null;
         long resourceID = -1;
-        AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
         ILSMOperationTracker opTracker = index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index
                 .getDatasetId().getId()) : new BaseOperationTracker((DatasetLifecycleManager) indexLifecycleManager,
                 index.getDatasetId().getId());
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 100ec40..8bdd92b 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,11 +41,9 @@
     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
@@ -94,11 +92,5 @@
                 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 11f9c91..50681ee 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataRecordTypes.java
@@ -47,7 +47,6 @@
     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.
@@ -77,7 +76,6 @@
             NODEGROUP_RECORDTYPE = createNodeGroupRecordType();
             FUNCTION_RECORDTYPE = createFunctionRecordType();
             DATASOURCE_ADAPTER_RECORDTYPE = createDatasourceAdapterRecordType();
-            EXTERNAL_FILE_RECORDTYPE = createExternalFileRecordType();
         } catch (AsterixException e) {
             throw new MetadataException(e);
         }
@@ -127,13 +125,12 @@
     // 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", "GroupName" };
-        IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType, BuiltinType.ASTRING };
+        String[] fieldNames = { "DatasourceAdapter", "Properties" };
+        IAType[] fieldTypes = { BuiltinType.ASTRING, orderedPropertyListType };
         return new ARecordType(null, fieldNames, fieldTypes, true);
     }
 
@@ -359,19 +356,5 @@
                 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 de94045..4909e21 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,14 +18,11 @@
 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;
@@ -47,17 +44,13 @@
 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;
@@ -71,7 +64,6 @@
 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;
@@ -168,7 +160,6 @@
     private boolean asyncResults;
     private ResultSetId resultSetId;
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
-    private static boolean optimizeExternalIndexes = false;
 
     private final Dataverse defaultDataverse;
     private JobId jobId;
@@ -178,6 +169,22 @@
     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();
@@ -258,30 +265,6 @@
         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 {
         AqlSourceId aqlId = (AqlSourceId) id;
@@ -421,206 +404,6 @@
     }
 
     @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,
             IAType itemType, IParseFileSplitsDecl decl, IDataFormat format) throws AlgebricksException {
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
@@ -726,133 +509,85 @@
             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;
+        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;
-                int keysStartIndex = outputVars.size() - numKeys;
+                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);
+            } else {
+                typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
+                        context);
+                bloomFilterKeyFields = new int[numPrimaryKeys];
+                for (int i = 0; i < numPrimaryKeys; i++) {
+                    bloomFilterKeyFields[i] = i;
                 }
-                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(dataset.getDatasetId())
-                                        : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                                rtcProvider, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                                storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
-                        searchCallbackFactory);
-                return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
-            } catch (MetadataException me) {
-                throw new AlgebricksException(me);
             }
-        } else {
+            IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+                    outputVars, keysStartIndex, numKeys, typeEnv, context);
+
+            IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
             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);
-                }
-
-                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();
-                    if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
-                        searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
-                                primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
-                    } else {
-                        searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
-                                primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
-                    }
-                }
-                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(dataset.getDatasetId())
-                                        : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                                rtcProvider, LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                                storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
-                        searchCallbackFactory);
-
-                return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
-
-            } catch (MetadataException me) {
-                throw new AlgebricksException(me);
+                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;
+                }
+
+                AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
+                ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+                if (aqlMetadataImplConfig != null && aqlMetadataImplConfig.isInstantLock()) {
+                    searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(jobId, datasetId,
+                            primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+                } else {
+                    searchCallbackFactory = new PrimaryIndexSearchOperationCallbackFactory(jobId, datasetId,
+                            primaryKeyFields, txnSubsystemProvider, ResourceType.LSM_BTREE);
+                }
+            }
+            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(dataset.getDatasetId())
+                                    : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                            storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
+
+            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+
+        } catch (MetadataException me) {
+            throw new AlgebricksException(me);
         }
     }
 
@@ -1609,17 +1344,9 @@
         }
 
         int numPartitions = 0;
-        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();
-        }
-
+        InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+        List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
+                .getNodeNames();
         for (String nd : nodeGroup) {
             numPartitions += AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
         }
@@ -1637,12 +1364,6 @@
         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);
@@ -1733,56 +1454,6 @@
         }
     }
 
-    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",
@@ -1795,8 +1466,6 @@
                 "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;
     }
 
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 849c6bd..18cef340 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,25 +40,17 @@
 
     private static final long serialVersionUID = 1L;
     private final String adapter;
-    private final Map<String,String> properties;
-    private final String nodeGroupName;
+    private final Map<String, String> properties;
 
-	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, String nodeGroupName) {
+    public ExternalDatasetDetails(String adapter, Map<String, String> properties) {
         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;
     }
 
@@ -66,10 +58,6 @@
         return properties;
     }
 
-    public String getNodeGroupName() {
-		return nodeGroupName;
-	}
-    
     @Override
     public DatasetType getDatasetType() {
         return DatasetType.EXTERNAL;
@@ -105,12 +93,6 @@
         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
deleted file mode 100644
index 0128783..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/ExternalFile.java
+++ /dev/null
@@ -1,110 +0,0 @@
-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 d346a6c..61f856a 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/DatasetTupleTranslator.java
@@ -219,10 +219,7 @@
                             .getStringValue();
                     properties.put(key, value);
                 }
-                String groupName = ((AString) datasetDetailsRecord
-                        .getValueByPos(MetadataRecordTypes.EXTERNAL_DETAILS_ARECORD_GROUPNAME_FIELD_INDEX ))
-                        .getStringValue();
-                datasetDetails = new ExternalDatasetDetails(adapter, properties,groupName);
+                datasetDetails = new ExternalDatasetDetails(adapter, properties);
         }
         
         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
deleted file mode 100644
index 6837c72..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entitytupletranslators/ExternalFileTupleTranslator.java
+++ /dev/null
@@ -1,158 +0,0 @@
-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 95d26d9..820f277 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/utils/DatasetUtils.java
@@ -20,12 +20,9 @@
 
 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;
@@ -35,8 +32,6 @@
 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 {
@@ -87,46 +82,6 @@
         }
         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 {
@@ -157,17 +112,6 @@
         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 00b7459..cc3fd04 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,10 +33,6 @@
     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 b6bdb4b..3c8a0c4 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,8 +243,6 @@
 
     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);
@@ -674,14 +672,6 @@
                 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-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 b44755c..4f4eba2 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
@@ -49,7 +49,7 @@
                 try {
                     ITransactionManager txnManager = ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext()
                             .getApplicationObject()).getTransactionSubsystem().getTransactionManager();
-                    ITransactionContext txnContext = txnManager.getTransactionContext(jobId);
+                    ITransactionContext txnContext = txnManager.getTransactionContext(jobId, false);
                     txnContext.setWriteTxn(transactionalWrite);
                     txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
                             !(jobStatus == JobStatus.FAILURE));
@@ -62,7 +62,7 @@
             public void jobletStart() {
                 try {
                     ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
-                            .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId);
+                            .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId, true);
                 } catch (ACIDException e) {
                     throw new Error(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 e08c509..b8c8659 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,16 +92,6 @@
             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
deleted file mode 100644
index a4f1691..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
+++ /dev/null
@@ -1,123 +0,0 @@
-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 78159f5..2322338 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,7 +62,6 @@
             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 34f75e6..7cb9bb0 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 adm formatted input files.
+ * parsing delimited 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
deleted file mode 100644
index aa2d9ba..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
+++ /dev/null
@@ -1,23 +0,0 @@
-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
deleted file mode 100644
index d72366c..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
+++ /dev/null
@@ -1,28 +0,0 @@
-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
deleted file mode 100644
index 15643d4..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
+++ /dev/null
@@ -1,23 +0,0 @@
-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
deleted file mode 100644
index ced33ef..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
+++ /dev/null
@@ -1,29 +0,0 @@
-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
deleted file mode 100644
index 10b09f5..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
+++ /dev/null
@@ -1,29 +0,0 @@
-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 22c43ec..5a639dc 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,17 +100,6 @@
     }
 
     @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);
@@ -176,7 +165,7 @@
         private static final int INITIAL_BUFFER_SIZE = 4096;
         private static final int INCREMENT = 4096;
 
-        private Reader in;
+        private final Reader in;
 
         private char[] buffer;
         private int start;
@@ -193,21 +182,6 @@
             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 cd9ae2e..46d281b 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,17 +52,4 @@
      * @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 ffd9edd..e037ec6 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,8 +224,7 @@
             while (continueIngestion) {
                 tb.reset();
                 if (!parser.parse(tb.getDataOutput())) {
-                	parser.close();
-                	break;
+                    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 c576cf8..bf3c086 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,17 +14,14 @@
  */
 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
@@ -70,12 +67,6 @@
     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) {
@@ -92,10 +83,4 @@
         }
     }
 
-	@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/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
index a6537cd..00f0c4a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
             throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             return new PrimaryIndexInstantSearchOperationCallback(datasetId, primaryKeyFields,
                     txnSubsystem.getLockManager(), txnCtx);
         } catch (ACIDException 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 abeec62..07daee4 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
@@ -58,7 +58,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
                     primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
                     indexOp);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
index b59eb75..01cb725 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
             throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             return new PrimaryIndexSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
                     txnCtx);
         } catch (ACIDException 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 d5bc877..563e9b7 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
@@ -55,7 +55,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
                     txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
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 6d86f70..c7df2f2 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
@@ -665,8 +665,6 @@
 
         latchLockTable();
         try {
-            validateJob(txnContext);
-
             if (IS_DEBUG_MODE) {
                 trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
                         dLockInfo, eLockInfo);
@@ -2212,14 +2210,14 @@
                 if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
                     tempDatasetIdObj.setId(logRecord.getDatasetId());
                     tempJobIdObj.setId(logRecord.getJobId());
-                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
                     unlock(tempDatasetIdObj, logRecord.getPKHashValue(), txnCtx);
                     txnCtx.notifyOptracker(false);
-                } else if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+                } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
                     tempJobIdObj.setId(logRecord.getJobId());
-                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
                     txnCtx.notifyOptracker(true);
-                    ((LogPage) logPage).notifyJobCommitter();
+                    ((LogPage) logPage).notifyJobTerminator();
                 }
                 logRecord = logPageReader.next();
             }
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 4f0bb59..933afcd 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
@@ -38,6 +38,7 @@
 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.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
 import edu.uci.ics.asterix.common.transactions.MutableLong;
 import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
@@ -74,16 +75,16 @@
         logDir = logManagerProperties.getLogDir();
         logFilePrefix = logManagerProperties.getLogFilePrefix();
         flushLSN = new MutableLong();
-        initializeLogManager();
+        initializeLogManager(0);
     }
 
-    private void initializeLogManager() {
+    private void initializeLogManager(long nextLogFileId) {
         emptyQ = new LinkedBlockingQueue<LogPage>(numLogPages);
         flushQ = new LinkedBlockingQueue<LogPage>(numLogPages);
         for (int i = 0; i < numLogPages; i++) {
             emptyQ.offer(new LogPage((LockManager) txnSubsystem.getLockManager(), logPageSize, flushLSN));
         }
-        appendLSN = initializeLogAnchor();
+        appendLSN = initializeLogAnchor(nextLogFileId);
         flushLSN.set(appendLSN);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("LogManager starts logging in LSN: " + appendLSN);
@@ -95,12 +96,13 @@
     }
 
     @Override
-    public void log(ILogRecord logRecord) {
+    public void log(ILogRecord logRecord) throws ACIDException {
         if (logRecord.getLogSize() > logPageSize) {
             throw new IllegalStateException();
         }
         syncLog(logRecord);
-        if (logRecord.getLogType() == LogType.JOB_COMMIT && !logRecord.isFlushed()) {
+        if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
+                && !logRecord.isFlushed()) {
             synchronized (logRecord) {
                 while (!logRecord.isFlushed()) {
                     try {
@@ -113,13 +115,16 @@
         }
     }
 
-    private synchronized void syncLog(ILogRecord logRecord) {
+    private synchronized void syncLog(ILogRecord logRecord) throws ACIDException {
         ITransactionContext txnCtx = logRecord.getTxnCtx();
+        if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
+            throw new ACIDException("Aborted job(" + txnCtx.getJobId() + ") tried to write non-abort type log record.");
+        }
         if (getLogFileOffset(appendLSN) + logRecord.getLogSize() > logFileSize) {
             prepareNextLogFile();
             appendPage.isFull(true);
             getAndInitNewPage();
-        } else if (!appendPage.hasSpace(logRecord.getLogSize(), getLogFileOffset(appendLSN))) {
+        } else if (!appendPage.hasSpace(logRecord.getLogSize())) {
             appendPage.isFull(true);
             getAndInitNewPage();
         }
@@ -141,7 +146,6 @@
         }
         appendPage.reset();
         appendPage.setFileChannel(appendChannel);
-        appendPage.setInitialFlushOffset(getLogFileOffset(appendLSN));
         flushQ.offer(appendPage);
     }
 
@@ -229,7 +233,7 @@
         return flushLSN;
     }
 
-    private long initializeLogAnchor() {
+    private long initializeLogAnchor(long nextLogFileId) {
         long fileId = 0;
         long offset = 0;
         File fileLogDir = new File(logDir);
@@ -237,9 +241,10 @@
             if (fileLogDir.exists()) {
                 List<Long> logFileIds = getLogFileIds();
                 if (logFileIds == null) {
-                    createFileIfNotExists(getLogFilePath(0));
+                    fileId = nextLogFileId;
+                    createFileIfNotExists(getLogFilePath(fileId));
                     if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("created a log file: " + getLogFilePath(0));
+                        LOGGER.info("created a log file: " + getLogFilePath(fileId));
                     }
                 } else {
                     fileId = logFileIds.get(logFileIds.size() - 1);
@@ -247,13 +252,14 @@
                     offset = logFile.length();
                 }
             } else {
+                fileId = nextLogFileId;
                 createNewDirectory(logDir);
                 if (LOGGER.isLoggable(Level.INFO)) {
                     LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
                 }
-                createFileIfNotExists(getLogFilePath(0));
+                createFileIfNotExists(getLogFilePath(fileId));
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created a log file: " + getLogFilePath(0));
+                    LOGGER.info("created a log file: " + getLogFilePath(fileId));
                 }
             }
         } catch (IOException ioe) {
@@ -267,8 +273,8 @@
 
     public void renewLogFiles() {
         terminateLogFlusher();
-        deleteAllLogFiles();
-        initializeLogManager();
+        long lastMaxLogFileId = deleteAllLogFiles();
+        initializeLogManager(lastMaxLogFileId + 1);
     }
 
     private void terminateLogFlusher() {
@@ -290,7 +296,7 @@
         }
     }
 
-    private void deleteAllLogFiles() {
+    private long deleteAllLogFiles() {
         if (appendChannel != null) {
             try {
                 appendChannel.close();
@@ -305,6 +311,7 @@
                 throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
             }
         }
+        return logFileIds.get(logFileIds.size() - 1);
     }
 
     private List<Long> getLogFileIds() {
@@ -384,10 +391,16 @@
         }
         return newFileChannel;
     }
+
+    public long getReadableSmallestLSN() {
+        List<Long> logFileIds = getLogFileIds();
+        return logFileIds.get(0) * logFileSize;
+    }
 }
 
 class LogFlusher implements Callable<Boolean> {
-    private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_COMMIT_LOG_SIZE, null);
+    private static final Logger LOGGER = Logger.getLogger(LogFlusher.class.getName());
+    private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_TERMINATE_LOG_SIZE, null);
     private final LogManager logMgr;//for debugging
     private final LinkedBlockingQueue<LogPage> emptyQ;
     private final LinkedBlockingQueue<LogPage> flushQ;
@@ -402,13 +415,13 @@
         flushPage = null;
         isStarted = new AtomicBoolean(false);
         terminateFlag = new AtomicBoolean(false);
-        
+
     }
 
     public void terminate() {
         //make sure the LogFlusher thread started before terminating it.
         synchronized (isStarted) {
-            while(!isStarted.get()) {
+            while (!isStarted.get()) {
                 try {
                     isStarted.wait();
                 } catch (InterruptedException e) {
@@ -416,7 +429,7 @@
                 }
             }
         }
-        
+
         terminateFlag.set(true);
         if (flushPage != null) {
             synchronized (flushPage) {
@@ -432,24 +445,34 @@
 
     @Override
     public Boolean call() {
-        synchronized(isStarted) {
+        synchronized (isStarted) {
             isStarted.set(true);
             isStarted.notify();
         }
-        while (true) {
-            flushPage = null;
-            try {
-                flushPage = flushQ.take();
-                if (flushPage == POISON_PILL || terminateFlag.get()) {
-                    return true;
+        try {
+            while (true) {
+                flushPage = null;
+                try {
+                    flushPage = flushQ.take();
+                    if (flushPage == POISON_PILL || terminateFlag.get()) {
+                        return true;
+                    }
+                } catch (InterruptedException e) {
+                    if (flushPage == null) {
+                        continue;
+                    }
                 }
-            } catch (InterruptedException e) {
-                if (flushPage == null) {
-                    continue;
-                }
+                flushPage.flush();
+                emptyQ.offer(flushPage);
             }
-            flushPage.flush();
-            emptyQ.offer(flushPage);
+        } catch (Exception e) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("-------------------------------------------------------------------------");
+                LOGGER.info("LogFlusher is terminating abnormally. System is in unusalbe state.");
+                LOGGER.info("-------------------------------------------------------------------------");
+            }
+            e.printStackTrace();
+            throw e;
         }
     }
 }
\ 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
index edfec69..8921f35 100644
--- 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
@@ -58,7 +58,7 @@
         appendOffset = 0;
         flushOffset = 0;
         isLastPage = false;
-        syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_COMMIT_LOG_SIZE);
+        syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_TERMINATE_LOG_SIZE);
     }
 
     ////////////////////////////////////
@@ -76,7 +76,7 @@
             if (IS_DEBUG_MODE) {
                 LOGGER.info("append()| appendOffset: " + appendOffset);
             }
-            if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+            if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
                 logRecord.isFlushed(false);
                 syncCommitQ.offer(logRecord);
             }
@@ -105,7 +105,7 @@
         this.isLastPage = isLastPage;
     }
 
-    public boolean hasSpace(int logSize, long logFileOffset) {
+    public boolean hasSpace(int logSize) {
         return appendOffset + logSize <= logPageSize;
     }
 
@@ -192,7 +192,7 @@
         }
     }
 
-    public void notifyJobCommitter() {
+    public void notifyJobTerminator() {
         ILogRecord logRecord = null;
         while (logRecord == null) {
             try {
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
index 4b0e1f2..dd81df7 100644
--- 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
@@ -34,10 +34,9 @@
  * LogType(1)
  * JobId(4)
  * ---------------------------
- * [Header2] (16 bytes + PKValueSize) : for entity_commit and update log types 
+ * [Header2] (12 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)
  * ---------------------------
@@ -61,10 +60,10 @@
  * ---------------------------
  * = 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
+ * 2) ENTITY_COMMIT: 25 + PKSize (5 + 12 + PKSize + 8)
+ *    --> ENTITY_COMMIT_LOG_BASE_SIZE = 25
+ * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 12 + PKSize + 21 + 14 + New/OldValueSize + 8)
+ *    --> UPDATE_LOG_BASE_SIZE = 60
  */
 public class LogRecord implements ILogRecord {
 
@@ -73,7 +72,6 @@
     private int jobId;
     private int datasetId;
     private int PKHashValue;
-    private int PKFieldCnt;
     private int PKValueSize;
     private ITupleReference PKValue;
     private long prevLSN;
@@ -90,12 +88,13 @@
     private long checksum;
     //------------- fields in a log record (end) --------------//
 
+    private int PKFieldCnt;
     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 PrimaryKeyTupleReference readPKValue;
     private final SimpleTupleReference readNewValue;
     private final SimpleTupleReference readOldValue;
     private final CRC32 checksumGen;
@@ -104,7 +103,7 @@
     public LogRecord() {
         isFlushed = new AtomicBoolean(false);
         tupleWriter = new SimpleTupleWriter();
-        readPKValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+        readPKValue = new PrimaryKeyTupleReference();
         readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
         readOldValue = (SimpleTupleReference) tupleWriter.createTupleReference();
         checksumGen = new CRC32();
@@ -115,10 +114,9 @@
         int beginOffset = buffer.position();
         buffer.put(logType);
         buffer.putInt(jobId);
-        if (logType != LogType.JOB_COMMIT) {
+        if (logType == LogType.UPDATE || logType == LogType.ENTITY_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");
             }
@@ -173,13 +171,12 @@
         try {
             logType = buffer.get();
             jobId = buffer.getInt();
-            if (logType == LogType.JOB_COMMIT) {
+            if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
                 datasetId = -1;
                 PKHashValue = -1;
             } else {
-                datasetId = buffer.getInt();    
+                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");
@@ -217,12 +214,20 @@
         }
         return true;
     }
-    
+
     private ITupleReference readPKValue(ByteBuffer buffer) {
-        return readTuple(buffer, readPKValue, PKFieldCnt, PKValueSize);
+        if (buffer.position() + PKValueSize > buffer.limit()) {
+            throw new BufferUnderflowException();
+        }
+        readPKValue.reset(buffer.array(), buffer.position(), PKValueSize);
+        buffer.position(buffer.position() + PKValueSize);
+        return readPKValue;
     }
 
     private ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt, int size) {
+        if (srcBuffer.position() + size > srcBuffer.limit()) {
+            throw new BufferUnderflowException();
+        }
         destTuple.setFieldCount(fieldCnt);
         destTuple.resetByTupleOffset(srcBuffer, srcBuffer.position());
         srcBuffer.position(srcBuffer.position() + size);
@@ -230,9 +235,9 @@
     }
 
     @Override
-    public void formJobCommitLogRecord(ITransactionContext txnCtx) {
+    public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit) {
         this.txnCtx = txnCtx;
-        this.logType = LogType.JOB_COMMIT;
+        this.logType = isCommit ? LogType.JOB_COMMIT : LogType.ABORT;
         this.jobId = txnCtx.getJobId().getId();
         this.datasetId = -1;
         this.PKHashValue = -1;
@@ -281,7 +286,8 @@
                 setUpdateLogSize();
                 break;
             case LogType.JOB_COMMIT:
-                logSize = JOB_COMMIT_LOG_SIZE;
+            case LogType.ABORT:
+                logSize = JOB_TERMINATE_LOG_SIZE;
                 break;
             case LogType.ENTITY_COMMIT:
                 logSize = ENTITY_COMMIT_LOG_BASE_SIZE + PKValueSize;
@@ -298,7 +304,7 @@
         builder.append(" LogType : ").append(LogType.toString(logType));
         builder.append(" LogSize : ").append(logSize);
         builder.append(" JobId : ").append(jobId);
-        if (logType != LogType.JOB_COMMIT) {
+        if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
             builder.append(" DatasetId : ").append(datasetId);
             builder.append(" PKHashValue : ").append(PKHashValue);
             builder.append(" PKFieldCnt : ").append(PKFieldCnt);
@@ -496,17 +502,17 @@
     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;
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 823c8d3..f9e9304 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
@@ -19,11 +19,14 @@
     public static final byte UPDATE = 0;
     public static final byte JOB_COMMIT = 1;
     public static final byte ENTITY_COMMIT = 2;
+    public static final byte ABORT = 3;
     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_ABORT = "ABORT";
     private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
 
+
     public static String toString(byte logType) {
         switch (logType) {
             case LogType.UPDATE:
@@ -32,6 +35,8 @@
                 return STRING_JOB_COMMIT;
             case LogType.ENTITY_COMMIT:
                 return STRING_ENTITY_COMMIT;
+            case LogType.ABORT:
+                return STRING_ABORT;
             default:
                 return STRING_INVALID_LOG_TYPE;
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
new file mode 100644
index 0000000..d45b209
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class PrimaryKeyTupleReference implements ITupleReference {
+    private byte[] fieldData;
+    private int start;
+    private int length;
+
+    public void reset(byte[] fieldData, int start, int length) {
+        this.fieldData = fieldData;
+        this.start = start;
+        this.length = length;
+    }
+    
+    @Override
+    public int getFieldCount() {
+        return 1;
+    }
+
+    @Override
+    public byte[] getFieldData(int fIdx) {
+        return fieldData;
+    }
+
+    @Override
+    public int getFieldStart(int fIdx) {
+        return start;
+    }
+
+    @Override
+    public int getFieldLength(int fIdx) {
+        return length;
+    }
+
+}
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 d0e9348..fcb9aa1 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
@@ -76,6 +76,7 @@
     private final TransactionSubsystem txnSubsystem;
     private final LogManager logMgr;
     private final int checkpointHistory;
+    private final long SHARP_CHECKPOINT_LSN = -1;
 
     /**
      * A file at a known location that contains the LSN of the last log record
@@ -114,18 +115,23 @@
             return state;
         }
 
-        //#. if minMCTFirstLSN is equal to -1 && 
-        //   checkpointLSN in the checkpoint file is equal to the lastLSN in the log file,
-        //   then return healthy state. Otherwise, return corrupted.
-        if ((checkpointObject.getMinMCTFirstLsn() == -2 && logMgr.getAppendLSN() == 0)
-                || (checkpointObject.getMinMCTFirstLsn() == -1 && checkpointObject.getCheckpointLsn() == logMgr
-                        .getAppendLSN())) {
+        long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+        if (logMgr.getAppendLSN() == readableSmallestLSN) {
+            if (checkpointObject.getMinMCTFirstLsn() != SHARP_CHECKPOINT_LSN) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("[Warning] ---------------------------------------------------");
+                    LOGGER.info("[Warning] Some(or all) of transaction log files are lost.");
+                    LOGGER.info("[Warning] ---------------------------------------------------");
+                    //No choice but continuing when the log files are lost. 
+                }
+            }
+            state = SystemState.HEALTHY;
+            return state;
+        } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
+                && checkpointObject.getMinMCTFirstLsn() == SHARP_CHECKPOINT_LSN) {
             state = SystemState.HEALTHY;
             return state;
         } else {
-            if (logMgr.getAppendLSN() == 0) {
-                throw new IllegalStateException("Transaction log files are lost.");
-            }
             state = SystemState.CORRUPTED;
             return state;
         }
@@ -153,10 +159,11 @@
         TxnId winnerEntity = null;
 
         //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
+        long readableSmallestLSN = logMgr.getReadableSmallestLSN();
         CheckpointObject checkpointObject = readCheckpoint();
-        long lowWaterMarkLsn = checkpointObject.getMinMCTFirstLsn();
-        if (lowWaterMarkLsn == -1 || lowWaterMarkLsn == -2) {
-            lowWaterMarkLsn = 0;
+        long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLsn();
+        if (lowWaterMarkLSN < readableSmallestLSN) {
+            lowWaterMarkLSN = readableSmallestLSN;
         }
         int maxJobId = checkpointObject.getMaxJobId();
 
@@ -170,11 +177,11 @@
 
         //#. set log reader to the lowWaterMarkLsn
         ILogReader logReader = logMgr.getLogReader(true);
-        logReader.initializeScan(lowWaterMarkLsn);
+        logReader.initializeScan(lowWaterMarkLSN);
         ILogRecord logRecord = logReader.next();
         while (logRecord != null) {
             if (IS_DEBUG_MODE) {
-                System.out.println(logRecord.getLogRecordForDisplay());
+                LOGGER.info(logRecord.getLogRecordForDisplay());
             }
             //update max jobId
             if (logRecord.getJobId() > maxJobId) {
@@ -182,16 +189,12 @@
             }
             switch (logRecord.getLogType()) {
                 case LogType.UPDATE:
-                    if (IS_DEBUG_MODE) {
-                        updateLogCount++;
-                    }
+                    updateLogCount++;
                     break;
                 case LogType.JOB_COMMIT:
                     winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
                     jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
-                    if (IS_DEBUG_MODE) {
-                        jobCommitLogCount++;
-                    }
+                    jobCommitLogCount++;
                     break;
                 case LogType.ENTITY_COMMIT:
                     jobId = logRecord.getJobId();
@@ -204,9 +207,10 @@
                         winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
                     }
                     winnerEntitySet.add(winnerEntity);
-                    if (IS_DEBUG_MODE) {
-                        entityCommitLogCount++;
-                    }
+                    entityCommitLogCount++;
+                    break;
+                case LogType.ABORT:
+                    //ignore
                     break;
                 default:
                     throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
@@ -238,11 +242,11 @@
         ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
 
         //#. set log reader to the lowWaterMarkLsn again.
-        logReader.initializeScan(lowWaterMarkLsn);
+        logReader.initializeScan(lowWaterMarkLSN);
         logRecord = logReader.next();
         while (logRecord != null) {
-            if (LogManager.IS_DEBUG_MODE) {
-                System.out.println(logRecord.getLogRecordForDisplay());
+            if (IS_DEBUG_MODE) {
+                LOGGER.info(logRecord.getLogRecordForDisplay());
             }
             LSN = logRecord.getLSN();
             jobId = logRecord.getJobId();
@@ -307,15 +311,14 @@
 
                         if (LSN > maxDiskLastLsn) {
                             redo(logRecord);
-                            if (IS_DEBUG_MODE) {
-                                redoCount++;
-                            }
+                            redoCount++;
                         }
                     }
                     break;
 
                 case LogType.JOB_COMMIT:
                 case LogType.ENTITY_COMMIT:
+                case LogType.ABORT:
                     //do nothing
                     break;
 
@@ -335,9 +338,7 @@
 
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("[RecoveryMgr] recovery is completed.");
-        }
-        if (IS_DEBUG_MODE) {
-            System.out.println("[RecoveryMgr] Count: Update/EntityCommit/JobCommit/Redo = " + updateLogCount + "/"
+            LOGGER.info("[RecoveryMgr] Count: Update/EntityCommit/JobCommit/Redo = " + updateLogCount + "/"
                     + entityCommitLogCount + "/" + jobCommitLogCount + "/" + redoCount);
         }
     }
@@ -389,7 +390,7 @@
                     throw new ACIDException(e);
                 }
             }
-            minMCTFirstLSN = -2;
+            minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
         } else {
             long firstLSN;
             minMCTFirstLSN = Long.MAX_VALUE;
@@ -400,7 +401,7 @@
                     minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
                 }
             } else {
-                minMCTFirstLSN = -1;
+                minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
             }
         }
         CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
@@ -577,7 +578,7 @@
                 break;
             } else {
                 if (IS_DEBUG_MODE) {
-                    System.out.println(logRecord.getLogRecordForDisplay());
+                    LOGGER.info(logRecord.getLogRecordForDisplay());
                 }
                 currentLSN = logRecord.getLSN();
             }
@@ -597,9 +598,9 @@
                         loserTxnTable.put(loserEntity, undoLSNSet);
                     }
                     undoLSNSet.add(Long.valueOf(currentLSN));
+                    updateLogCount++;
                     if (IS_DEBUG_MODE) {
-                        updateLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+                        LOGGER.info("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
                                 + tempKeyTxnId);
                     }
                     break;
@@ -608,12 +609,19 @@
                     throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
 
                 case LogType.ENTITY_COMMIT:
-                    loserTxnTable.remove(tempKeyTxnId);
-                    if (IS_DEBUG_MODE) {
-                        entityCommitLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN
-                                + "]" + tempKeyTxnId);
+                    undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
+                    if (undoLSNSet == null) {
+                        undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
                     }
+                    entityCommitLogCount++;
+                    if (IS_DEBUG_MODE) {
+                        LOGGER.info("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
+                                + tempKeyTxnId);
+                    }
+                    break;
+
+                case LogType.ABORT:
+                    //ignore
                     break;
 
                 default:
@@ -647,23 +655,17 @@
                     throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
                 }
                 if (IS_DEBUG_MODE) {
-                    System.out.println(logRecord.getLogRecordForDisplay());
+                    LOGGER.info(logRecord.getLogRecordForDisplay());
                 }
                 undo(logRecord);
-                if (IS_DEBUG_MODE) {
-                    undoCount++;
-                }
+                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 + "/" + entityCommitLogCount
-                    + "/" + undoCount);
+            LOGGER.info("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + entityCommitLogCount + "/"
+                    + undoCount);
         }
     }
 
@@ -762,6 +764,7 @@
         this.datasetId = datasetId;
         this.pkHashValue = pkHashValue;
         this.tupleReferencePKValue = pkValue;
+        this.pkSize = pkSize;
         isByteArrayPKValue = false;
     }
 
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 c54cb7f..4468109 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
@@ -220,4 +220,10 @@
     public LogRecord getLogRecord() {
         return logRecord;
     }
+
+    public void cleanupForAbort() {
+        if (primaryIndexOpTracker != null) {
+            primaryIndexOpTracker.cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
+        }
+    }
 }
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 01b38c2..07fc152 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
@@ -53,7 +53,12 @@
             txnCtx.setTxnState(ITransactionManager.ABORTED);
         }
         try {
-            txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+            if (txnCtx.isWriteTxn()) {
+                LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
+                logRecord.formJobTerminateLogRecord(txnCtx, false);
+                txnSubsystem.getLogManager().log(logRecord);
+                txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+            }
         } catch (Exception ae) {
             String msg = "Could not complete rollback! System is in an inconsistent state";
             if (LOGGER.isLoggable(Level.SEVERE)) {
@@ -62,6 +67,7 @@
             ae.printStackTrace();
             throw new ACIDException(msg, ae);
         } finally {
+            ((TransactionContext) txnCtx).cleanupForAbort();
             txnSubsystem.getLockManager().releaseLocks(txnCtx);
             transactionContextRepository.remove(txnCtx.getJobId());
         }
@@ -69,20 +75,24 @@
 
     @Override
     public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
-        return getTransactionContext(jobId);
+        return getTransactionContext(jobId, true);
     }
 
     @Override
-    public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException {
+    public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException {
         setMaxJobId(jobId.getId());
         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);
+            if (createIfNotExist) {
+                synchronized (this) {
+                    txnCtx = transactionContextRepository.get(jobId);
+                    if (txnCtx == null) {
+                        txnCtx = new TransactionContext(jobId, txnSubsystem);
+                        transactionContextRepository.put(jobId, txnCtx);
+                    }
                 }
+            } else {
+                throw new ACIDException("TransactionContext of " + jobId + " doesn't exist.");
             }
         }
         return txnCtx;
@@ -94,7 +104,7 @@
         try {
             if (txnCtx.isWriteTxn()) {
                 LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
-                logRecord.formJobCommitLogRecord(txnCtx);
+                logRecord.formJobTerminateLogRecord(txnCtx, true);
                 txnSubsystem.getLogManager().log(logRecord);
             }
         } catch (Exception ae) {