Revert "Merge external indexes into master"

This reverts commit 1a0d2b4b80677ddadb524503ad9c84e2f03aade7, reversing
changes made to 5eba6a0795f2278f4ea7747f8b54c858529b5502.
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/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 de4d075..af56894 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;
@@ -59,17 +55,6 @@
                 metadataProvider, physicalOptimizationConfig);
         return secondaryIndexCreator.buildLoadingJobSpec();
     }
-    
-    public static void addExternalDatasetFilesToMetadata(AqlMetadataProvider metadataProvider, 
-			Dataset dataset) throws AlgebricksException, MetadataException{
-			//get the file list
-			ArrayList<ExternalFile> files = metadataProvider.getExternalDatasetFiles(dataset);
-			//add files to metadata
-			for(int i=0; i < files.size(); i++)
-			{
-				MetadataManager.INSTANCE.addExternalFile(metadataProvider.getMetadataTxnContext(), files.get(i));
-			}
-	}
 
     public static JobSpecification buildDropSecondaryIndexJobSpec(CompiledIndexDropStatement indexDropStmt,
             AqlMetadataProvider metadataProvider, Dataset dataset) throws AlgebricksException, MetadataException {
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 144131f..e3832d4 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -17,31 +17,20 @@
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
-
-import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
-import edu.uci.ics.asterix.external.util.ExternalIndexHashPartitionComputerFactory;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
@@ -60,16 +49,18 @@
         super(physOptConf, propertiesProvider);
     }
 
-	@Override
-	public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
-		JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-		AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
-		//prepare a LocalResourceMetadata which will be stored in NC's local resource repository
-		ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
-				secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
-				dataset.getDatasetId());
-		ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
-				localResourceMetadata, LocalResource.LSMBTreeResource);
+    @Override
+    public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
+
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+        //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
+        ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
+                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, true,
+                dataset.getDatasetId());
+        ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
+                localResourceMetadata, LocalResource.LSMBTreeResource);
+
         TreeIndexCreateOperatorDescriptor secondaryIndexCreateOp = new TreeIndexCreateOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
@@ -87,92 +78,30 @@
         return spec;
     }
 
+    @Override
+    public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException {
+        JobSpecification spec = JobSpecificationUtils.createJobSpecification();
 
-	@Override
-	public JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException{
-		if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-			JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-			Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> RIDScanOpAndConstraints;
-			AlgebricksMetaOperatorDescriptor asterixAssignOp;
-			try
-			{
-				//create external indexing scan operator
-				RIDScanOpAndConstraints = createExternalIndexingOp(spec);
+        // Create dummy key provider for feeding the primary index scan. 
+        AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(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");
-			}
+        // Create primary index scan op.
+        BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(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,RIDScanOpAndConstraints.second);
-			}
+        // Assign op.
+        AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
 
-			// Sort by secondary keys.
-			ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc,RIDScanOpAndConstraints.second);
-			// Create secondary BTree bulk load op.
-        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
-                spec,
-                numSecondaryKeys,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-                                LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
-                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
-			IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
+        // 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);
+        }
 
-			//select partitioning keys (always the first 2 after secondary keys)
-			int[] keys = new int[2];
-			keys[0] = numSecondaryKeys;
-			keys[1] = numSecondaryKeys + 1;
+        // Sort by secondary keys.
+        ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
 
-			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.
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+        // Create secondary BTree bulk load op.
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
                 spec,
                 numSecondaryKeys,
@@ -183,19 +112,18 @@
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
                                 .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
-			// Connect the operators.
-			spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
-			spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
-			if (anySecondaryKeyIsNullable) {
-				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-				spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
-			} else {
-				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
-			}
-			spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
-			spec.addRoot(secondaryBulkLoadOp);
-			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-			return spec;
-		}
-	}
+        // Connect the operators.
+        spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
+        spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
+        if (anySecondaryKeyIsNullable) {
+            spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
+            spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
+        } else {
+            spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
+        }
+        spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
+        spec.addRoot(secondaryBulkLoadOp);
+        spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
+        return spec;
+    }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index 60a4451..a9b3881 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
@@ -28,10 +28,6 @@
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory;
-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,11 +35,9 @@
 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;
@@ -51,7 +45,6 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.NotDescriptor;
 import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
-import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
@@ -285,18 +278,26 @@
         int[] lowKeyFields = null;
         // +Infinity
         int[] highKeyFields = null;
+        ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
+        JobId jobId = JobIdFactory.generateJobId();
+        metadataProvider.setJobId(jobId);
+        boolean isWriteTransaction = metadataProvider.isWriteTransaction();
+        IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(jobId, isWriteTransaction);
+        spec.setJobletEventListenerFactory(jobEventListenerFactory);
+
+        ISearchOperationCallbackFactory searchCallbackFactory = new PrimaryIndexInstantSearchOperationCallbackFactory(
+                jobId, dataset.getDatasetId(), primaryBloomFilterKeyFields, txnSubsystemProvider,
+                ResourceType.LSM_BTREE);
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
-                AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
                 primaryBloomFilterKeyFields, lowKeyFields, highKeyFields, true, true,
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                        new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
-                                .getBloomFilterFalsePositiveRate()), false,
-                NoOpOperationCallbackFactory.INSTANCE);
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        storageProperties.getBloomFilterFalsePositiveRate()), false, searchCallbackFactory);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
@@ -349,11 +350,10 @@
             fieldPermutation[i] = i;
         }
         TreeIndexBulkLoadOperatorDescriptor treeIndexBulkLoadOp = new TreeIndexBulkLoadOperatorDescriptor(spec,
-                AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER,
-                AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, secondaryFileSplitProvider,
-                secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields,
-                fieldPermutation, fillFactor, false, numElementsHint, false, dataflowHelperFactory,
-                NoOpOperationCallbackFactory.INSTANCE);
+                AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                secondaryFileSplitProvider, secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories,
+                secondaryBloomFilterKeyFields, fieldPermutation, fillFactor, false, numElementsHint, false,
+                dataflowHelperFactory, NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, treeIndexBulkLoadOp,
                 secondaryPartitionConstraint);
         return treeIndexBulkLoadOp;
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 6f64aa2..40e0aa9 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 c91903f..ec62068 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;
@@ -161,121 +150,8 @@
     }
 
     @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,RIDScanOpAndConstraints.second);
-			}
-
-			// 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.LSMRTREE_PROVIDER,
-							AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-							AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-							AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
-									keyType, secondaryComparatorFactories.length), storageProperties
-									.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
-			// Connect the operators.
-			// Create a hash partitioning connector
-			ExternalDatasetDetails edsd = (ExternalDatasetDetails)dataset.getDatasetDetails();
-			IBinaryHashFunctionFactory[] hashFactories = null;
-			if(edsd.getProperties().get(HDFSAdapterFactory.KEY_INPUT_FORMAT).trim().equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
-			{
-				hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
-			}
-			else
-			{
-				hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
-			}	 
-			//select partitioning keys (always the first 2 after secondary keys)
-			int[] keys = new int[2];
-			keys[0] = numSecondaryKeys;
-			keys[1] = numSecondaryKeys + 1;
-
-			IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
-					new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
-			spec.connect(new OneToOneConnectorDescriptor(spec), RIDScanOpAndConstraints.first, 0, asterixAssignOp, 0);
-			if (anySecondaryKeyIsNullable) {
-				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-				spec.connect(hashConn, selectOp, 0, secondaryBulkLoadOp, 0);
-			} else {
-				spec.connect(hashConn, asterixAssignOp, 0, secondaryBulkLoadOp, 0);
-			}
-			spec.addRoot(secondaryBulkLoadOp);
-			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-			return spec;
-		}
-		else
-		{
 
         // Create dummy key provider for feeding the primary index scan. 
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -319,6 +195,5 @@
         spec.addRoot(secondaryBulkLoadOp);
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         return spec;
-		}
     }
 }
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-external-data/pom.xml b/asterix-external-data/pom.xml
index 7966b52..f8d5ea2 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -153,11 +153,6 @@
 			<artifactId>jdom</artifactId>
 			<version>1.0</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 6509da4..8765aae 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
@@ -29,7 +29,6 @@
 import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -42,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;
@@ -53,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;
@@ -63,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;
@@ -174,56 +169,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 {
@@ -397,60 +384,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)
@@ -1161,99 +1135,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 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 ed89cfc..4a02bc5 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
@@ -117,7 +117,7 @@
                 MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
                 MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
                 MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.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 };
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 f0c3a63..8b422c8 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;
@@ -273,30 +264,6 @@
     public IResultSerializerFactoryProvider getResultSerializerFactoryProvider() {
         return resultSerializerFactoryProvider;
     }
-    
-    public String getPropertyValue(String propertyName) {
-        return config.get(propertyName);
-    }
-
-    public void setConfig(Map<String, String> config) {
-        this.config = config;
-    }
-
-    public Map<String, String[]> getAllStores() {
-        return stores;
-    }
-
-    public Map<String, String> getConfig() {
-        return config;
-    }
-
-    public static boolean isOptimizeExternalIndexes() {
-		return optimizeExternalIndexes;
-	}
-    
-    public static void setOptimizeExternalIndexes(boolean optimizeExternalIndexes) {
-		AqlMetadataProvider.optimizeExternalIndexes = optimizeExternalIndexes;
-	}
 
     @Override
     public AqlDataSource findDataSource(AqlSourceId id) throws AlgebricksException {
@@ -435,215 +402,6 @@
 
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
     }
-    
-    @SuppressWarnings("rawtypes")
-	public Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataIndexingRuntime(
-			JobSpecification jobSpec, IAType itemType, Dataset dataset, IDataFormat format)
-					throws AlgebricksException {
-		IGenericDatasetAdapterFactory adapterFactory;
-		IDatasourceAdapter adapter;
-		String adapterName;
-		DatasourceAdapter adapterEntity;
-		String adapterFactoryClassname;
-		ExternalDatasetDetails datasetDetails = null;
-		try {
-			datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-			adapterName = datasetDetails.getAdapter();
-			adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
-					adapterName);
-			if (adapterEntity != null) {
-				adapterFactoryClassname = adapterEntity.getClassname();
-				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-			} else {
-				adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
-				if (adapterFactoryClassname == null) {
-					throw new AlgebricksException(" Unknown adapter :" + adapterName);
-				}
-				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-			}
-
-			adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createIndexingAdapter(
-					wrapProperties(datasetDetails.getProperties()), itemType, null);
-		} catch (AlgebricksException ae) {
-			throw ae;
-		} catch (Exception e) {
-			e.printStackTrace();
-			throw new AlgebricksException("Unable to create adapter " + e);
-		}
-		if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
-				IDatasourceAdapter.AdapterType.READ_WRITE))) {
-			throw new AlgebricksException("external dataset adapter does not support read operation");
-		}
-		ARecordType rt = (ARecordType) itemType;
-		ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
-		RecordDescriptor indexerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
-		ExternalDataIndexingOperatorDescriptor dataIndexScanner = null;
-		List<ExternalFile> files = null;
-		HashMap<String, Integer> filesNumbers = null;
-		if(optimizeExternalIndexes)
-		{
-			try {
-				files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
-			} catch (MetadataException e) {
-				e.printStackTrace();
-				throw new AlgebricksException("Unable to get list of external files from metadata " + e);
-			}
-			
-			filesNumbers = new HashMap<String,Integer>();
-			for(int i=0; i< files.size(); i++)
-			{
-				filesNumbers.put(files.get(i).getFileName(), files.get(i).getFileNumber());
-			}
-			
-			dataIndexScanner = new ExternalDataIndexingOperatorDescriptor(jobSpec,
-					wrapPropertiesEmpty(datasetDetails.getProperties()), rt, indexerDesc, adapterFactory,filesNumbers);
-		}
-		else
-		{
-		dataIndexScanner = new ExternalDataIndexingOperatorDescriptor(jobSpec,
-				wrapPropertiesEmpty(datasetDetails.getProperties()), rt, indexerDesc, adapterFactory,filesNumbers);
-		}
-		AlgebricksPartitionConstraint constraint;
-		try {
-			constraint = adapter.getPartitionConstraint();
-		} catch (Exception e) {
-			throw new AlgebricksException(e);
-		}
-		return new Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint>(dataIndexScanner, constraint);
-	}
-    
-    public ArrayList<ExternalFile> getExternalDatasetFiles(Dataset dataset) throws AlgebricksException
-	{
-    	ArrayList<ExternalFile> files = new ArrayList<ExternalFile>();
-		if(dataset.getDatasetType() != DatasetType.EXTERNAL)
-		{
-			throw new AlgebricksException("Can only get external dataset files");
-		}
-		ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails)dataset.getDatasetDetails();
-		IGenericDatasetAdapterFactory adapterFactory;
-		IDatasourceAdapter adapter;
-		String adapterName;
-		DatasourceAdapter adapterEntity;
-		String adapterFactoryClassname;
-		try {
-			adapterName = datasetDetails.getAdapter();
-			adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
-					adapterName);
-			if (adapterEntity != null) {
-				adapterFactoryClassname = adapterEntity.getClassname();
-				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-			} else {
-				adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
-				if (adapterFactoryClassname == null) {
-					throw new AlgebricksException(" Unknown adapter :" + adapterName);
-				}
-				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-			}
-
-			adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
-					wrapProperties(datasetDetails.getProperties()), null);
-		}
-		catch (Exception e) {
-			e.printStackTrace();
-			throw new AlgebricksException("Unable to create adapter " + e);
-		}
-		
-		try {
-			ArrayList<FileStatus> fileStatuses = ExternalDataFilesMetadataProvider.getHDFSFileStatus((AbstractDatasourceAdapter) adapter);
-			for(int i=0; i<fileStatuses.size(); i++)
-			{
-				files.add(new ExternalFile(dataset.getDataverseName(), dataset.getDatasetName(), new Date(fileStatuses.get(i).getModificationTime()),
-						fileStatuses.get(i).getLen(),
-						fileStatuses.get(i).getPath().toUri().getPath(),
-						i));
-			}
-			return files;
-		} catch (IOException e) {
-			e.printStackTrace();
-			throw new AlgebricksException("Unable to get list of HDFS files " + e);
-		}
-	}
-
-	@SuppressWarnings("rawtypes")
-	public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataAccesByRIDRuntime(
-			JobSpecification jobSpec, Dataset dataset, Index secondaryIndex)
-					throws AlgebricksException {
-		IAType itemType = null;
-		try {
-			itemType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
-		} catch (MetadataException e) {
-			e.printStackTrace();
-			throw new AlgebricksException("Unable to get item type from metadata " + e);
-		}
-		if (itemType.getTypeTag() != ATypeTag.RECORD) {
-			throw new AlgebricksException("Can only scan datasets of records.");
-		}
-
-		ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails)dataset.getDatasetDetails();
-		IGenericDatasetAdapterFactory adapterFactory;
-		IDatasourceAdapter adapter;
-		String adapterName;
-		DatasourceAdapter adapterEntity;
-		String adapterFactoryClassname;
-		try {
-			adapterName = datasetDetails.getAdapter();
-			adapterEntity = MetadataManager.INSTANCE.getAdapter(mdTxnCtx, MetadataConstants.METADATA_DATAVERSE_NAME,
-					adapterName);
-			if (adapterEntity != null) {
-				adapterFactoryClassname = adapterEntity.getClassname();
-				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-			} else {
-				adapterFactoryClassname = adapterFactoryMapping.get(adapterName);
-				if (adapterFactoryClassname == null) {
-					throw new AlgebricksException(" Unknown adapter :" + adapterName);
-				}
-				adapterFactory = (IGenericDatasetAdapterFactory) Class.forName(adapterFactoryClassname).newInstance();
-			}
-
-			adapter = ((IGenericDatasetAdapterFactory) adapterFactory).createAdapter(
-					wrapProperties(datasetDetails.getProperties()), itemType);
-		} catch (AlgebricksException ae) {
-			throw ae;
-		} catch (Exception e) {
-			e.printStackTrace();
-			throw new AlgebricksException("Unable to create adapter " + e);
-		}
-
-		if (!(adapter.getAdapterType().equals(IDatasourceAdapter.AdapterType.READ) || adapter.getAdapterType().equals(
-				IDatasourceAdapter.AdapterType.READ_WRITE))) {
-			throw new AlgebricksException("external dataset adapter does not support read operation");
-		}
-		IDataFormat format = NonTaggedDataFormat.INSTANCE;
-		ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
-		RecordDescriptor outRecDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
-
-		ExternalDataAccessByRIDOperatorDescriptor dataAccessOperator = null;
-		if(optimizeExternalIndexes)
-		{
-			//create the hashmap
-			List<ExternalFile> files=null;
-			try {
-				files = MetadataManager.INSTANCE.getDatasetExternalFiles(mdTxnCtx, dataset);
-			} catch (MetadataException e) {
-				e.printStackTrace();
-				throw new AlgebricksException("Couldn't get file names for access by optimized RIDs",e);
-			}
-			HashMap<Integer, String> filesMapping = new HashMap<Integer, String>();
-			for(int i=0; i < files.size(); i++)
-			{
-				filesMapping.put(files.get(i).getFileNumber(), files.get(i).getFileName());
-			}
-			dataAccessOperator = new ExternalDataAccessByRIDOperatorDescriptor(jobSpec, wrapPropertiesEmpty(datasetDetails.getProperties()),
-					itemType, outRecDesc, adapterFactory, filesMapping);
-		}
-		else
-		{
-			dataAccessOperator = new ExternalDataAccessByRIDOperatorDescriptor(jobSpec, wrapPropertiesEmpty(datasetDetails.getProperties()),
-					itemType, outRecDesc, adapterFactory, null);
-		}
-		Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraints = splitProviderAndPartitionConstraintsForExternalDataset(dataset.getDataverseName(),dataset.getDatasetName(),secondaryIndex.getIndexName());
-		return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataAccessOperator, splitsAndConstraints.second);
-	}
 
     @SuppressWarnings("rawtypes")
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildScannerRuntime(JobSpecification jobSpec,
@@ -746,109 +504,64 @@
     }
 
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
-			List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-			JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] lowKeyFields,
-			int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive, Object implConfig)
-					throws AlgebricksException {
-		boolean isSecondary = true;
-		if(dataset.getDatasetType() == DatasetType.EXTERNAL){
-			try {
-				int numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
-				RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-				int numKeys = numPrimaryKeys;;
-				ITypeTraits[] typeTraits = null;
-				int[] bloomFilterKeyFields;
-				Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-						dataset.getDatasetName(), indexName);
-				int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
-				numKeys += numSecondaryKeys;
-				int keysStartIndex = outputVars.size() - numKeys;
-				typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
-				bloomFilterKeyFields = new int[numSecondaryKeys];
-				for (int i = 0; i < numSecondaryKeys; i++) {
-					bloomFilterKeyFields[i] = i;
-				}
-				IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
-						outputVars, keysStartIndex, numKeys, typeEnv, context);
-				IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-				Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-				try {
-					spPc = splitProviderAndPartitionConstraintsForExternalDataset(dataset.getDataverseName(),
-							dataset.getDatasetName(), indexName);
-				} catch (Exception e) {
-					throw new AlgebricksException(e);
-				}
-				ISearchOperationCallbackFactory searchCallbackFactory = null;
-				searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
-				AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_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,
-								AsterixRuntimeComponentsProvider.LSMBTREE_SECONDARY_PROVIDER, rtcProvider,
-								rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
-								searchCallbackFactory);
-				return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
-			} catch (MetadataException me) {
-				throw new AlgebricksException(me);
-			}
-		}
-		else
-		{
-			try {
-				Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-						dataset.getDatasetName(), dataset.getDatasetName());
-				if (primaryIndex != null) {
-					isSecondary = !indexName.equals(primaryIndex.getIndexName());
-				}
-				int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-				RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-				int numKeys = numPrimaryKeys;
-				int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
-				ITypeTraits[] typeTraits = null;
-				int[] bloomFilterKeyFields;
-				if (isSecondary) {
-					Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-							dataset.getDatasetName(), indexName);
-					int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
-					numKeys += numSecondaryKeys;
-					keysStartIndex = outputVars.size() - numKeys;
-					typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
-					bloomFilterKeyFields = new int[numSecondaryKeys];
-					for (int i = 0; i < numSecondaryKeys; i++) {
-						bloomFilterKeyFields[i] = i;
-					}
-				} else {
-					typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
-							context);
-					bloomFilterKeyFields = new int[numPrimaryKeys];
-					for (int i = 0; i < numPrimaryKeys; i++) {
-						bloomFilterKeyFields[i] = i;
-					}
-				}
-				IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
-						outputVars, keysStartIndex, numKeys, typeEnv, context);
+            List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
+            JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] lowKeyFields,
+            int[] highKeyFields, boolean lowKeyInclusive, boolean highKeyInclusive, Object implConfig)
+            throws AlgebricksException {
+        boolean isSecondary = true;
+        try {
+            Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+                    dataset.getDatasetName(), dataset.getDatasetName());
+            if (primaryIndex != null) {
+                isSecondary = !indexName.equals(primaryIndex.getIndexName());
+            }
+            int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+            RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
+            int numKeys = numPrimaryKeys;
+            int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
+            ITypeTraits[] typeTraits = null;
+            int[] bloomFilterKeyFields;
+            if (isSecondary) {
+                Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
+                        dataset.getDatasetName(), indexName);
+                int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
+                numKeys += numSecondaryKeys;
+                keysStartIndex = outputVars.size() - numKeys;
+                typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
+                bloomFilterKeyFields = new int[numSecondaryKeys];
+                for (int i = 0; i < numSecondaryKeys; i++) {
+                    bloomFilterKeyFields[i] = i;
+                }
+            } else {
+                typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
+                        context);
+                bloomFilterKeyFields = new int[numPrimaryKeys];
+                for (int i = 0; i < numPrimaryKeys; i++) {
+                    bloomFilterKeyFields[i] = i;
+                }
+            }
+            IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+                    outputVars, keysStartIndex, numKeys, typeEnv, context);
 
-				IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-				Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-				try {
-					spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
-							dataset.getDatasetName(), indexName);
-				} catch (Exception e) {
-					throw new AlgebricksException(e);
-				}
+            IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+            Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
+            try {
+                spPc = splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
+                        dataset.getDatasetName(), indexName);
+            } catch (Exception e) {
+                throw new AlgebricksException(e);
+            }
 
-				ISearchOperationCallbackFactory searchCallbackFactory = null;
-				if (isSecondary) {
-					searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
-				} else {
-					JobId jobId = ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId();
-					int datasetId = dataset.getDatasetId();
-					int[] primaryKeyFields = new int[numPrimaryKeys];
-					for (int i = 0; i < numPrimaryKeys; i++) {
-						primaryKeyFields[i] = i;
-					}
+            ISearchOperationCallbackFactory searchCallbackFactory = null;
+            if (isSecondary) {
+                searchCallbackFactory = new SecondaryIndexSearchOperationCallbackFactory();
+            } else {
+                JobId jobId = ((JobEventListenerFactory) jobSpec.getJobletEventListenerFactory()).getJobId();
+                int datasetId = dataset.getDatasetId();
+                int[] primaryKeyFields = new int[numPrimaryKeys];
+                for (int i = 0; i < numPrimaryKeys; i++) {
+                    primaryKeyFields[i] = i;
+                }
 
                 AqlMetadataImplConfig aqlMetadataImplConfig = (AqlMetadataImplConfig) implConfig;
                 ITransactionSubsystemProvider txnSubsystemProvider = new TransactionSubsystemProvider();
@@ -872,14 +585,13 @@
                             rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
                     searchCallbackFactory);
 
-				return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
+            return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
 
-			} catch (MetadataException me) {
-				throw new AlgebricksException(me);
-			}
-		}
-	}
-    
+        } catch (MetadataException me) {
+            throw new AlgebricksException(me);
+        }
+    }
+
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             JobGenContext context, boolean retainInput, Dataset dataset, String indexName, int[] keyFields)
@@ -936,10 +648,10 @@
                     typeTraits, comparatorFactories, keyFields, new LSMRTreeDataflowHelperFactory(
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            new SecondaryIndexOperationTrackerProvider(LSMRTreeIOOperationCallbackFactory.INSTANCE,
+                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
                             storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
@@ -1162,10 +874,10 @@
                     appContext.getIndexLifecycleManagerProvider(), splitsAndConstraint.first, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
                     new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PRIMARY_PROVIDER, storageProperties
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
                                     .getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory, true);
 
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
@@ -1635,20 +1347,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);
         }
@@ -1666,12 +1367,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);
@@ -1762,56 +1457,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",
@@ -1824,8 +1469,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/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