Merge branch 'master' into yingyi/asterix_fix
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
index 693a19e..ba940d6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -39,7 +39,6 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExtensionOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -388,13 +387,6 @@
     public ILogicalOperator visitUnnestMapOperator(UnnestMapOperator op, ILogicalOperator arg) {
         throw new UnsupportedOperationException();
     }
-    
-    @Override
-	public ILogicalOperator visitExternalDataAccessByRIDOperator(
-			ExternalDataAccessByRIDOperator op, ILogicalOperator arg)
-			throws AlgebricksException {
-    	throw new UnsupportedOperationException();
-	}
 
     @Override
     public ILogicalOperator visitUnnestOperator(UnnestOperator op, ILogicalOperator arg) throws AlgebricksException {
@@ -446,5 +438,4 @@
     public Map<LogicalVariable, LogicalVariable> getVariableMapping() {
         return outVarMapping;
     }
-
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
index 6dd11bd..32bfa58 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/CommitRuntime.java
@@ -72,7 +72,7 @@
     @Override
     public void open() throws HyracksDataException {
         try {
-            transactionContext = transactionManager.getTransactionContext(jobId);
+            transactionContext = transactionManager.getTransactionContext(jobId, false);
             transactionContext.setWriteTxn(isWriteTransaction);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
@@ -89,7 +89,11 @@
             pkHash = computePrimaryKeyHashValue(frameTupleReference, primaryKeyFields);
             logRecord.formEntityCommitLogRecord(transactionContext, datasetId, pkHash, frameTupleReference,
                     primaryKeyFields);
-            logMgr.log(logRecord);
+            try {
+                logMgr.log(logRecord);
+            } catch (ACIDException e) {
+                throw new HyracksDataException(e);
+            }
         }
     }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
deleted file mode 100644
index 447555e..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/ExternalDataAccessByRIDPOperator.java
+++ /dev/null
@@ -1,110 +0,0 @@
-package edu.uci.ics.asterix.algebra.operators.physical;
-
-import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
-import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.Index;
-import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractScanOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ExternalDataAccessByRIDOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.physical.AbstractScanPOperator;
-import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-
-public class ExternalDataAccessByRIDPOperator extends AbstractScanPOperator{
-
-    private AqlSourceId datasetId;
-    private Dataset dataset;
-    private ARecordType recordType;
-    private Index secondaryIndex;
-	public ExternalDataAccessByRIDPOperator(AqlSourceId datasetId, Dataset dataset, ARecordType recordType,Index secondaryIndex)
-    {
-    	this.datasetId = datasetId;
-    	this.dataset = dataset;
-    	this.recordType = recordType;
-    	this.secondaryIndex = secondaryIndex;
-    }
-    
-	public Dataset getDataset() {
-		return dataset;
-	}
-
-	public void setDataset(Dataset dataset) {
-		this.dataset = dataset;
-	}
-
-	public ARecordType getRecordType() {
-		return recordType;
-	}
-
-	public void setRecordType(ARecordType recordType) {
-		this.recordType = recordType;
-	}
-	
-	public AqlSourceId getDatasetId() {
-		return datasetId;
-	}
-
-	public void setDatasetId(AqlSourceId datasetId) {
-		this.datasetId = datasetId;
-	}
-	
-	@Override
-	public PhysicalOperatorTag getOperatorTag() {
-		return PhysicalOperatorTag.EXTERNAL_ACCESS_BY_RID;
-	}
-
-	@Override
-	public void computeDeliveredProperties(ILogicalOperator op,
-			IOptimizationContext context) throws AlgebricksException {
-		AqlDataSource ds = new AqlDataSource(datasetId, dataset, recordType);
-        IDataSourcePropertiesProvider dspp = ds.getPropertiesProvider();
-        AbstractScanOperator as = (AbstractScanOperator) op;
-        deliveredProperties = dspp.computePropertiesVector(as.getVariables());
-	}
-
-	@Override
-	public void contributeRuntimeOperator(IHyracksJobBuilder builder,
-			JobGenContext context, ILogicalOperator op,
-			IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas,
-			IOperatorSchema outerPlanSchema) throws AlgebricksException {
-		ExternalDataAccessByRIDOperator edabro = (ExternalDataAccessByRIDOperator) op;
-        ILogicalExpression expr = edabro.getExpressionRef().getValue();
-        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            throw new IllegalStateException();
-        }
-        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-        FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
-        if (!funcIdent.equals(AsterixBuiltinFunctions.EXTERNAL_ACCESS_BY_RID)) {
-            return;
-        }
-        AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> externalAccessByRID = metadataProvider.buildExternalDataAccesByRIDRuntime(
-        		builder.getJobSpec(), dataset,secondaryIndex);
-        builder.contributeHyracksOperator(edabro, externalAccessByRID.first);
-        builder.contributeAlgebricksPartitionConstraint(externalAccessByRID.first, externalAccessByRID.second);
-        ILogicalOperator srcExchange = edabro.getInputs().get(0).getValue();
-        builder.contributeGraphEdge(srcExchange, 0, edabro, 0);
-	}
-	
-	@Override
-	public boolean isMicroOperator() {
-		return false;
-	}
-
-}
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 56fc0e7..af503a6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -220,17 +220,15 @@
                 dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
                         new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
                         storageProperties.getBloomFilterFalsePositiveRate());
             } else {
                 dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
                         new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
                         storageProperties.getBloomFilterFalsePositiveRate());
             }
             LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodUtils.java
index 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/api/common/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index ee9dfae..b528381 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -18,10 +18,8 @@
 
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -95,26 +93,6 @@
     }
 
     @Override
-    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
-    }
-
-    @Override
-    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider(boolean isPrimary) {
-        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
-    }
-
-    @Override
-    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
-    }
-
-    @Override
-    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider() {
-        return AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
-    }
-
-    @Override
     public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID) {
         return asterixAppRuntimeContext.getLSMBTreeOperationTracker(datasetID);
     }
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/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index 5d8896c..565d916 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -30,6 +30,7 @@
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
@@ -127,8 +128,7 @@
                 splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
                         dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                         new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
@@ -180,7 +180,7 @@
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
                                 dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
                                 .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
@@ -267,7 +267,7 @@
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
                             storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -293,7 +293,7 @@
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
                             storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                     splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index de4d075..bf31db7 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -14,18 +14,14 @@
  */
 package edu.uci.ics.asterix.file;
 
-import java.util.ArrayList;
-
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
@@ -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 {
@@ -86,9 +71,8 @@
                 AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                 splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
                         dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset
-                                .getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
                         storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index baf16de..3ae5f6d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -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,24 +49,25 @@
         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,
                 secondaryBloomFilterKeyFields, new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
                         dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset
-                                .getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMBTreeIOOperationCallbackFactory.INSTANCE,
                         storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -87,117 +77,51 @@
         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);
-			}
+        // Assign op.
+        AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
 
-			// Sort by secondary keys.
-			ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc,RIDScanOpAndConstraints.second);
-			AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
-			// Create secondary BTree bulk load op.
+        // If any of the secondary fields are nullable, then add a select op that filters nulls.
+        AlgebricksMetaOperatorDescriptor selectOp = null;
+        if (anySecondaryKeyIsNullable) {
+            selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
+        }
+
+        // Sort by secondary keys.
+        ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
+
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
+        // Create secondary BTree bulk load op.
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
                 spec,
                 numSecondaryKeys,
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-                                LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
-                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
-			IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
-
-			//select partitioning keys (always the first 2 after secondary keys)
-			int[] keys = new int[2];
-			keys[0] = numSecondaryKeys;
-			keys[1] = numSecondaryKeys + 1;
-
-			IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
-					new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
-
-			spec.connect(new OneToOneConnectorDescriptor(spec), RIDScanOpAndConstraints.first, 0, asterixAssignOp, 0);
-			if (anySecondaryKeyIsNullable) {
-				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-				spec.connect(hashConn, selectOp, 0, sortOp, 0);
-			} else {
-				spec.connect(hashConn, asterixAssignOp, 0, sortOp, 0);
-			}
-			spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
-			spec.addRoot(secondaryBulkLoadOp);
-			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-			return spec;
-		}
-		else
-		{
-			JobSpecification spec = JobSpecificationUtils.createJobSpecification();
-
-			// Create dummy key provider for feeding the primary index scan. 
-			AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
-
-			// Create primary index scan op.
-			BTreeSearchOperatorDescriptor primaryScanOp = createPrimaryIndexScanOp(spec);
-
-			// Assign op.
-			AlgebricksMetaOperatorDescriptor asterixAssignOp = createAssignOp(spec, primaryScanOp, numSecondaryKeys);
-
-			// If any of the secondary fields are nullable, then add a select op that filters nulls.
-			AlgebricksMetaOperatorDescriptor selectOp = null;
-			if (anySecondaryKeyIsNullable) {
-				selectOp = createFilterNullsSelectOp(spec, numSecondaryKeys);
-			}
-
-			// Sort by secondary keys.
-			ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
-
-			AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
-			// Create secondary BTree bulk load op.
-        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
-                spec,
-                numSecondaryKeys,
-                new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-                                LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
                                 .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
-			// Connect the operators.
-			spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
-			spec.connect(new OneToOneConnectorDescriptor(spec), primaryScanOp, 0, asterixAssignOp, 0);
-			if (anySecondaryKeyIsNullable) {
-				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-				spec.connect(new OneToOneConnectorDescriptor(spec), selectOp, 0, sortOp, 0);
-			} else {
-				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, sortOp, 0);
-			}
-			spec.connect(new OneToOneConnectorDescriptor(spec), sortOp, 0, secondaryBulkLoadOp, 0);
-			spec.addRoot(secondaryBulkLoadOp);
-			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-			return spec;
-		}
-	}
+        // 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 5da336f..9f80568 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -19,9 +19,6 @@
 import java.io.IOException;
 import java.util.List;
 
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
-import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
@@ -29,19 +26,18 @@
 import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.AsterixAppContextInfo;
@@ -83,7 +79,6 @@
 import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.misc.ConstantTupleSourceOperatorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor;
-import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
@@ -91,7 +86,6 @@
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.dataflow.LSMBTreeDataflowHelperFactory;
 
-
 @SuppressWarnings("rawtypes")
 // TODO: We should eventually have a hierarchy of classes that can create all
 // possible index job specs,
@@ -109,11 +103,11 @@
     protected ISerializerDeserializer payloadSerde;
     protected IFileSplitProvider primaryFileSplitProvider;
     protected AlgebricksPartitionConstraint primaryPartitionConstraint;
-    protected List<String> secondaryKeyFields;
     protected IFileSplitProvider secondaryFileSplitProvider;
     protected AlgebricksPartitionConstraint secondaryPartitionConstraint;
     protected String secondaryIndexName;
     protected boolean anySecondaryKeyIsNullable = false;
+
     protected long numElementsHint;
     protected IBinaryComparatorFactory[] primaryComparatorFactories;
     protected int[] primaryBloomFilterKeyFields;
@@ -122,6 +116,7 @@
     protected int[] secondaryBloomFilterKeyFields;
     protected RecordDescriptor secondaryRecDesc;
     protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
+
     protected IAsterixPropertiesProvider propertiesProvider;
 
     // Prevent public construction. Should be created via createIndexCreator().
@@ -165,68 +160,37 @@
     public abstract JobSpecification buildLoadingJobSpec() throws AsterixException, AlgebricksException;
 
     protected void init(CompiledCreateIndexStatement createIndexStmt, AqlMetadataProvider metadataProvider)
-                        throws AsterixException, AlgebricksException {
-                this.metadataProvider = metadataProvider;
-                dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
-                                : createIndexStmt.getDataverseName();
-                datasetName = createIndexStmt.getDatasetName();
-                secondaryIndexName = createIndexStmt.getIndexName();
-                dataset = metadataProvider.findDataset(dataverseName, datasetName);
-                if (dataset == null) {
-                        throw new AsterixException("Unknown dataset " + datasetName);
-                }
-                if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
-                        //get external dataset details
-                        ExternalDatasetDetails edsd = ((ExternalDatasetDetails)dataset.getDatasetDetails());
-                        //get adapter name
-                        String adapter = edsd.getAdapter();
-                        //if not an hdfs adapter, throw an exception
-                        if(!adapter.equals(HDFSAdapterFactory.HDFS_ADAPTER_NAME) && !adapter.equals(HiveAdapter.class.getName()))
-                        {
-                                throw new AsterixException("Cannot index an external dataset with adapter type(" + adapter + ").");
-                        }
-                        //get the item type
-                        ARecordType externalItemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
-                        //number of primary keys here depends on the file input, 3 for rcfiles and 2 for text and sequence files.
-                        numPrimaryKeys = DatasetUtils.getExternalRIDSize(dataset);
-                        itemType = createExternalItemTypeWithRID(externalItemType);
-                        payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-                        numSecondaryKeys = createIndexStmt.getKeyFields().size();
-                        //splits and constraints <--They don't exist-->
-                        primaryFileSplitProvider = null;
-                        primaryPartitionConstraint = null;
-                        //create secondary split and constraints
-                        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                                        .splitProviderAndPartitionConstraintsForExternalDataset(dataverseName, datasetName,
-                                                        secondaryIndexName);
-                        secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
-                        secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
-                        // Must be called in this order.
-                        setExternalRIDDescAndComparators();
-                        setExternalSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
-                        numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
-                }
-                else
-                {
-                        itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
-                        payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-                        numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-                        numSecondaryKeys = createIndexStmt.getKeyFields().size();
-                        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
-                                        .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
-                        primaryFileSplitProvider = primarySplitsAndConstraint.first;
-                        primaryPartitionConstraint = primarySplitsAndConstraint.second;
-                        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
-                                        .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
-                                                        secondaryIndexName);
-                        secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
-                        secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
-                        // Must be called in this order.
-                        setPrimaryRecDescAndComparators();
-                        setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
-                        numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
-                }
+            throws AsterixException, AlgebricksException {
+        this.metadataProvider = metadataProvider;
+        dataverseName = createIndexStmt.getDataverseName() == null ? metadataProvider.getDefaultDataverseName()
+                : createIndexStmt.getDataverseName();
+        datasetName = createIndexStmt.getDatasetName();
+        secondaryIndexName = createIndexStmt.getIndexName();
+        dataset = metadataProvider.findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AsterixException("Unknown dataset " + datasetName);
         }
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
+        }
+        itemType = (ARecordType) metadataProvider.findType(dataset.getDataverseName(), dataset.getItemTypeName());
+        payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
+        numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+        numSecondaryKeys = createIndexStmt.getKeyFields().size();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadataProvider
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, datasetName);
+        primaryFileSplitProvider = primarySplitsAndConstraint.first;
+        primaryPartitionConstraint = primarySplitsAndConstraint.second;
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadataProvider
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName,
+                        secondaryIndexName);
+        secondaryFileSplitProvider = secondarySplitsAndConstraint.first;
+        secondaryPartitionConstraint = secondarySplitsAndConstraint.second;
+        // Must be called in this order.
+        setPrimaryRecDescAndComparators();
+        setSecondaryRecDescAndComparators(createIndexStmt, metadataProvider);
+        numElementsHint = metadataProvider.getCardinalityPerPartitionHint(dataset);
+    }
 
     protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
         List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
@@ -310,210 +274,6 @@
         return keyProviderOp;
     }
 
-protected ARecordType createExternalItemTypeWithRID(
-                        ARecordType externalItemType) throws AsterixException {
-
-                String[] fieldsNames = new String[externalItemType.getFieldNames().length+numPrimaryKeys];
-                IAType[] fieldsTypes = new IAType[externalItemType.getFieldTypes().length+numPrimaryKeys];
-
-                //add RID fields names and types
-                if(AqlMetadataProvider.isOptimizeExternalIndexes())
-                {
-                        fieldsNames[0] = "_file-number";
-                        fieldsTypes[0] = BuiltinType.AINT32;
-                }
-                else
-                {
-                        fieldsNames[0] = "_file-name";
-                        fieldsTypes[0] = BuiltinType.ASTRING;
-                }
-                fieldsNames[1] = "_byte-location";
-                fieldsTypes[1] = BuiltinType.AINT64;
-                if(numPrimaryKeys == 3)
-                {       
-                        //add the row number for rc files
-                        fieldsNames[2] = "_row-number";
-                        fieldsTypes[2] = BuiltinType.AINT32;
-                }
-                
-                //add the original fields names and types
-                for(int i=0; i < externalItemType.getFieldNames().length; i++)
-                {
-                        fieldsNames[i+numPrimaryKeys] = externalItemType.getFieldNames()[i];
-                        fieldsTypes[i+numPrimaryKeys] = externalItemType.getFieldTypes()[i];
-                }
-                return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
-        }
-
-        protected void setExternalRIDDescAndComparators() throws AlgebricksException {
-
-                ISerializerDeserializer[] externalRecFields = new ISerializerDeserializer[itemType.getFieldNames().length];
-                ITypeTraits[] externalTypeTraits = new ITypeTraits[itemType.getFieldNames().length];
-
-                primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
-                primaryBloomFilterKeyFields = new int[numPrimaryKeys];
-                ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
-                
-                if(AqlMetadataProvider.isOptimizeExternalIndexes())
-                {
-                        primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true);
-                }
-                else
-                {
-                        primaryComparatorFactories[0] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.ASTRING, true);
-                }
-                primaryComparatorFactories[1] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT64, true);
-
-                primaryBloomFilterKeyFields[0]=0;
-                primaryBloomFilterKeyFields[1]=1;
-
-                if(numPrimaryKeys == 3)
-                {
-                        primaryComparatorFactories[2] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(BuiltinType.AINT32, true);
-                        primaryBloomFilterKeyFields[2]=2;
-                }
-
-                for(int i=0; i < itemType.getFieldNames().length; i++)
-                {
-                        externalRecFields[i] = serdeProvider.getSerializerDeserializer(itemType.getFieldTypes()[i]); 
-                        externalTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType.getFieldTypes()[i]);
-                }
-                primaryRecDesc = new RecordDescriptor(externalRecFields, externalTypeTraits);
-        }
-
-protected void setExternalSecondaryRecDescAndComparators(CompiledCreateIndexStatement createIndexStmt,
-                        AqlMetadataProvider metadataProvider) throws AlgebricksException, AsterixException {
-                secondaryKeyFields = createIndexStmt.getKeyFields();
-                secondaryFieldAccessEvalFactories = new ICopyEvaluatorFactory[numSecondaryKeys+ numPrimaryKeys];
-                secondaryComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys + numPrimaryKeys];
-                secondaryBloomFilterKeyFields = new int[numSecondaryKeys];
-                ISerializerDeserializer[] secondaryRecFields = new ISerializerDeserializer[numPrimaryKeys + numSecondaryKeys];
-                ITypeTraits[] secondaryTypeTraits = new ITypeTraits[numSecondaryKeys + numPrimaryKeys];
-                ISerializerDeserializerProvider serdeProvider = metadataProvider.getFormat().getSerdeProvider();
-                ITypeTraitProvider typeTraitProvider = metadataProvider.getFormat().getTypeTraitProvider();
-                IBinaryComparatorFactoryProvider comparatorFactoryProvider = metadataProvider.getFormat()
-                                .getBinaryComparatorFactoryProvider();
-
-                for (int i = 0; i < numSecondaryKeys; i++) {
-                        secondaryFieldAccessEvalFactories[i] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                                        itemType, secondaryKeyFields.get(i), 0);
-                        Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
-                        IAType keyType = keyTypePair.first;
-                        anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
-                        ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
-                        secondaryRecFields[i] = keySerde;
-                        secondaryComparatorFactories[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
-                        secondaryTypeTraits[i] = typeTraitProvider.getTypeTrait(keyType);
-                        secondaryBloomFilterKeyFields[i] = i;
-                }
-
-                if(AqlMetadataProvider.isOptimizeExternalIndexes())
-                {
-                        secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                                itemType, "_file-number", 0);
-                }
-                else
-                {
-                        secondaryFieldAccessEvalFactories[numSecondaryKeys] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                                        itemType, "_file-name", 0);
-                }
-                secondaryFieldAccessEvalFactories[numSecondaryKeys+1] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                                itemType, "_byte-location", 0);
-                if(numPrimaryKeys == 3)
-                {
-                        secondaryFieldAccessEvalFactories[numSecondaryKeys+2] = metadataProvider.getFormat().getFieldAccessEvaluatorFactory(
-                                        itemType, "_row-number", 0);
-                }
-
-                for (int i = 0; i < numPrimaryKeys; i++) {
-                        secondaryRecFields[numSecondaryKeys + i] = primaryRecDesc.getFields()[i];
-                        secondaryTypeTraits[numSecondaryKeys + i] = primaryRecDesc.getTypeTraits()[i];
-                        secondaryComparatorFactories[numSecondaryKeys + i] = primaryComparatorFactories[i];
-                }
-                secondaryRecDesc = new RecordDescriptor(secondaryRecFields, secondaryTypeTraits);
-        }
-
-protected Pair<ExternalDataIndexingOperatorDescriptor, AlgebricksPartitionConstraint> createExternalIndexingOp(JobSpecification spec) throws Exception {
-                Pair<ExternalDataIndexingOperatorDescriptor,AlgebricksPartitionConstraint> indexingOpAndConstraints = metadataProvider.buildExternalDataIndexingRuntime(spec, itemType, dataset, NonTaggedDataFormat.INSTANCE);
-                AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexingOpAndConstraints.first,
-                                indexingOpAndConstraints.second);
-                return indexingOpAndConstraints;
-        }
-
-protected AlgebricksMetaOperatorDescriptor createExternalAssignOp(JobSpecification spec) throws AlgebricksException {
-                int[] outColumns = new int[numSecondaryKeys + numPrimaryKeys];
-                int[] projectionList = new int[numSecondaryKeys + numPrimaryKeys];
-                for (int i = 0; i < numSecondaryKeys + numPrimaryKeys; i++) {
-                        outColumns[i] = i;
-                        projectionList[i] = i;
-                }
-
-                IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[secondaryFieldAccessEvalFactories.length];
-                for (int i = 0; i < secondaryFieldAccessEvalFactories.length; ++i) {
-                        sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
-                                        secondaryFieldAccessEvalFactories[i]);
-                }
-                AssignRuntimeFactory assign = new AssignRuntimeFactory(outColumns, sefs, projectionList);
-                AlgebricksMetaOperatorDescriptor asterixAssignOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
-                                new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { secondaryRecDesc });
-                return asterixAssignOp;
-        }
-
-        protected ExternalSortOperatorDescriptor createSortOp(JobSpecification spec,
-                        IBinaryComparatorFactory[] secondaryComparatorFactories, RecordDescriptor secondaryRecDesc,
-                        AlgebricksPartitionConstraint partitionConstraints) {
-                int[] sortFields = new int[secondaryComparatorFactories.length];
-                for (int i = 0; i < secondaryComparatorFactories.length; i++) {
-                        sortFields[i] = i;
-                }
-                ExternalSortOperatorDescriptor sortOp = new ExternalSortOperatorDescriptor(spec,
-                                physOptConf.getMaxFramesExternalSort(), sortFields, secondaryComparatorFactories, secondaryRecDesc);
-                AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, sortOp, partitionConstraints);
-                return sortOp;
-        }
-
-protected ARecordType createSecondaryItemType(ARecordType externalItemType, boolean isRCFile) throws AsterixException
-        {
-
-                String[] fieldsNames = new String[numSecondaryKeys+numPrimaryKeys];
-                IAType[] fieldsTypes = new IAType[numSecondaryKeys+numPrimaryKeys];
-
-                //first create the secondary index fields
-                for(int i=0; i<numSecondaryKeys; i++)
-                {
-                        fieldsNames[i] = secondaryKeyFields.get(i);
-                        try {
-                                fieldsTypes[i] = externalItemType.getFieldType(fieldsNames[i]);
-                        } catch (IOException e) {
-                                // TODO Auto-generated catch block
-                                throw new AsterixException(e);
-                        }
-                }
-
-                //second add RID fields (File name or number and byte location)
-                if(AqlMetadataProvider.isOptimizeExternalIndexes())
-                {
-                        fieldsNames[numSecondaryKeys] = "_file-number";
-                        fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
-                }
-                else
-                {
-                        fieldsNames[numSecondaryKeys] = "_file-name";
-                        fieldsTypes[numSecondaryKeys] = BuiltinType.ASTRING;
-                }
-                fieldsNames[numSecondaryKeys+1] = "_byte-location";
-                fieldsTypes[numSecondaryKeys+1] = BuiltinType.AINT64;
-
-                if(isRCFile)
-                {
-                        fieldsNames[numSecondaryKeys+2] = "_row-Number";
-                        fieldsTypes[numSecondaryKeys+2] = BuiltinType.AINT32;
-                }
-
-                //return type
-                return new ARecordType(externalItemType.getTypeName(), fieldsNames, fieldsTypes, externalItemType.isOpen());
-        }
-
     protected BTreeSearchOperatorDescriptor createPrimaryIndexScanOp(JobSpecification spec) throws AlgebricksException {
         // -Infinity
         int[] lowKeyFields = null;
@@ -537,7 +297,7 @@
                 new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new PrimaryIndexOperationTrackerProvider(
                                 dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE,
                         storageProperties.getBloomFilterFalsePositiveRate()), false, searchCallbackFactory);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
@@ -633,4 +393,3 @@
         return asterixSelectOp;
     }
 }
-
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index 6f64aa2..72d0d70 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -283,23 +283,17 @@
         if (!isPartitioned) {
             return new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
                     dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    new SecondaryIndexOperationTrackerProvider(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                            dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
                     storageProperties.getBloomFilterFalsePositiveRate());
         } else {
             return new PartitionedLSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(
                     dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                    new SecondaryIndexOperationTrackerProvider(LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
-                            dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                     AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                    LSMInvertedIndexIOOperationCallbackFactory.INSTANCE,
                     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 6ce694c..4d8118d 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -18,47 +18,36 @@
 
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixVirtualBufferCacheProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
-import edu.uci.ics.asterix.external.util.ExternalIndexHashPartitionComputerFactory;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Index;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
-import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.SecondaryIndexOperationTrackerProvider;
 import edu.uci.ics.asterix.transaction.management.resource.LSMRTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.translator.CompiledStatements.CompiledCreateIndexStatement;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.ConnectorPolicyAssignmentPolicy;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.meta.AlgebricksMetaOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.IConnectorDescriptor;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.dataflow.BTreeSearchOperatorDescriptor;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
@@ -102,10 +91,9 @@
                 new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
                         primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                         AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-                                LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
-                                keyType, secondaryComparatorFactories.length), storageProperties
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        LSMRTreeIOOperationCallbackFactory.INSTANCE, AqlMetadataProvider.proposeLinearizer(keyType,
+                                secondaryComparatorFactories.length), storageProperties
                                 .getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
@@ -161,121 +149,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);
-			}
-
-			// Create secondary RTree bulk load op.
-			AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
-			TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
-					spec,
-					numNestedSecondaryKeyFields,
-					new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-	                        primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-	                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-	                                LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-	                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-	                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
-	                                keyType, secondaryComparatorFactories.length), storageProperties
-	                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
-			// Connect the operators.
-			// Create a hash partitioning connector
-			ExternalDatasetDetails edsd = (ExternalDatasetDetails)dataset.getDatasetDetails();
-			IBinaryHashFunctionFactory[] hashFactories = null;
-			if(edsd.getProperties().get(HDFSAdapterFactory.KEY_INPUT_FORMAT).trim().equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
-			{
-				hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
-			}
-			else
-			{
-				hashFactories = DatasetUtils.computeExternalDataKeysBinaryHashFunFactories(dataset, NonTaggedDataFormat.INSTANCE.getBinaryHashFunctionFactoryProvider());
-			}	 
-			//select partitioning keys (always the first 2 after secondary keys)
-			int[] keys = new int[2];
-			keys[0] = numSecondaryKeys;
-			keys[1] = numSecondaryKeys + 1;
-
-			IConnectorDescriptor hashConn = new MToNPartitioningConnectorDescriptor(spec,
-					new ExternalIndexHashPartitionComputerFactory(keys, hashFactories));
-			spec.connect(new OneToOneConnectorDescriptor(spec), RIDScanOpAndConstraints.first, 0, asterixAssignOp, 0);
-			if (anySecondaryKeyIsNullable) {
-				spec.connect(new OneToOneConnectorDescriptor(spec), asterixAssignOp, 0, selectOp, 0);
-				spec.connect(hashConn, selectOp, 0, secondaryBulkLoadOp, 0);
-			} else {
-				spec.connect(hashConn, asterixAssignOp, 0, secondaryBulkLoadOp, 0);
-			}
-			spec.addRoot(secondaryBulkLoadOp);
-			spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
-			return spec;
-		}
-		else
-		{
 
         // Create dummy key provider for feeding the primary index scan. 
         AbstractOperatorDescriptor keyProviderOp = createDummyKeyProviderOp(spec);
@@ -295,17 +170,14 @@
 
         AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         // Create secondary RTree bulk load op.
-        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
-                spec,
-                numNestedSecondaryKeyFields,
-                new LSMRTreeDataflowHelperFactory(valueProviderFactories, RTreePolicyType.RTREE,
-                        primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, new SecondaryIndexOperationTrackerProvider(
-                                LSMRTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, AqlMetadataProvider.proposeLinearizer(
-                                keyType, secondaryComparatorFactories.length), storageProperties
-                                .getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
+        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec,
+                numNestedSecondaryKeyFields, new LSMRTreeDataflowHelperFactory(valueProviderFactories,
+                        RTreePolicyType.RTREE, primaryComparatorFactories, new AsterixVirtualBufferCacheProvider(
+                                dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                        new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, LSMRTreeIOOperationCallbackFactory.INSTANCE,
+                        AqlMetadataProvider.proposeLinearizer(keyType, secondaryComparatorFactories.length),
+                        storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
@@ -319,7 +191,5 @@
         spec.addRoot(secondaryBulkLoadOp);
         spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy());
         return spec;
-		}
     }
 }
-
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
index 8935d5d..b56fe7c 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta16/meta16.1.adm
@@ -1,9 +1,8 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 18:54:03 PST 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
index a44cbc0..d193dd1 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
@@ -1,63 +1,62 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModDate", "FieldType": "datetime" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:30 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
index 6037036..607bfd1 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta19/meta19.1.adm
@@ -1,12 +1,11 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
index 8935d5d..0078603 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_dataset/metadata_dataset.1.adm
@@ -1,9 +1,9 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 2, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 8, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 3, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 1, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "DataTypeName": "ExternalFileRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "FileNumber" ], "PrimaryKey": [ "DataverseName", "DatasetName", "FileNumber" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 9, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 7, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 4, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 5, "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "DatasetId": 6, "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "DataTypeName": "DatasetRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName" ], "PrimaryKey": [ "DataverseName", "DatasetName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "DataTypeName": "DatasourceAdapterRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name" ], "PrimaryKey": [ "DataverseName", "Name" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "DataTypeName": "DatatypeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatatypeName" ], "PrimaryKey": [ "DataverseName", "DatatypeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "DataTypeName": "DataverseRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName" ], "PrimaryKey": [ "DataverseName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "DataTypeName": "FunctionRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "Name", "Arity" ], "PrimaryKey": [ "DataverseName", "Name", "Arity" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "DataTypeName": "IndexRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "DataverseName", "DatasetName", "IndexName" ], "PrimaryKey": [ "DataverseName", "DatasetName", "IndexName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "DataTypeName": "NodeRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "NodeName" ], "PrimaryKey": [ "NodeName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "DataTypeName": "NodeGroupRecordType", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "GroupName" ], "PrimaryKey": [ "GroupName" ], "GroupName": "MetadataGroup" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index a44cbc0..cecdb85 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -1,63 +1,62 @@
-{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "DatasetId", "FieldType": "int32" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "ExternalFileRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "FileNumber", "FieldType": "int32" }, { "FieldName": "FileName", "FieldType": "string" }, { "FieldName": "FileSize", "FieldType": "int64" }, { "FieldName": "FileModDate", "FieldType": "datetime" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" }, { "FieldName": "PendingOp", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:02 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
-{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Sat Aug 31 20:36:01 AST 2013" }
\ No newline at end of file
+{ "DataverseName": "Metadata", "DatatypeName": "DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "DataTypeName", "FieldType": "string" }, { "FieldName": "DatasetType", "FieldType": "string" }, { "FieldName": "InternalDetails", "FieldType": "Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "ExternalDetails", "FieldType": "Field_ExternalDetails_in_DatasetRecordType" }, { "FieldName": "FeedDetails", "FieldType": "Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Hints", "FieldType": "Field_Hints_in_DatasetRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatasourceAdapterRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Classname", "FieldType": "string" }, { "FieldName": "Type", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatatypeName", "FieldType": "string" }, { "FieldName": "Derived", "FieldType": "Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "DataverseRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DataFormat", "FieldType": "string" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FieldName", "FieldType": "string" }, { "FieldName": "FieldType", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "Field_Hints_in_DatasetRecordType_ItemType", "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Hints_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_NodeNames_in_NodeGroupRecordType", "Derived": { "Tag": "UNORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": "string", "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Params_in_FunctionRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType_ItemType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Value", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_SearchKey_in_IndexRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "UNION", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": [ "null", "string" ], "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "FunctionRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "Name", "FieldType": "string" }, { "FieldName": "Arity", "FieldType": "string" }, { "FieldName": "Params", "FieldType": "Field_Params_in_FunctionRecordType" }, { "FieldName": "ReturnType", "FieldType": "string" }, { "FieldName": "Definition", "FieldType": "string" }, { "FieldName": "Language", "FieldType": "string" }, { "FieldName": "Kind", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "IndexRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DataverseName", "FieldType": "string" }, { "FieldName": "DatasetName", "FieldType": "string" }, { "FieldName": "IndexName", "FieldType": "string" }, { "FieldName": "IndexStructure", "FieldType": "string" }, { "FieldName": "SearchKey", "FieldType": "Field_SearchKey_in_IndexRecordType" }, { "FieldName": "IsPrimary", "FieldType": "boolean" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeGroupRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "NodeNames", "FieldType": "Field_NodeNames_in_NodeGroupRecordType" }, { "FieldName": "Timestamp", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "NodeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": false, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "NodeName", "FieldType": "string" }, { "FieldName": "NumberOfCores", "FieldType": "int32" }, { "FieldName": "WorkingMemorySize", "FieldType": "int32" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "Tag", "FieldType": "string" }, { "FieldName": "IsAnonymous", "FieldType": "boolean" }, { "FieldName": "EnumValues", "FieldType": "Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Record", "FieldType": "Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "Union", "FieldType": "Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "UnorderedList", "FieldType": "Field_UnorderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" }, { "FieldName": "OrderedList", "FieldType": "Field_OrderedList_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_EnumValues_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_ExternalDetails_in_DatasetRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" }, { "FieldName": "DatasourceAdapter", "FieldType": "string" }, { "FieldName": "Properties", "FieldType": "Field_Properties_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Function", "FieldType": "Field_Function_in_Type_#1_UnionType_Field_FeedDetails_in_DatasetRecordType" }, { "FieldName": "Status", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "FileStructure", "FieldType": "string" }, { "FieldName": "PartitioningStrategy", "FieldType": "string" }, { "FieldName": "PartitioningKey", "FieldType": "Field_PartitioningKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "PrimaryKey", "FieldType": "Field_PrimaryKey_in_Type_#1_UnionType_Field_InternalDetails_in_DatasetRecordType" }, { "FieldName": "GroupName", "FieldType": "string" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "RECORD", "IsAnonymous": true, "EnumValues": null, "Record": { "IsOpen": true, "Fields": [ { "FieldName": "IsOpen", "FieldType": "boolean" }, { "FieldName": "Fields", "FieldType": "Field_Fields_in_Type_#1_UnionType_Field_Record_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType" } ] }, "Union": null, "UnorderedList": null, "OrderedList": null }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "Type_#1_UnionType_Field_Union_in_Type_#1_UnionType_Field_Derived_in_DatatypeRecordType", "Derived": { "Tag": "ORDEREDLIST", "IsAnonymous": true, "EnumValues": null, "Record": null, "Union": null, "UnorderedList": null, "OrderedList": "string" }, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "boolean", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int16", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int32", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int64", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "int8", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "interval", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "line", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "null", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "point3d", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "polygon", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
index 6037036..607bfd1 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_index/metadata_index.1.adm
@@ -1,12 +1,11 @@
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Sat Aug 31 20:36:02 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "ExternalFile", "IndexName": "ExternalFile", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "FileNumber" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
-{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Sat Aug 31 20:36:01 AST 2013", "PendingOp": 0 }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "Dataset", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataset", "IndexName": "GroupName", "IndexStructure": "BTREE", "SearchKey": [ "GroupName", "DataverseName", "DatasetName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "DatasourceAdapter", "IndexName": "DatasourceAdapter", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "Datatype", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatatypeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Datatype", "IndexName": "DatatypeName", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "NestedDatatypeName", "TopDatatypeName" ], "IsPrimary": false, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Dataverse", "IndexName": "Dataverse", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Function", "IndexName": "Function", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "Name", "Arity" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Index", "IndexName": "Index", "IndexStructure": "BTREE", "SearchKey": [ "DataverseName", "DatasetName", "IndexName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Node", "IndexName": "Node", "IndexStructure": "BTREE", "SearchKey": [ "NodeName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
+{ "DataverseName": "Metadata", "DatasetName": "Nodegroup", "IndexName": "Nodegroup", "IndexStructure": "BTREE", "SearchKey": [ "GroupName" ], "IsPrimary": true, "Timestamp": "Mon Nov 05 10:33:40 PST 2012" }
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.ddl.aql
new file mode 100644
index 0000000..754ea81
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.ddl.aql
@@ -0,0 +1,3 @@
+drop dataverse test if exists;
+create dataverse test;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.3.query.aql
new file mode 100644
index 0000000..79de83c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.3.query.aql
@@ -0,0 +1,19 @@
+use dataverse test;
+set import-private-functions 'true';
+
+let $v1 := [ "query", "processing", "in", "multidatabase", "systems" ] 
+let $v2 := [ "query", "processing", "in", "object", "oriented", "database", "systems" ]
+let $v3 := [ "dynamic", "query", "optimization", "and", "query", "processing", "in", "multidatabase", "systems", "1" ]
+let $v4 := [ "transaction", "management", "in", "multidatabase", "systems" ]
+let $v5 := [ "overview", "of", "multidatabase", "transaction", "management" ]
+
+
+let $results :=
+[
+	similarity-jaccard-check($v1, $v2, 0.5f),
+	similarity-jaccard-check($v1, $v3, 0.5f),
+	similarity-jaccard-check($v4, $v5, 0.5f)
+]
+
+for $i in $results
+return $i
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
index 445122f..7be9c57 100644
--- a/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/cross-dataverse/cross-dv19/cross-dv19.1.adm
@@ -1,7 +1,7 @@
-{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 125, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 119, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 122, "PendingOp": 0 }
-{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 123, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 120, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:36 AST 2013", "DatasetId": 121, "PendingOp": 0 }
-{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Mon Sep 02 08:36:37 AST 2013", "DatasetId": 124, "PendingOp": 0 }
+{ "DataverseName": "test1", "DatasetName": "TwitterData", "DataTypeName": "Tweet", "DatasetType": "EXTERNAL", "InternalDetails": null, "ExternalDetails": { "DatasourceAdapter": "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter", "Properties": [ { "Name": "path", "Value": "nc1://data/twitter/extrasmalltweets.txt" }, { "Name": "format", "Value": "adm" } ] }, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "t1", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test1", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test2", "DatasetName": "t2", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test2", "DatasetName": "t3", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
+{ "DataverseName": "test2", "DatasetName": "t4", "DataTypeName": "testtype", "DatasetType": "INTERNAL", "InternalDetails": { "FileStructure": "BTREE", "PartitioningStrategy": "HASH", "PartitioningKey": [ "id" ], "PrimaryKey": [ "id" ], "GroupName": "DEFAULT_NG_ALL_NODES" }, "ExternalDetails": null, "FeedDetails": null, "Hints": {{  }}, "Timestamp": "Tue Jan 29 19:04:36 PST 2013" }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm
index 10357e5..b79bfe0 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-aqlplus_1/dblp-aqlplus_1.1.adm
@@ -1,3 +1,4 @@
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "dblp2": { "id": 24, "dblpid": "books/acm/kim95/OzsuB95", "title": "Query Processing in Object-Oriented Database Systems.", "authors": "M. Tamer Özsu José A. Blakeley", "misc": "2002-01-03 146-174 1995 Modern Database Systems db/books/collections/kim95.html#OzsuB95" } }
 { "dblp": { "id": 81, "dblpid": "journals/siamcomp/AspnesW96", "title": "Randomized Consensus in Expected O(n log² n) Operations Per Processor.", "authors": "James Aspnes Orli Waarts", "misc": "2002-01-03 1024-1044 1996 25 SIAM J. Comput. 5 db/journals/siamcomp/siamcomp25.html#AspnesW96" }, "dblp2": { "id": 82, "dblpid": "conf/focs/AspnesW92", "title": "Randomized Consensus in Expected O(n log ^2 n) Operations Per Processor", "authors": "James Aspnes Orli Waarts", "misc": "2006-04-25 137-146 conf/focs/FOCS33 1992 FOCS db/conf/focs/focs92.html#AspnesW92" } }
 { "dblp": { "id": 83, "dblpid": "journals/siamcomp/Bloniarz83", "title": "A Shortest-Path Algorithm with Expected Time O(n² log n log* n).", "authors": "Peter A. Bloniarz", "misc": "2002-01-03 588-600 1983 12 SIAM J. Comput. 3 db/journals/siamcomp/siamcomp12.html#Bloniarz83" }, "dblp2": { "id": 84, "dblpid": "conf/stoc/Bloniarz80", "title": "A Shortest-Path Algorithm with Expected Time O(n^2 log n log ^* n)", "authors": "Peter A. Bloniarz", "misc": "2006-04-25 378-384 conf/stoc/STOC12 1980 STOC db/conf/stoc/stoc80.html#Bloniarz80" } }
 { "dblp": { "id": 83, "dblpid": "journals/siamcomp/Bloniarz83", "title": "A Shortest-Path Algorithm with Expected Time O(n² log n log* n).", "authors": "Peter A. Bloniarz", "misc": "2002-01-03 588-600 1983 12 SIAM J. Comput. 3 db/journals/siamcomp/siamcomp12.html#Bloniarz83" }, "dblp2": { "id": 87, "dblpid": "journals/siamcomp/MoffatT87", "title": "An All Pairs Shortest Path Algorithm with Expected Time O(n² log n).", "authors": "Alistair Moffat Tadao Takaoka", "misc": "2002-01-03 1023-1031 1987 16 SIAM J. Comput. 6 db/journals/siamcomp/siamcomp16.html#MoffatT87" } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm
index 9a58a6d..7848ca1 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_1/dblp-csx-aqlplus_1.1.adm
@@ -1,5 +1,6 @@
 { "dblp": { "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }, "csx": { "id": 1, "csxid": "oai CiteSeerXPSU 10.1.1.39.1830", "title": "Object SQL - A Language for the Design and Implementation of Object Databases", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Dan Fishman Mike Heytens William Kent", "misc": "2009-04-13 ly, a function application expression consists of two expressions  a function reference (labelled func_ref in Figure 3 line 2), and an argument (labelled arg). The func_ref expression evaluates to a (generic or specific) function identifier, which may be the same as the function that the expression is a part of, thus allowing recursive function invocations. The expression labelled arg evaluates to an arbitrary object or aggregate object. The semantics of evaluating function applications was discussed in detail in section 2. For example, to set the name of a person, we evaluate the following expression   FunAssign(function name.person) (p1,'John')  In this example, the first expression is itself a function call, applying the function FunAssign to the function name.person (an example of a specific function reference). This returns the oid of the function that sets a person's name, which is subsequently applied to a tuple of two elements, the oid of the person and the new name (a string o... CiteSeerX ACM Press 2009-04-13 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.39.1830 http //www.tu-chemnitz.de/~igrdb/docs/OpenODB/osql.ps.gz en 10.1.1.31.2534 10.1.1.28.4658 10.1.1.44.5947 10.1.1.39.199 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "csx": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems  topics covered include active database rule models and languages, rule execution semantics, and implementation issues.  1 Introduction  Conventional database systems are passive  they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "csx": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction  The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema.  The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer.  When a global query is submitted, it is decomposed into two types of queries [1]   -- subqueries, operating on sharable data items from local databases,  -- assembling queries, consisting of, CiteSeerX  2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "csx": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures  and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application  of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of  different types) over different processing entities. Such applications are referred to as transactional  workflows. In this chapter we discuss the specification of such workflows and the issues involved in their  execution.  1 What is a Workflow?  Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems  it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments.  We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX  2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems  it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm
index 9a58a6d..7848ca1 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_2/dblp-csx-aqlplus_2.1.adm
@@ -1,5 +1,6 @@
 { "dblp": { "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }, "csx": { "id": 1, "csxid": "oai CiteSeerXPSU 10.1.1.39.1830", "title": "Object SQL - A Language for the Design and Implementation of Object Databases", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Dan Fishman Mike Heytens William Kent", "misc": "2009-04-13 ly, a function application expression consists of two expressions  a function reference (labelled func_ref in Figure 3 line 2), and an argument (labelled arg). The func_ref expression evaluates to a (generic or specific) function identifier, which may be the same as the function that the expression is a part of, thus allowing recursive function invocations. The expression labelled arg evaluates to an arbitrary object or aggregate object. The semantics of evaluating function applications was discussed in detail in section 2. For example, to set the name of a person, we evaluate the following expression   FunAssign(function name.person) (p1,'John')  In this example, the first expression is itself a function call, applying the function FunAssign to the function name.person (an example of a specific function reference). This returns the oid of the function that sets a person's name, which is subsequently applied to a tuple of two elements, the oid of the person and the new name (a string o... CiteSeerX ACM Press 2009-04-13 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.39.1830 http //www.tu-chemnitz.de/~igrdb/docs/OpenODB/osql.ps.gz en 10.1.1.31.2534 10.1.1.28.4658 10.1.1.44.5947 10.1.1.39.199 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "csx": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems  topics covered include active database rule models and languages, rule execution semantics, and implementation issues.  1 Introduction  Conventional database systems are passive  they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "csx": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction  The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema.  The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer.  When a global query is submitted, it is decomposed into two types of queries [1]   -- subqueries, operating on sharable data items from local databases,  -- assembling queries, consisting of, CiteSeerX  2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "csx": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures  and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application  of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of  different types) over different processing entities. Such applications are referred to as transactional  workflows. In this chapter we discuss the specification of such workflows and the issues involved in their  execution.  1 What is a Workflow?  Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems  it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments.  We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX  2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems  it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm
index 9a58a6d..7848ca1 100644
--- a/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/fuzzyjoin/dblp-csx-aqlplus_3/dblp-csx-aqlplus_3.1.adm
@@ -1,5 +1,6 @@
 { "dblp": { "id": 1, "dblpid": "books/acm/kim95/AnnevelinkACFHK95", "title": "Object SQL - A Language for the Design and Implementation of Object Databases.", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Daniel H. Fishman Michael L. Heytens William Kent", "misc": "2002-01-03 42-68 1995 Modern Database Systems db/books/collections/kim95.html#AnnevelinkACFHK95" }, "csx": { "id": 1, "csxid": "oai CiteSeerXPSU 10.1.1.39.1830", "title": "Object SQL - A Language for the Design and Implementation of Object Databases", "authors": "Jurgen Annevelink Rafiul Ahad Amelia Carlson Dan Fishman Mike Heytens William Kent", "misc": "2009-04-13 ly, a function application expression consists of two expressions  a function reference (labelled func_ref in Figure 3 line 2), and an argument (labelled arg). The func_ref expression evaluates to a (generic or specific) function identifier, which may be the same as the function that the expression is a part of, thus allowing recursive function invocations. The expression labelled arg evaluates to an arbitrary object or aggregate object. The semantics of evaluating function applications was discussed in detail in section 2. For example, to set the name of a person, we evaluate the following expression   FunAssign(function name.person) (p1,'John')  In this example, the first expression is itself a function call, applying the function FunAssign to the function name.person (an example of a specific function reference). This returns the oid of the function that sets a person's name, which is subsequently applied to a tuple of two elements, the oid of the person and the new name (a string o... CiteSeerX ACM Press 2009-04-13 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.39.1830 http //www.tu-chemnitz.de/~igrdb/docs/OpenODB/osql.ps.gz en 10.1.1.31.2534 10.1.1.28.4658 10.1.1.44.5947 10.1.1.39.199 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "csx": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems  topics covered include active database rule models and languages, rule execution semantics, and implementation issues.  1 Introduction  Conventional database systems are passive  they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "dblp": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "csx": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction  The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema.  The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer.  When a global query is submitted, it is decomposed into two types of queries [1]   -- subqueries, operating on sharable data items from local databases,  -- assembling queries, consisting of, CiteSeerX  2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "csx": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures  and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application  of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of  different types) over different processing entities. Such applications are referred to as transactional  workflows. In this chapter we discuss the specification of such workflows and the issues involved in their  execution.  1 What is a Workflow?  Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems  it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments.  We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX  2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "dblp": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "csx": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems  it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
index f6c6049..194ac75 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "jacc": 0.527027f }
 { "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management", "jacc": 0.55932206f }
 { "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management", "jacc": 0.55932206f }
 { "a": "Active Database Systems.", "b": "Active Database Systems", "jacc": 0.95454544f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm
index b528fe7..7594b6d 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.1.adm
@@ -1,6 +1,7 @@
 { "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management" }
 { "a": "Transaction Management in Multidatabase Systems.", "b": "Overview of Multidatabase Transaction Management" }
 { "a": "Active Database Systems.", "b": "Active Database Systems" }
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1." }
 { "a": "Specification and Execution of Transactional Workflows.", "b": "Specification and Execution of Transactional Workflows" }
 { "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
 { "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm
index 29e66a2..2a29873 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "jacc": 0.5f }
 { "a": "Active Database Systems.", "b": "Active Database Systems", "jacc": 1.0f }
 { "a": "Specification and Execution of Transactional Workflows.", "b": "Specification and Execution of Transactional Workflows", "jacc": 1.0f }
 { "a": "Integrated Office Systems.", "b": "Integrated Office Systems", "jacc": 1.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm
index 2bd52e3..d01a56f 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.1.adm
@@ -1,4 +1,5 @@
 { "a": "Active Database Systems.", "b": "Active Database Systems" }
+{ "a": "Query Processing in Multidatabase Systems.", "b": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1." }
 { "a": "Specification and Execution of Transactional Workflows.", "b": "Specification and Execution of Transactional Workflows" }
 { "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
 { "a": "Integrated Office Systems.", "b": "Integrated Office Systems" }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
index 568272c..6171f93 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction  The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema.  The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer.  When a global query is submitted, it is decomposed into two types of queries [1]   -- subqueries, operating on sharable data items from local databases,  -- assembling queries, consisting of, CiteSeerX  2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.527027f }
 { "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 85, "csxid": "oai CiteSeerXPSU 10.1.1.37.8818", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-Molina Avi Silberschatz", "misc": "2009-06-22 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX  2009-06-22 2007-11-22 1992 text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.37.8818 ftp //ftp.cs.utexas.edu/pub/avi/UT-CS-TR-92-21.PS.Z en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.55932206f }
 { "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 86, "csxid": "oai CiteSeerXPSU 10.1.1.54.6302", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-molina Avi Silberschatz", "misc": "2009-04-12 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX  2009-04-12 2007-11-22 1992 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.54.6302 http //www-db.stanford.edu/pub/papers/multidatabase.ps en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.55932206f }
 { "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems  topics covered include active database rule models and languages, rule execution semantics, and implementation issues.  1 Introduction  Conventional database systems are passive  they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.95454544f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm
index f48c6c9..bb0d055 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/ngram-jaccard/ngram-jaccard.1.adm
@@ -1,6 +1,7 @@
 { "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 85, "csxid": "oai CiteSeerXPSU 10.1.1.37.8818", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-Molina Avi Silberschatz", "misc": "2009-06-22 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX  2009-06-22 2007-11-22 1992 text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.37.8818 ftp //ftp.cs.utexas.edu/pub/avi/UT-CS-TR-92-21.PS.Z en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 3, "dblpid": "books/acm/kim95/BreitbartGS95", "title": "Transaction Management in Multidatabase Systems.", "authors": "Yuri Breitbart Hector Garcia-Molina Abraham Silberschatz", "misc": "2004-03-08 573-591 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#BreitbartGS95 1995" }, "brec": { "id": 86, "csxid": "oai CiteSeerXPSU 10.1.1.54.6302", "title": "Overview of Multidatabase Transaction Management", "authors": "Yuri Breitbart Hector Garcia-molina Avi Silberschatz", "misc": "2009-04-12 A multidatabase system (MDBS) is a facility that allows users access to data located in multiple autonomous database management systems (DBMSs). In such a system, global transactions are executed under the control of the MDBS. Independently, local transactions are executed under the control of the local DBMSs. Each local DBMS integrated by the MDBS may employ a different transaction management scheme. In addition, each local DBMS has complete control over all transactions (global and local) executing at its site, including the ability to abort at any point any of the transactions executing at its site. Typically, no design or internal DBMS structure changes are allowed in order to accommodate the MDBS. Furthermore, the local DBMSs may not be aware of each other, and, as a consequence, cannot coordinate their actions. Thus, traditional techniques for ensuring transaction atomicity and consistency in homogeneous distributed database systems may not be appropriate for an MDBS environment.... CiteSeerX  2009-04-12 2007-11-22 1992 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.54.6302 http //www-db.stanford.edu/pub/papers/multidatabase.ps en 10.1.1.101.8988 10.1.1.130.1772 10.1.1.38.6210 10.1.1.34.3768 10.1.1.36.1275 10.1.1.104.3430 10.1.1.112.244 10.1.1.94.9106 10.1.1.41.4043 10.1.1.49.5143 10.1.1.59.2034 10.1.1.53.875 10.1.1.137.5642 10.1.1.41.8832 10.1.1.21.1100 10.1.1.105.3626 10.1.1.44.773 10.1.1.21.2576 10.1.1.40.6484 10.1.1.144.2713 10.1.1.48.6718 10.1.1.16.6166 10.1.1.40.832 10.1.1.36.2660 10.1.1.30.3087 10.1.1.47.322 10.1.1.17.6532 10.1.1.33.2301 10.1.1.20.4306 10.1.1.47.6258 10.1.1.39.9212 10.1.1.46.4334 10.1.1.71.485 10.1.1.43.1405 10.1.1.49.1308 10.1.1.35.6530 10.1.1.42.5177 10.1.1.54.4068 10.1.1.133.3692 10.1.1.40.4220 10.1.1.48.7743 10.1.1.26.575 10.1.1.107.596 10.1.1.116.3495 10.1.1.33.2074 10.1.1.38.7229 10.1.1.59.4464 10.1.1.103.9562 10.1.1.36.5887 10.1.1.40.9658 10.1.1.53.6783 10.1.1.29.5010 10.1.1.107.876 10.1.1.46.2273 10.1.1.46.3657 10.1.1.49.5281 10.1.1.50.4114 10.1.1.63.3234 10.1.1.79.9607 10.1.1.83.4819 10.1.1.83.4980 10.1.1.84.8136 10.1.1.90.953 10.1.1.90.9785 10.1.1.92.2397 10.1.1.93.8911 10.1.1.94.3702 10.1.1.97.672 10.1.1.98.4604 10.1.1.117.6190 10.1.1.118.4814 10.1.1.130.880 10.1.1.137.1167 10.1.1.51.5111 10.1.1.45.2774 10.1.1.45.9165 10.1.1.40.4684 10.1.1.35.5866 10.1.1.38.3606 10.1.1.29.9166 10.1.1.31.3667 10.1.1.21.7181 10.1.1.33.2343 10.1.1.23.3117 10.1.1.24.7879 10.1.1.18.8936 10.1.1.19.3770 10.1.1.19.5246 10.1.1.12.3293 10.1.1.2.2325 10.1.1.60.116 10.1.1.140.5244 10.1.1.143.3448 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems  topics covered include active database rule models and languages, rule execution semantics, and implementation issues.  1 Introduction  Conventional database systems are passive  they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction  The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema.  The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer.  When a global query is submitted, it is decomposed into two types of queries [1]   -- subqueries, operating on sharable data items from local databases,  -- assembling queries, consisting of, CiteSeerX  2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "brec": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures  and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application  of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of  different types) over different processing entities. Such applications are referred to as transactional  workflows. In this chapter we discuss the specification of such workflows and the issues involved in their  execution.  1 What is a Workflow?  Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems  it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments.  We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX  2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems  it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm
index 7e80ba2..2f6eb63 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard-inline/word-jaccard-inline.1.adm
@@ -1,3 +1,4 @@
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction  The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema.  The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer.  When a global query is submitted, it is decomposed into two types of queries [1]   -- subqueries, operating on sharable data items from local databases,  -- assembling queries, consisting of, CiteSeerX  2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 0.5f }
 { "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems  topics covered include active database rule models and languages, rule execution semantics, and implementation issues.  1 Introduction  Conventional database systems are passive  they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 1.0f }
 { "arec": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "brec": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures  and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application  of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of  different types) over different processing entities. Such applications are referred to as transactional  workflows. In this chapter we discuss the specification of such workflows and the issues involved in their  execution.  1 What is a Workflow?  Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 1.0f }
 { "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems  it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments.  We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX  2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." }, "jacc": 1.0f }
diff --git a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm
index 3550136..3ca1911 100644
--- a/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/inverted-index-join/word-jaccard/word-jaccard.1.adm
@@ -1,4 +1,5 @@
 { "arec": { "id": 5, "dblpid": "books/acm/kim95/DayalHW95", "title": "Active Database Systems.", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2002-01-03 434-456 1995 Modern Database Systems db/books/collections/kim95.html#DayalHW95" }, "brec": { "id": 98, "csxid": "oai CiteSeerXPSU 10.1.1.49.2910", "title": "Active Database Systems", "authors": "Umeshwar Dayal Eric N. Hanson Jennifer Widom", "misc": "2009-04-12 In Won Kim editor Modern Database Systems The Object Model Integrating a production rules facility into a database system provides a uniform mechanism for a number of advanced database features including integrity constraint enforcement, derived data maintenance, triggers, alerters, protection, version control, and others. In addition, a database system with rule processing capabilities provides a useful platform for large and efficient knowledge-base and expert systems. Database systems with production rules are referred to as active database systems, and the field of active database systems has indeed been active. This chapter summarizes current work in active database systems  topics covered include active database rule models and languages, rule execution semantics, and implementation issues.  1 Introduction  Conventional database systems are passive  they only execute queries or transactions explicitly submitted by a user or an application program. For many applications, however, it is important to monitor situations of interest, and to ... CiteSeerX ACM Press 2009-04-12 2007-11-22 1994 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.49.2910 http //www-db.stanford.edu/pub/papers/book-chapter.ps en 10.1.1.17.1323 10.1.1.143.7196 10.1.1.50.3821 10.1.1.51.9946 10.1.1.41.2030 10.1.1.46.2504 10.1.1.52.4421 10.1.1.38.2083 10.1.1.34.661 10.1.1.103.7630 10.1.1.100.9015 10.1.1.97.1699 10.1.1.107.4220 10.1.1.47.9217 10.1.1.133.7157 10.1.1.101.5051 10.1.1.30.9989 10.1.1.53.6941 10.1.1.50.8529 10.1.1.133.4287 10.1.1.50.7278 10.1.1.10.1688 10.1.1.19.8669 10.1.1.44.7600 10.1.1.144.376 10.1.1.44.1348 10.1.1.47.9998 10.1.1.90.4428 10.1.1.108.344 10.1.1.48.9470 10.1.1.53.5472 10.1.1.52.4872 10.1.1.144.4965 10.1.1.31.7578 10.1.1.32.6426 10.1.1.58.6335 10.1.1.85.8052 10.1.1.93.1931 10.1.1.55.4610 10.1.1.21.3821 10.1.1.26.9208 10.1.1.31.4869 10.1.1.48.1833 10.1.1.83.8628 10.1.1.87.9318 10.1.1.90.2195 10.1.1.36.5184 10.1.1.21.1704 10.1.1.53.1733 10.1.1.90.3181 10.1.1.53.6783 10.1.1.52.6151 10.1.1.104.6911 10.1.1.105.1691 10.1.1.21.1984 10.1.1.23.2775 10.1.1.62.5556 10.1.1.68.9063 10.1.1.74.4746 10.1.1.78.5097 10.1.1.84.743 10.1.1.84.904 10.1.1.87.6019 10.1.1.88.3907 10.1.1.89.9631 10.1.1.90.4147 10.1.1.92.365 10.1.1.100.2747 10.1.1.98.5083 10.1.1.98.6663 10.1.1.99.1894 10.1.1.99.8174 10.1.1.133.8073 10.1.1.52.7823 10.1.1.39.5341 10.1.1.35.3458 10.1.1.26.4620 10.1.1.18.8936 10.1.1.19.3694 10.1.1.12.631 10.1.1.48.6394 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
+{ "arec": { "id": 21, "dblpid": "books/acm/kim95/MengY95", "title": "Query Processing in Multidatabase Systems.", "authors": "Weiyi Meng Clement T. Yu", "misc": "2002-01-03 551-572 1995 Modern Database Systems db/books/collections/kim95.html#MengY95" }, "brec": { "id": 89, "csxid": "oai CiteSeerXPSU 10.1.1.33.8596", "title": "Dynamic Query Optimization and Query Processing in Multidatabase Systems 1.", "authors": "Henryk Josinski", "misc": "2009-04-15 Introduction  The multidatabase system (MDBS) approach, as a solution for integrated access to information distributed among diverse data sources, has gained a lot of attention in recent years. The multidatabase system is a database system which integrates pre--existing databases allowing the users to access simultaneously database systems (DBMSs) formulating a global query based on a global schema.  The component DBMSs are assumed to be heterogeneous and autonomous. Heterogeneity refers to different user interfaces, data models, query languages, and query optimization strategies [5]. Local autonomy means that each DBMS retains complete control over local data and processing. As result of this, its cost model may not be available to the global query optimizer.  When a global query is submitted, it is decomposed into two types of queries [1]   -- subqueries, operating on sharable data items from local databases,  -- assembling queries, consisting of, CiteSeerX  2009-04-15 2007-11-22 2000 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.33.8596 http //www.edbt2000.uni-konstanz.de/phd-workshop/papers/Josinski.pdf en 10.1.1.27.4704 10.1.1.51.8352 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 25, "dblpid": "books/acm/kim95/RusinkiewiczS95", "title": "Specification and Execution of Transactional Workflows.", "authors": "Marek Rusinkiewicz Amit P. Sheth", "misc": "2004-03-08 592-620 Modern Database Systems books/acm/Kim95 db/books/collections/kim95.html#RusinkiewiczS95 1995" }, "brec": { "id": 88, "csxid": "oai CiteSeerXPSU 10.1.1.43.3839", "title": "Specification and Execution of Transactional Workflows", "authors": "Marek Rusinkiewicz Amit Sheth", "misc": "2009-04-13 The basic transaction model has evolved over time to incorporate more complex transaction structures  and to selectively modify the atomicity and isolation properties. In this chapter we discuss the application  of transaction concepts to activities that involve coordinated execution of multiple tasks (possibly of  different types) over different processing entities. Such applications are referred to as transactional  workflows. In this chapter we discuss the specification of such workflows and the issues involved in their  execution.  1 What is a Workflow?  Workflows are activities involving the coordinated execution of multiple tasks performed by different processing entities. A task defines some work to be done and can be specified in a number of ways, including a textual description in a file or an email, a form, a message, or a computer program. A processing entity that performs the tasks may be a person or a software system (e.g., a mailer, an application program, a database mana... CiteSeerX ACM Press 2009-04-13 2007-11-22 1995 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.43.3839 http //lsdis.cs.uga.edu/lib/././download/RS93.ps en 10.1.1.17.1323 10.1.1.59.5051 10.1.1.38.6210 10.1.1.68.7445 10.1.1.109.5175 10.1.1.17.7962 10.1.1.44.7778 10.1.1.112.244 10.1.1.13.7602 10.1.1.102.7874 10.1.1.41.4043 10.1.1.49.5143 10.1.1.41.7252 10.1.1.17.3225 10.1.1.54.7761 10.1.1.55.5255 10.1.1.108.958 10.1.1.35.7733 10.1.1.52.3682 10.1.1.36.1618 10.1.1.45.6317 10.1.1.43.3180 10.1.1.35.8718 10.1.1.44.6365 10.1.1.51.2883 10.1.1.50.9206 10.1.1.6.9085 10.1.1.30.1707 10.1.1.80.6634 10.1.1.49.355 10.1.1.127.3550 10.1.1.35.3562 10.1.1.137.8832 10.1.1.49.4085 10.1.1.41.5506 10.1.1.40.4657 10.1.1.43.2369 10.1.1.40.832 10.1.1.74.5411 10.1.1.90.4428 10.1.1.110.6967 10.1.1.27.2122 10.1.1.15.5605 10.1.1.54.727 10.1.1.49.7512 10.1.1.45.8796 10.1.1.50.5984 10.1.1.53.137 10.1.1.30.3262 10.1.1.28.1680 10.1.1.21.7110 10.1.1.29.3148 10.1.1.57.687 10.1.1.59.5924 10.1.1.46.2812 10.1.1.51.5552 10.1.1.17.7375 10.1.1.40.1598 10.1.1.52.9787 10.1.1.1.3496 10.1.1.50.6791 10.1.1.55.3358 10.1.1.137.7582 10.1.1.118.4127 10.1.1.49.3580 10.1.1.35.5825 10.1.1.46.9382 10.1.1.31.7411 10.1.1.48.5504 10.1.1.55.5163 10.1.1.18.1603 10.1.1.52.8129 10.1.1.1.9723 10.1.1.21.9113 10.1.1.49.7644 10.1.1.52.6646 10.1.1.75.3106 10.1.1.80.2072 10.1.1.55.8770 10.1.1.54.8188 10.1.1.101.7919 10.1.1.104.8176 10.1.1.24.5741 10.1.1.29.4667 10.1.1.4.1055 10.1.1.48.9175 10.1.1.56.792 10.1.1.65.3172 10.1.1.66.5947 10.1.1.73.8532 10.1.1.83.8299 10.1.1.86.8521 10.1.1.87.2402 10.1.1.87.4648 10.1.1.90.5638 10.1.1.91.1709 10.1.1.94.4248 10.1.1.114.511 10.1.1.119.5037 10.1.1.124.7957 10.1.1.49.215 10.1.1.53.7777 10.1.1.53.9711 10.1.1.45.9409 10.1.1.40.8789 10.1.1.43.4845 10.1.1.34.8273 10.1.1.35.4783 10.1.1.28.3176 10.1.1.16.8151 10.1.1.8.9117 10.1.1.58.3449 10.1.1.142.7041 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 92, "csxid": "oai CiteSeerXPSU 10.1.1.13.2374", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-17 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of o#ce information systems  it is costly and di#cult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"o#ce objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to o#ce software. In order to fully exploit the approach to achieve integrated o#ce systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments.  We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt to enhance productivity through, f CiteSeerX  2009-04-17 2007-11-21 1988 application/pdf text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.13.2374 http //www.iam.unibe.ch/~scg/Archive/OSG/Nier89bIntegOfficeSystems.pdf en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
 { "arec": { "id": 51, "dblpid": "books/aw/kimL89/NierstraszT89", "title": "Integrated Office Systems.", "authors": "Oscar Nierstrasz Dennis Tsichritzis", "misc": "2002-01-03 199-215 1989 Object-Oriented Concepts, Databases, and Applications db/books/collections/kim89.html#NierstraszT89" }, "brec": { "id": 93, "csxid": "oai CiteSeerXPSU 10.1.1.42.9253", "title": "Integrated Office Systems", "authors": "O. M. Nierstrasz D. C. Tsichritzis", "misc": "2009-04-11 Introduction  New techniques are sorely needed to aid in the development and maintenance of large application systems. The problem with traditional approaches to software engineering is well in evidence in the field of office information systems  it is costly and difficult to extend existing applications, and to get unrelated applications to \"talk\" to each other. The objectoriented approach is already being tentatively applied in the modeling of \"office objects\" and in the presentation of these entities to users as such in \"desktop\" interfaces to office software. In order to fully exploit the approach to achieve integrated office systems, we need to use object-oriented programming languages, object-oriented run-time support, and object-oriented software engineering environments. We can view the fundamental idea behind the object-oriented approach as that of encapsulation  object-oriented languages and systems exploit encapsulation in various ways in an attempt t CiteSeerX ACM Press and Addison-Wesley 2009-04-11 2007-11-22 1988 application/postscript text http //citeseerx.ist.psu.edu/viewdoc/summary?doi=10.1.1.42.9253 ftp //ftp.iam.unibe.ch/pub/scg/Papers/integratedOfficeSystems.ps.gz en 10.1.1.26.9545 10.1.1.65.5865 10.1.1.34.624 10.1.1.12.8544 10.1.1.144.6983 10.1.1.26.6746 10.1.1.49.3064 10.1.1.30.4607 10.1.1.38.4894 10.1.1.20.8197 10.1.1.26.4381 10.1.1.29.1890 Metadata may be used without restrictions as long as the oai identifier remains attached to it." } }
diff --git a/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.adm b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.adm
new file mode 100644
index 0000000..f14f6a7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/similarity/similarity-jaccard-check_strings_issue628/similarity-jaccard-check_strings_issue628.1.adm
@@ -0,0 +1,3 @@
+[ true, 0.5f ]
+[ true, 0.5f ]
+[ false, 0.0f ]
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index d65da69..2d07061 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -3174,6 +3174,11 @@
         <output-dir compare="Text">similarity-jaccard_strings</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="similarity">
+      <compilation-unit name="similarity-jaccard-check_strings_issue628">
+        <output-dir compare="Text">similarity-jaccard-check_strings_issue628</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="spatial">
     <test-case FilePath="spatial">
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
index 2d97f0a..4233225 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/ExternalDetailsDecl.java
@@ -16,20 +16,10 @@
 
 import java.util.Map;
 
-import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
-
 public class ExternalDetailsDecl implements IDatasetDetailsDecl {
     private Map<String, String> properties;
     private String adapter;
-    private Identifier nodegroupName;
 
-    public ExternalDetailsDecl(Map<String, String> properties, String adapter, Identifier nodegroupName) {
-		this.properties = properties;
-		this.adapter = adapter;
-		this.nodegroupName = nodegroupName == null ? new Identifier(MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME)
-        : nodegroupName;
-	}
-    
     public void setAdapter(String adapter) {
         this.adapter = adapter;
     }
@@ -45,12 +35,4 @@
     public Map<String, String> getProperties() {
         return properties;
     }
-    
-    public void setNodegroupName(Identifier nodegroupName) {
-		this.nodegroupName = nodegroupName;
-	}
-    
-    public Identifier getNodegroupName() {
-		return nodegroupName;
-	}
 }
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 8dd7bfb..cb6336b 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -308,11 +308,11 @@
     <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
     ifNotExists = IfNotExists()
     "using" adapterName = AdapterName() properties = Configuration()
-    ("on" nodeGroupName = Identifier() )?
     ( "hints" hints = Properties() )?
       {
-        ExternalDetailsDecl edd = new ExternalDetailsDecl(properties,
-        adapterName,nodeGroupName != null? new Identifier(nodeGroupName): null);
+        ExternalDetailsDecl edd = new ExternalDetailsDecl();
+        edd.setAdapter(adapterName);
+        edd.setProperties(properties);
         dsetDecl = new DatasetDecl(nameComponents.first,
                                    nameComponents.second,
                                    new Identifier(typeName),
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
index 3610478..7e7ffd9 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/BaseOperationTracker.java
@@ -17,52 +17,18 @@
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
-import edu.uci.ics.hyracks.storage.am.lsm.common.impls.NoOpIOOperationCallback;
 
 public class BaseOperationTracker implements ILSMOperationTracker {
 
     protected final DatasetLifecycleManager datasetLifecycleManager;
-    protected final ILSMIOOperationCallback ioOpCallback;
-    protected long lastLSN;
-    protected long firstLSN;
     protected final int datasetID;
 
-    public BaseOperationTracker(DatasetLifecycleManager datasetLifecycleManager,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory, int datasetID) {
+    public BaseOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID) {
         this.datasetLifecycleManager = datasetLifecycleManager;
-        this.ioOpCallback = ioOpCallbackFactory == null ? NoOpIOOperationCallback.INSTANCE : ioOpCallbackFactory
-                .createIOOperationCallback(this);
         this.datasetID = datasetID;
-        resetLSNs();
-    }
-
-    public ILSMIOOperationCallback getIOOperationCallback() {
-        return ioOpCallback;
-    }
-
-    public long getLastLSN() {
-        return lastLSN;
-    }
-
-    public long getFirstLSN() {
-        return firstLSN;
-    }
-
-    public void updateLastLSN(long lastLSN) {
-        if (firstLSN == -1) {
-            firstLSN = lastLSN;
-        }
-        this.lastLSN = Math.max(this.lastLSN, lastLSN);
-    }
-
-    public void resetLSNs() {
-        lastLSN = -1;
-        firstLSN = -1;
     }
 
     @Override
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
index cf69bfe..dee652e 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/ConstantMergePolicy.java
@@ -43,7 +43,7 @@
         if (!ctx.isShuttingdown() && immutableComponents.size() >= threshold) {
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
-            accessor.scheduleMerge(NoOpIOOperationCallback.INSTANCE);
+            accessor.scheduleMerge(index.getIOOperationCallback());
         }
     }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
index 617b6ff..6335fb1 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/DatasetLifecycleManager.java
@@ -26,7 +26,6 @@
 
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
-import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
@@ -253,7 +252,7 @@
         if (iInfo.isOpen) {
             ILSMIndexAccessor accessor = (ILSMIndexAccessor) iInfo.index.createAccessor(NoOpOperationCallback.INSTANCE,
                     NoOpOperationCallback.INSTANCE);
-            accessor.scheduleFlush(((BaseOperationTracker) iInfo.index.getOperationTracker()).getIOOperationCallback());
+            accessor.scheduleFlush(iInfo.index.getIOOperationCallback());
         }
         // Wait for the above flush op.
         while (dsInfo.numActiveIOOps > 0) {
@@ -316,8 +315,7 @@
         synchronized (datasetOpTrackers) {
             ILSMOperationTracker opTracker = datasetOpTrackers.get(datasetID);
             if (opTracker == null) {
-                opTracker = new PrimaryIndexOperationTracker(this, datasetID,
-                        LSMBTreeIOOperationCallbackFactory.INSTANCE);
+                opTracker = new PrimaryIndexOperationTracker(this, datasetID);
                 datasetOpTrackers.put(datasetID, opTracker);
             }
 
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
index 2ed4b0ec..ef58937 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -18,11 +18,11 @@
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexInternal;
@@ -33,9 +33,8 @@
     // Number of active operations on an ILSMIndex instance.
     private final AtomicInteger numActiveOperations;
 
-    public PrimaryIndexOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID,
-            ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
-        super(datasetLifecycleManager, ioOpCallbackFactory, datasetID);
+    public PrimaryIndexOperationTracker(DatasetLifecycleManager datasetLifecycleManager, int datasetID) {
+        super(datasetLifecycleManager, datasetID);
         this.numActiveOperations = new AtomicInteger();
     }
 
@@ -43,7 +42,7 @@
     public void beforeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
-            numActiveOperations.incrementAndGet();
+            incrementNumActiveOperations(modificationCallback);
         } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
             datasetLifecycleManager.declareActiveIOOperation(datasetID);
         }
@@ -62,14 +61,11 @@
     public void completeOperation(ILSMIndex index, LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
-            numActiveOperations.decrementAndGet();
+            decrementNumActiveOperations(modificationCallback);
+            flushIfFull();
         } else if (opType == LSMOperationType.FLUSH || opType == LSMOperationType.MERGE) {
             datasetLifecycleManager.undeclareActiveIOOperation(datasetID);
         }
-
-        if (opType == LSMOperationType.MODIFICATION || opType == LSMOperationType.FORCE_MODIFICATION) {
-            flushIfFull();
-        }
     }
 
     private void flushIfFull() throws HyracksDataException {
@@ -88,8 +84,7 @@
                 for (ILSMIndex lsmIndex : indexes) {
                     ILSMIndexAccessor accessor = (ILSMIndexAccessor) lsmIndex.createAccessor(
                             NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-                    accessor.scheduleFlush(((BaseOperationTracker) lsmIndex.getOperationTracker())
-                            .getIOOperationCallback());
+                    accessor.scheduleFlush(lsmIndex.getIOOperationCallback());
                 }
             }
         }
@@ -103,4 +98,27 @@
     public int getNumActiveOperations() {
         return numActiveOperations.get();
     }
+
+    private void incrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+        //modificationCallback can be NoOpOperationCallback when redo/undo operations are executed. 
+        if (modificationCallback != NoOpOperationCallback.INSTANCE) {
+            numActiveOperations.incrementAndGet();
+            ((AbstractOperationCallback) modificationCallback).incrementLocalNumActiveOperations();
+        }
+    }
+
+    private void decrementNumActiveOperations(IModificationOperationCallback modificationCallback) {
+        //modificationCallback can be NoOpOperationCallback when redo/undo operations are executed.
+        if (modificationCallback != NoOpOperationCallback.INSTANCE) {
+            numActiveOperations.decrementAndGet();
+            ((AbstractOperationCallback) modificationCallback).decrementLocalNumActiveOperations();
+        }
+    }
+
+    public void cleanupNumActiveOperationsForAbortedJob(AbstractOperationCallback callback) {
+        int delta = callback.getLocalNumActiveOperations() * -1;
+        numActiveOperations.getAndAdd(delta);
+        callback.resetLocalNumActiveOperations();
+    }
+
 }
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
index da08cd8..684068b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -17,34 +17,51 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
 import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
 import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 
 public abstract class AbstractLSMIOOperationCallback implements ILSMIOOperationCallback {
 
-    protected final BaseOperationTracker opTracker;
+    protected long firstLSN;
+    protected long lastLSN;
+    protected long[] immutableLastLSNs;
+    protected int readIndex;
+    protected int writeIndex;
 
-    public AbstractLSMIOOperationCallback(BaseOperationTracker opTracker) {
-        this.opTracker = opTracker;
+    public AbstractLSMIOOperationCallback() {
+        resetLSNs();
     }
 
     @Override
-    public void beforeOperation() {
+    public void setNumOfMutableComponents(int count) {
+        immutableLastLSNs = new long[count];
+        readIndex = 0;
+        writeIndex = 0;
+    }
+
+    @Override
+    public void beforeOperation(LSMOperationType opType) {
+        if (opType == LSMOperationType.FLUSH) {
+            synchronized (this) {
+                immutableLastLSNs[writeIndex] = lastLSN;
+                writeIndex = (writeIndex + 1) % immutableLastLSNs.length;
+                resetLSNs();
+            }
+        }
+    }
+
+    @Override
+    public void afterFinalize(LSMOperationType opType, ILSMComponent newComponent) {
         // Do nothing.
     }
 
-    @Override
-    public void afterFinalize(ILSMComponent newComponent) {
-        opTracker.resetLSNs();
-    }
-
     public abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
 
     protected void putLSNIntoMetadata(ITreeIndex treeIndex, List<ILSMComponent> oldComponents)
@@ -80,4 +97,25 @@
             bufferCache.unpin(metadataPage);
         }
     }
+
+    protected void resetLSNs() {
+        firstLSN = -1;
+        lastLSN = -1;
+    }
+
+    public void updateLastLSN(long lastLSN) {
+        if (firstLSN == -1) {
+            firstLSN = lastLSN;
+        }
+        this.lastLSN = Math.max(this.lastLSN, lastLSN);
+    }
+
+    public long getFirstLSN() {
+        return firstLSN;
+    }
+
+    public long getLastLSN() {
+        return lastLSN;
+    }
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index b6025cb..8e9b44e 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -17,22 +17,22 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 
 public class LSMBTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
 
-    public LSMBTreeIOOperationCallback(BaseOperationTracker opTracker) {
-        super(opTracker);
+    public LSMBTreeIOOperationCallback() {
+        super();
     }
 
     @Override
-    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+    public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
-        if (oldComponents != null && newComponent != null) {
+        if (newComponent != null) {
             LSMBTreeDiskComponent btreeComponent = (LSMBTreeDiskComponent) newComponent;
             putLSNIntoMetadata(btreeComponent.getBTree(), oldComponents);
         }
@@ -42,7 +42,11 @@
     public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
         if (diskComponents == null) {
             // Implies a flush IO operation.
-            return opTracker.getLastLSN();
+            synchronized (this) {
+                long lsn = immutableLastLSNs[readIndex];
+                readIndex = (readIndex + 1) % immutableLastLSNs.length;
+                return lsn;
+            }
         }
         // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
index 92ba9ec..1028015 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallbackFactory.java
@@ -15,7 +15,6 @@
 
 package edu.uci.ics.asterix.common.ioopcallbacks;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 
@@ -29,7 +28,7 @@
     }
 
     @Override
-    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
-        return new LSMBTreeIOOperationCallback((BaseOperationTracker) syncObj);
+    public ILSMIOOperationCallback createIOOperationCallback() {
+        return new LSMBTreeIOOperationCallback();
     }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
index 4f99ae6..5532f97 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -17,21 +17,21 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponent;
 
 public class LSMInvertedIndexIOOperationCallback extends AbstractLSMIOOperationCallback {
 
-    public LSMInvertedIndexIOOperationCallback(BaseOperationTracker opTracker) {
-        super(opTracker);
+    public LSMInvertedIndexIOOperationCallback() {
+        super();
     }
 
     @Override
-    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+    public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
-        if (oldComponents != null && newComponent != null) {
+        if (newComponent != null) {
             LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) newComponent;
             putLSNIntoMetadata(invIndexComponent.getDeletedKeysBTree(), oldComponents);
         }
@@ -41,7 +41,11 @@
     public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
         if (diskComponents == null) {
             // Implies a flush IO operation.
-            return opTracker.getLastLSN();
+            synchronized (this) {
+                long lsn = immutableLastLSNs[readIndex];
+                readIndex = (readIndex + 1) % immutableLastLSNs.length;
+                return lsn;
+            }
         }
         // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
index c20cdb3..5dc0c0b 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallbackFactory.java
@@ -15,7 +15,6 @@
 
 package edu.uci.ics.asterix.common.ioopcallbacks;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 
@@ -29,7 +28,7 @@
     }
 
     @Override
-    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
-        return new LSMInvertedIndexIOOperationCallback((BaseOperationTracker) syncObj);
+    public ILSMIOOperationCallback createIOOperationCallback() {
+        return new LSMInvertedIndexIOOperationCallback();
     }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
index cd7b7a0..1497e17 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -17,21 +17,21 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponent;
 
 public class LSMRTreeIOOperationCallback extends AbstractLSMIOOperationCallback {
 
-    public LSMRTreeIOOperationCallback(BaseOperationTracker opTracker) {
-        super(opTracker);
+    public LSMRTreeIOOperationCallback() {
+        super();
     }
 
     @Override
-    public void afterOperation(List<ILSMComponent> oldComponents, ILSMComponent newComponent)
+    public void afterOperation(LSMOperationType opType, List<ILSMComponent> oldComponents, ILSMComponent newComponent)
             throws HyracksDataException {
-        if (oldComponents != null && newComponent != null) {
+        if (newComponent != null) {
             LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) newComponent;
             putLSNIntoMetadata(rtreeComponent.getRTree(), oldComponents);
             putLSNIntoMetadata(rtreeComponent.getBTree(), oldComponents);
@@ -42,7 +42,11 @@
     public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
         if (diskComponents == null) {
             // Implies a flush IO operation.
-            return opTracker.getLastLSN();
+            synchronized (this) {
+                long lsn = immutableLastLSNs[readIndex];
+                readIndex = (readIndex + 1) % immutableLastLSNs.length;
+                return lsn;
+            }
         }
         // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
index 0cd2539..841a1d5 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallbackFactory.java
@@ -15,7 +15,6 @@
 
 package edu.uci.ics.asterix.common.ioopcallbacks;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 
@@ -29,7 +28,7 @@
     }
 
     @Override
-    public ILSMIOOperationCallback createIOOperationCallback(Object syncObj) {
-        return new LSMRTreeIOOperationCallback((BaseOperationTracker) syncObj);
+    public ILSMIOOperationCallback createIOOperationCallback() {
+        return new LSMRTreeIOOperationCallback();
     }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
index d4b26f7..c549e7d 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/AbstractOperationCallback.java
@@ -15,6 +15,8 @@
 
 package edu.uci.ics.asterix.common.transactions;
 
+import java.util.concurrent.atomic.AtomicInteger;
+
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.bloomfilter.impls.MurmurHash128Bit;
 
@@ -27,6 +29,7 @@
     protected final ITransactionContext txnCtx;
     protected final ILockManager lockManager;
     protected final long[] longHashes;
+    protected final AtomicInteger transactorLocalNumActiveOperations;
 
     public AbstractOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
             ILockManager lockManager) {
@@ -34,6 +37,7 @@
         this.primaryKeyFields = primaryKeyFields;
         this.txnCtx = txnCtx;
         this.lockManager = lockManager;
+        this.transactorLocalNumActiveOperations = new AtomicInteger(0);
         this.longHashes = new long[2];
     }
 
@@ -41,4 +45,21 @@
         MurmurHash128Bit.hash3_x64_128(tuple, primaryKeyFields, SEED, longHashes);
         return Math.abs((int) longHashes[0]);
     }
+    
+    public void resetLocalNumActiveOperations() {
+        transactorLocalNumActiveOperations.set(0);
+    }
+
+    public int getLocalNumActiveOperations() {
+        return transactorLocalNumActiveOperations.get();
+    }
+
+    public void incrementLocalNumActiveOperations() {
+        transactorLocalNumActiveOperations.incrementAndGet();
+    }
+
+    public void decrementLocalNumActiveOperations() {
+        transactorLocalNumActiveOperations.decrementAndGet();
+    }
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
index 2f522b9..2638c9f 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
@@ -18,7 +18,6 @@
 
 import edu.uci.ics.hyracks.api.io.IIOManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
@@ -44,14 +43,6 @@
 
     public ILSMOperationTracker getLSMBTreeOperationTracker(int datasetID);
 
-    public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider(boolean isPrimary);
-
-    public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider();
-
-    public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider();
-
-    public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider();
-
     public ILSMIOOperationScheduler getLSMIOScheduler();
 
     public ILocalResourceRepository getLocalResourceRepository();
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 8913f8a..27a91a4 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -14,9 +14,11 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+
 public interface ILogManager {
 
-    public void log(ILogRecord logRecord);
+    public void log(ILogRecord logRecord) throws ACIDException;
 
     public ILogReader getLogReader(boolean isRecoveryMode);
 
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
index d13ef6c..3068867 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
@@ -20,15 +20,15 @@
 
 public interface ILogRecord {
 
-    public static final int JOB_COMMIT_LOG_SIZE = 13;
-    public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 29;
-    public static final int UPDATE_LOG_BASE_SIZE = 64;
+    public static final int JOB_TERMINATE_LOG_SIZE = 13; //JOB_COMMIT or ABORT log type
+    public static final int ENTITY_COMMIT_LOG_BASE_SIZE = 25;
+    public static final int UPDATE_LOG_BASE_SIZE = 60;
 
     public boolean readLogRecord(ByteBuffer buffer);
 
     public void writeLogRecord(ByteBuffer buffer);
 
-    public void formJobCommitLogRecord(ITransactionContext txnCtx);
+    public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit);
 
     public void formEntityCommitLogRecord(ITransactionContext txnCtx, int datasetId, int PKHashValue,
             ITupleReference tupleReference, int[] primaryKeyFields);
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
index 77e960b..ffd4cc2 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionManager.java
@@ -52,10 +52,11 @@
      * 
      * @param jobId
      *            a unique value for the transaction id.
+     * @param createIfNotExist TODO
      * @return
      * @throws ACIDException
      */
-    public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException;
+    public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException;
 
     /**
      * Commits a transaction.
diff --git a/asterix-external-data/pom.xml b/asterix-external-data/pom.xml
index ffc0eac..f8d5ea2 100644
--- a/asterix-external-data/pom.xml
+++ b/asterix-external-data/pom.xml
@@ -153,17 +153,6 @@
 			<artifactId>jdom</artifactId>
 			<version>1.0</version>
 		</dependency>
-		<dependency>
-			<groupId>javax.jdo</groupId>
-			<artifactId>jdo2-api</artifactId>
-			<version>2.3-20090302111651</version>
-		</dependency>
-		<dependency>
-			<groupId>org.apache.hive</groupId>
-			<artifactId>hive-exec</artifactId>
-			<version>0.11.0</version>
-		</dependency>
 	</dependencies>
 
 </project>
-
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
index e9d2175..4ca3d72 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HDFSAdapterFactory.java
@@ -20,21 +20,12 @@
 
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.conf.Configuration;
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSAccessByRIDAdapter;
+
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSIndexingAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
 import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -44,190 +35,76 @@
  */
 @SuppressWarnings("deprecation")
 public class HDFSAdapterFactory implements IGenericDatasetAdapterFactory {
-	private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-	public static final String HDFS_ADAPTER_NAME = "hdfs";
-	public static final String CLUSTER_LOCATIONS = "cluster-locations";
-	public static transient String SCHEDULER = "hdfs-scheduler";
+    public static final String HDFS_ADAPTER_NAME = "hdfs";
+    public static final String CLUSTER_LOCATIONS = "cluster-locations";
+    public static transient String SCHEDULER = "hdfs-scheduler";
 
-	public static final String KEY_HDFS_URL = "hdfs";
-	public static final String KEY_PATH = "path";
-	public static final String KEY_INPUT_FORMAT = "input-format";
-	public static final String INPUT_FORMAT_TEXT = "text-input-format";
-	public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
-	public static final String INPUT_FORMAT_RC = "rc-input-format";
-	public static final String KEY_DELIMITER = "delimiter";
-	public static final String KEY_FORMAT = "format";
-	public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+    public static final String KEY_HDFS_URL = "hdfs";
+    public static final String KEY_PATH = "path";
+    public static final String KEY_INPUT_FORMAT = "input-format";
+    public static final String INPUT_FORMAT_TEXT = "text-input-format";
+    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
 
-	private transient AlgebricksPartitionConstraint clusterLocations;
-	private String[] readSchedule;
-	private boolean executed[];
-	private InputSplitsFactory inputSplitsFactory;
-	private ConfFactory confFactory;
-	private boolean setup = false;
+    private transient AlgebricksPartitionConstraint clusterLocations;
+    private String[] readSchedule;
+    private boolean executed[];
+    private InputSplitsFactory inputSplitsFactory;
+    private ConfFactory confFactory;
+    private boolean setup = false;
 
-	private static final Map<String, String> formatClassNames = initInputFormatMap();
+    private static final Map<String, String> formatClassNames = initInputFormatMap();
 
-	private static Map<String, String> initInputFormatMap() {
-		Map<String, String> formatClassNames = new HashMap<String, String>();
-		formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
-		formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
-		formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
-		return formatClassNames;
-	}
+    private static Map<String, String> initInputFormatMap() {
+        Map<String, String> formatClassNames = new HashMap<String, String>();
+        formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
+        formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
+        return formatClassNames;
+    }
 
-	@Override
-	public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-		if (!setup) {
-			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-			configureJobConf(configuration);
-			JobConf conf = configureJobConf(configuration);
-			confFactory = new ConfFactory(conf);
+    @Override
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+        if (!setup) {
+            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+            configureJobConf(configuration);
+            JobConf conf = configureJobConf(configuration);
+            confFactory = new ConfFactory(conf);
 
-			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
 
-			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-			inputSplitsFactory = new InputSplitsFactory(inputSplits);
+            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+            inputSplitsFactory = new InputSplitsFactory(inputSplits);
 
-			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-			readSchedule = scheduler.getLocationConstraints(inputSplits);
-			executed = new boolean[readSchedule.length];
-			Arrays.fill(executed, false);
+            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+            readSchedule = scheduler.getLocationConstraints(inputSplits);
+            executed = new boolean[readSchedule.length];
+            Arrays.fill(executed, false);
 
-			setup = true;
-		}
-		JobConf conf = confFactory.getConf();
-		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-		HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+            setup = true;
+        }
+        JobConf conf = confFactory.getConf();
+        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+        HDFSAdapter hdfsAdapter = new HDFSAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+        hdfsAdapter.configure(configuration);
+        return hdfsAdapter;
+    }
 
-		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
-		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
-		{
-			char delimeter = 0x01;
-			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
-			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
-		}
+    @Override
+    public String getName() {
+        return HDFS_ADAPTER_NAME;
+    }
 
-		hdfsAdapter.configure(configuration);
-		return hdfsAdapter;
-	}
-
-	@Override
-	public IControlledAdapter createAccessByRIDAdapter(
-			Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
-		Configuration conf = configureHadoopConnection(configuration);
-		clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-		
-		//Create RID record desc
-		RecordDescriptor ridRecordDesc = null;
-
-		//If input format is rcfile, configure parser expected format to delimeted text with control char 0x01 as delimiter
-		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
-		{
-			char delimeter = 0x01;
-			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
-			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
-			ridRecordDesc = getRIDRecDesc(true, files != null);
-		}
-		else
-		{
-			ridRecordDesc = getRIDRecDesc(false, files != null);
-		}
-		HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf, files);
-		adapter.configure(configuration);
-		return adapter;
-	}
-
-	@Override
-	public IDatasourceAdapter createIndexingAdapter(Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
-		if (!setup) {
-			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-			configureJobConf(configuration);
-			JobConf conf = configureJobConf(configuration);
-			confFactory = new ConfFactory(conf);
-
-			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
-			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-			inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
-			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-			readSchedule = scheduler.getLocationConstraints(inputSplits);
-			executed = new boolean[readSchedule.length];
-			Arrays.fill(executed, false);
-
-			setup = true;
-		}
-		JobConf conf = confFactory.getConf();
-		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default) as delimiter
-		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
-		{
-			char delimeter = 0x01;
-			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
-			configuration.put(KEY_DELIMITER, Character.toString(delimeter));	
-		}
-		HDFSIndexingAdapter hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
-		hdfsIndexingAdapter.configure(configuration);
-		return hdfsIndexingAdapter;
-	}
-
-	@Override
-	public String getName() {
-		return HDFS_ADAPTER_NAME;
-	}
-
-	private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
-		JobConf conf = new JobConf();
-		conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
-		conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
-		conf.setClassLoader(HDFSAdapter.class.getClassLoader());
-		conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
-		conf.set("mapred.input.format.class",
-				(String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
-		return conf;
-	}
-
-	public static Configuration configureHadoopConnection(Map<String, Object> configuration)
-	{
-		Configuration conf = new Configuration();
-		conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
-		conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
-		return conf;
-	}
-	
-	public static RecordDescriptor getRIDRecDesc(boolean isRCFile, boolean optimize){
-		int numOfPrimaryKeys = 2;
-		if(isRCFile)
-		{
-			numOfPrimaryKeys++;
-		}
-		@SuppressWarnings("rawtypes")
-		ISerializerDeserializer[] serde = new ISerializerDeserializer[numOfPrimaryKeys];
-		ITypeTraits[] tt = new ITypeTraits[numOfPrimaryKeys];
-		if(optimize)
-		{
-			serde[0] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-			tt[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT32);
-		}
-		else
-		{
-			serde[0] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
-			tt[0] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.ASTRING);
-		}
-		serde[1] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-		tt[1] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT64);
-		if(isRCFile)
-		{
-			//we add the row number for rc-files
-			serde[2] = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-			tt[2] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(BuiltinType.AINT32);
-		}
-		return new RecordDescriptor(serde, tt);
-	}
-
+    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+        JobConf conf = new JobConf();
+        conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
+        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
+        conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
+        conf.set("mapred.input.format.class",
+                (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
+        return conf;
+    }
 
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
index 64c8153..409eb7a 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/HiveAdapterFactory.java
@@ -18,20 +18,15 @@
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSAccessByRIDAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.HiveIndexingAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.hdfs.dataflow.ConfFactory;
 import edu.uci.ics.hyracks.hdfs.dataflow.InputSplitsFactory;
 import edu.uci.ics.hyracks.hdfs.scheduler.Scheduler;
@@ -41,185 +36,109 @@
  */
 @SuppressWarnings("deprecation")
 public class HiveAdapterFactory implements IGenericDatasetAdapterFactory {
-	private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-	public static final String HDFS_ADAPTER_NAME = "hdfs";
-	public static final String CLUSTER_LOCATIONS = "cluster-locations";
-	public static transient String SCHEDULER = "hdfs-scheduler";
+    public static final String HDFS_ADAPTER_NAME = "hdfs";
+    public static final String CLUSTER_LOCATIONS = "cluster-locations";
+    public static transient String SCHEDULER = "hdfs-scheduler";
 
-	public static final String KEY_HDFS_URL = "hdfs";
-	public static final String KEY_PATH = "path";
-	public static final String KEY_INPUT_FORMAT = "input-format";
-	public static final String INPUT_FORMAT_TEXT = "text-input-format";
-	public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
-	public static final String INPUT_FORMAT_RC = "rc-input-format";
+    public static final String KEY_HDFS_URL = "hdfs";
+    public static final String KEY_PATH = "path";
+    public static final String KEY_INPUT_FORMAT = "input-format";
+    public static final String INPUT_FORMAT_TEXT = "text-input-format";
+    public static final String INPUT_FORMAT_SEQUENCE = "sequence-input-format";
 
-	public static final String KEY_FORMAT = "format";
-	public static final String KEY_PARSER_FACTORY = "parser";
-	public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
-	public static final String FORMAT_ADM = "adm";
-	public static final String KEY_DELIMITER = "delimiter";
+    public static final String KEY_FORMAT = "format";
+    public static final String KEY_PARSER_FACTORY = "parser";
+    public static final String FORMAT_DELIMITED_TEXT = "delimited-text";
+    public static final String FORMAT_ADM = "adm";
 
-	public static final String HIVE_DATABASE = "database";
-	public static final String HIVE_TABLE = "table";
-	public static final String HIVE_HOME = "hive-home";
-	public static final String HIVE_METASTORE_URI = "metastore-uri";
-	public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
-	public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
+    public static final String HIVE_DATABASE = "database";
+    public static final String HIVE_TABLE = "table";
+    public static final String HIVE_HOME = "hive-home";
+    public static final String HIVE_METASTORE_URI = "metastore-uri";
+    public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
+    public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
 
-	private String[] readSchedule;
-	private boolean executed[];
-	private InputSplitsFactory inputSplitsFactory;
-	private ConfFactory confFactory;
-	private transient AlgebricksPartitionConstraint clusterLocations;
-	private boolean setup = false;
+    private String[] readSchedule;
+    private boolean executed[];
+    private InputSplitsFactory inputSplitsFactory;
+    private ConfFactory confFactory;
+    private transient AlgebricksPartitionConstraint clusterLocations;
+    private boolean setup = false;
 
-	private static final Map<String, String> formatClassNames = initInputFormatMap();
+    private static final Map<String, String> formatClassNames = initInputFormatMap();
 
-	private static Map<String, String> initInputFormatMap() {
-		Map<String, String> formatClassNames = new HashMap<String, String>();
-		formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
-		formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
-		formatClassNames.put(INPUT_FORMAT_RC, "org.apache.hadoop.hive.ql.io.RCFileInputFormat");
-		return formatClassNames;
-	}
+    private static Map<String, String> initInputFormatMap() {
+        Map<String, String> formatClassNames = new HashMap<String, String>();
+        formatClassNames.put(INPUT_FORMAT_TEXT, "org.apache.hadoop.mapred.TextInputFormat");
+        formatClassNames.put(INPUT_FORMAT_SEQUENCE, "org.apache.hadoop.mapred.SequenceFileInputFormat");
+        return formatClassNames;
+    }
 
+    @Override
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+        if (!setup) {
+            /** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
+            configureJobConf(configuration);
+            JobConf conf = configureJobConf(configuration);
+            confFactory = new ConfFactory(conf);
 
-	@Override
-	public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
-		Configuration conf = HDFSAdapterFactory.configureHadoopConnection(configuration);
-		clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-		//Create RID record desc
-		RecordDescriptor ridRecordDesc = null;
+            clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
+            int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
 
-		//If input format is rcfile, configure parser expected format to delimeted text with control char 0x01 as delimiter
-		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
-		{
-			char delimeter = 0x01;
-			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
-			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
-			ridRecordDesc = HDFSAdapterFactory.getRIDRecDesc(true, files != null);
-		}
-		else
-		{
-			ridRecordDesc = HDFSAdapterFactory.getRIDRecDesc(false, files != null);
-		}
-		HDFSAccessByRIDAdapter adapter = new HDFSAccessByRIDAdapter(atype, ((String)configuration.get(KEY_INPUT_FORMAT)), clusterLocations,ridRecordDesc, conf, files);
-		adapter.configure(configuration);
-		return adapter;
-	}
+            InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
+            inputSplitsFactory = new InputSplitsFactory(inputSplits);
 
-	@Override
-	public IDatasourceAdapter createIndexingAdapter(
-			Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
-		if (!setup) {
-			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-			configureJobConf(configuration);
-			JobConf conf = configureJobConf(configuration);
-			confFactory = new ConfFactory(conf);
+            Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
+            readSchedule = scheduler.getLocationConstraints(inputSplits);
+            executed = new boolean[readSchedule.length];
+            Arrays.fill(executed, false);
 
-			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
+            setup = true;
+        }
+        JobConf conf = confFactory.getConf();
+        InputSplit[] inputSplits = inputSplitsFactory.getSplits();
+        HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
+        hiveAdapter.configure(configuration);
+        return hiveAdapter;
+    }
 
-			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-			inputSplitsFactory = new InputSplitsFactory(inputSplits);
+    @Override
+    public String getName() {
+        return "hive";
+    }
 
-			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-			readSchedule = scheduler.getLocationConstraints(inputSplits);
-			executed = new boolean[readSchedule.length];
-			Arrays.fill(executed, false);
+    private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
+        JobConf conf = new JobConf();
 
-			setup = true;
-		}
-		JobConf conf = confFactory.getConf();
-		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-		HiveIndexingAdapter hiveIndexingAdapter = new HiveIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
+        /** configure hive */
+        String database = (String) configuration.get(HIVE_DATABASE);
+        String tablePath = null;
+        if (database == null) {
+            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
+        } else {
+            tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
+                    + configuration.get(HIVE_TABLE);
+        }
+        configuration.put(HDFSAdapter.KEY_PATH, tablePath);
+        if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
+            throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
+        }
 
-		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
-		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
-		{
-			char delimeter = 0x01;
-			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
-			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
-		}
+        if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
+                .get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))) {
+            throw new IllegalArgumentException("file input format"
+                    + configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
+        }
 
-		hiveIndexingAdapter.configure(configuration);
-		return hiveIndexingAdapter;
-	}
-
-	@Override
-	public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-		if (!setup) {
-			/** set up the factory --serializable stuff --- this if-block should be called only once for each factory instance */
-			configureJobConf(configuration);
-			JobConf conf = configureJobConf(configuration);
-			confFactory = new ConfFactory(conf);
-
-			clusterLocations = (AlgebricksPartitionConstraint) configuration.get(CLUSTER_LOCATIONS);
-			int numPartitions = ((AlgebricksAbsolutePartitionConstraint) clusterLocations).getLocations().length;
-
-			InputSplit[] inputSplits = conf.getInputFormat().getSplits(conf, numPartitions);
-			inputSplitsFactory = new InputSplitsFactory(inputSplits);
-
-			Scheduler scheduler = (Scheduler) configuration.get(SCHEDULER);
-			readSchedule = scheduler.getLocationConstraints(inputSplits);
-			executed = new boolean[readSchedule.length];
-			Arrays.fill(executed, false);
-
-			setup = true;
-		}
-		JobConf conf = confFactory.getConf();
-		InputSplit[] inputSplits = inputSplitsFactory.getSplits();
-		HiveAdapter hiveAdapter = new HiveAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations);
-
-		//If input format is rcfile, configure parser expected format to delimeted text with 0x01 (default ) as delimiter
-		if(((String)configuration.get(KEY_INPUT_FORMAT)).equals(INPUT_FORMAT_RC))
-		{
-			char delimeter = 0x01;
-			configuration.put(KEY_FORMAT, FORMAT_DELIMITED_TEXT);
-			configuration.put(KEY_DELIMITER, Character.toString(delimeter));
-		}
-
-		hiveAdapter.configure(configuration);
-		return hiveAdapter;
-	}
-
-	@Override
-	public String getName() {
-		return "hive";
-	}
-
-	private JobConf configureJobConf(Map<String, Object> configuration) throws Exception {
-		JobConf conf = new JobConf();
-
-		/** configure hive */
-		String database = (String) configuration.get(HIVE_DATABASE);
-		String tablePath = null;
-		if (database == null) {
-			tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + configuration.get(HIVE_TABLE);
-		} else {
-			tablePath = configuration.get(HIVE_WAREHOUSE_DIR) + "/" + tablePath + ".db" + "/"
-					+ configuration.get(HIVE_TABLE);
-		}
-		configuration.put(HDFSAdapter.KEY_PATH, tablePath);
-		if (!configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT)) {
-			throw new IllegalArgumentException("format" + configuration.get(KEY_FORMAT) + " is not supported");
-		}
-
-		if (!(configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT) || configuration
-				.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE) || configuration
-				.get(HDFSAdapterFactory.KEY_INPUT_FORMAT).equals(HDFSAdapterFactory.INPUT_FORMAT_RC))) {
-			throw new IllegalArgumentException("file input format"
-					+ configuration.get(HDFSAdapterFactory.KEY_INPUT_FORMAT) + " is not supported");
-		}
-
-		/** configure hdfs */
-		conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
-		conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
-		conf.setClassLoader(HDFSAdapter.class.getClassLoader());
-		conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
-		conf.set("mapred.input.format.class",
-				(String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
-		return conf;
-	}
-}
\ No newline at end of file
+        /** configure hdfs */
+        conf.set("fs.default.name", ((String) configuration.get(KEY_HDFS_URL)).trim());
+        conf.set("fs.hdfs.impl", "org.apache.hadoop.hdfs.DistributedFileSystem");
+        conf.setClassLoader(HDFSAdapter.class.getClassLoader());
+        conf.set("mapred.input.dir", ((String) configuration.get(KEY_PATH)).trim());
+        conf.set("mapred.input.format.class",
+                (String) formatClassNames.get(((String) configuration.get(KEY_INPUT_FORMAT)).trim()));
+        return conf;
+    }
+}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
index f046f88..0a178a7 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/IGenericDatasetAdapterFactory.java
@@ -14,10 +14,8 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
-import java.util.HashMap;
 import java.util.Map;
 
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
 
@@ -42,29 +40,4 @@
      */
     public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception;
 
-    /**
-     * Creates an instance of IDatasourceAdapter that is used to read records and their RIDs.
-     * 
-     * @param configuration
-     *            The configuration parameters for the adapter that is instantiated.
-     *            The passed-in configuration is used to configure the created instance of the adapter.
-     * @param atype
-     *            The type for the ADM records that are returned by the adapter (contains both original fields and RID fields).
-     * @return An instance of IDatasourceAdapter.
-     * @throws Exception
-     */
-    public IDatasourceAdapter createIndexingAdapter(Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception;
-
-    /**
-     * Creates an instance of IDatasourceAdapter that is used to read records using their RIDs.
-     * 
-     * @param configuration
-     *            The configuration parameters for the adapter that is instantiated.
-     *            The passed-in configuration is used to configure the created instance of the adapter.
-     * @param atype
-     *            The type for the ADM records that are returned by the adapter.
-     * @return An instance of IControlledAdapter.
-     * @throws Exception
-     */
-    public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception;
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
index 4fae7e7..e680232 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/adapter/factory/NCFileSystemAdapterFactory.java
@@ -14,14 +14,11 @@
  */
 package edu.uci.ics.asterix.external.adapter.factory;
 
-import java.util.HashMap;
 import java.util.Map;
 
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 
 /**
  * Factory class for creating an instance of NCFileSystemAdapter. An
@@ -29,31 +26,18 @@
  * an NC.
  */
 public class NCFileSystemAdapterFactory implements IGenericDatasetAdapterFactory {
-	private static final long serialVersionUID = 1L;
-	public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
+    private static final long serialVersionUID = 1L;
+    public static final String NC_FILE_SYSTEM_ADAPTER_NAME = "localfs";
 
-	@Override
-	public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
-		NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
-		fsAdapter.configure(configuration);
-		return fsAdapter;
-	}
+    @Override
+    public IDatasourceAdapter createAdapter(Map<String, Object> configuration, IAType atype) throws Exception {
+        NCFileSystemAdapter fsAdapter = new NCFileSystemAdapter(atype);
+        fsAdapter.configure(configuration);
+        return fsAdapter;
+    }
 
-	@Override
-	public String getName() {
-		return NC_FILE_SYSTEM_ADAPTER_NAME;
-	}
-
-	@Override
-	public IDatasourceAdapter createIndexingAdapter(
-			Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
-		throw new NotImplementedException("Indexing Adapter is not implemented for NC FileSystem Data");
-	}
-
-
-
-	@Override
-	public IControlledAdapter createAccessByRIDAdapter(Map<String, Object> configuration, IAType atype, HashMap<Integer, String> files) throws Exception {
-		throw new NotImplementedException("Access by RID Adapter is not implemented for NC FileSystem Data");
-	}
+    @Override
+    public String getName() {
+        return NC_FILE_SYSTEM_ADAPTER_NAME;
+    }
 }
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
deleted file mode 100644
index aa91a56..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataAccessByRIDOperatorDescriptor.java
+++ /dev/null
@@ -1,78 +0,0 @@
-package edu.uci.ics.asterix.external.data.operator;
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.IOperatorDescriptorRegistry;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
-
-public class ExternalDataAccessByRIDOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor {
-
-	/**
-	 * This operator is used to access external data residing in hdfs using record ids pushed in frame buffers
-	 */
-	private static final long serialVersionUID = 1L;
-	private final Map<String, Object> adapterConfiguration;
-	private final IAType atype;
-	private IGenericDatasetAdapterFactory datasourceAdapterFactory;
-	private IControlledAdapter adapter;
-	private final HashMap<Integer, String> files;
-	
-	public ExternalDataAccessByRIDOperatorDescriptor(
-			IOperatorDescriptorRegistry spec, Map<String, Object> arguments, IAType atype,
-			RecordDescriptor outRecDesc,IGenericDatasetAdapterFactory dataSourceAdapterFactory, HashMap<Integer, String> files) {
-		super(spec, 1, 1);
-		this.atype = atype;
-		this.adapterConfiguration = arguments;
-		this.datasourceAdapterFactory = dataSourceAdapterFactory;
-		this.recordDescriptors[0] = outRecDesc;
-		this.files = files;
-	}
-
-	@Override
-	public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-			IRecordDescriptorProvider recordDescProvider, int partition,
-			int nPartitions) throws HyracksDataException {
-		return new AbstractUnaryInputUnaryOutputOperatorNodePushable() {
-			@Override
-			public void open() throws HyracksDataException {
-				//create the access by index adapter
-				try {
-					adapter = datasourceAdapterFactory.createAccessByRIDAdapter(adapterConfiguration, atype, files);
-					adapter.initialize(ctx);
-				} catch (Exception e) {
-					throw new HyracksDataException("error during creation of external read by RID adapter", e);
-				}
-				writer.open();
-			}
-
-			@Override
-			public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
-				adapter.processNextFrame(buffer, writer);
-			}
-
-			@Override
-			public void close() throws HyracksDataException {
-				//close adapter and flush remaining frame if needed
-				adapter.close(writer);
-				//close writer
-				writer.close();
-			}
-
-			@Override
-			public void fail() throws HyracksDataException {
-				writer.fail();
-			}
-		};	
-	}
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
deleted file mode 100644
index 9ff1f06..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/data/operator/ExternalDataIndexingOperatorDescriptor.java
+++ /dev/null
@@ -1,64 +0,0 @@
-package edu.uci.ics.asterix.external.data.operator;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.IOperatorNodePushable;
-import edu.uci.ics.hyracks.api.dataflow.value.IRecordDescriptorProvider;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.api.job.JobSpecification;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractSingleActivityOperatorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable;
-
-/*
- * A single activity operator that provides the functionality of scanning data along 
- * with their RIDs using an instance of the configured adapter.
- */
-
-public class ExternalDataIndexingOperatorDescriptor extends AbstractSingleActivityOperatorDescriptor{
-
-	private static final long serialVersionUID = 1L;
-
-	private final Map<String, Object> adapterConfiguration;
-	private final Map<String,Integer> files;
-	private final IAType atype;
-	private IGenericDatasetAdapterFactory datasourceAdapterFactory;
-
-	public ExternalDataIndexingOperatorDescriptor(JobSpecification spec, Map<String, Object> arguments, IAType atype,
-			RecordDescriptor rDesc, IGenericDatasetAdapterFactory dataSourceAdapterFactory, Map<String,Integer> files) {
-		super(spec, 0, 1);
-		recordDescriptors[0] = rDesc;
-		this.adapterConfiguration = arguments;
-		this.atype = atype;
-		this.datasourceAdapterFactory = dataSourceAdapterFactory;
-		this.files = files;
-	}
-
-	@Override
-	public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
-			IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
-					throws HyracksDataException {
-
-		return new AbstractUnaryOutputSourceOperatorNodePushable() {
-			@Override
-			public void initialize() throws HyracksDataException {
-				writer.open();
-				IDatasourceAdapter adapter = null;
-				try {
-					adapter = ((IGenericDatasetAdapterFactory) datasourceAdapterFactory).createIndexingAdapter(
-							adapterConfiguration, atype, files);
-					adapter.initialize(ctx);
-					adapter.start(partition, writer);
-				} catch (Exception e) {
-					throw new HyracksDataException("exception during reading from external data source", e);
-				} finally {
-					writer.close();
-				}
-			}
-		};
-	}
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
deleted file mode 100644
index 86a060c..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAccessByRIDAdapter.java
+++ /dev/null
@@ -1,1170 +0,0 @@
-/*
- * Copyright 2009-2013 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.SequenceFile;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.ql.io.RCFile;
-import org.apache.hadoop.hive.ql.io.RCFile.Reader;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.om.base.AInt32;
-import edu.uci.ics.asterix.om.base.AInt64;
-import edu.uci.ics.asterix.om.base.AString;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.runtime.operators.file.ControlledADMTupleParserFactory;
-import edu.uci.ics.asterix.runtime.operators.file.ControlledDelimitedDataTupleParserFactory;
-import edu.uci.ics.asterix.runtime.operators.file.ControlledTupleParser;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * Provides functionality for fetching specific external data records stored in an HDFS instance
- * using their RID.
- */
-@SuppressWarnings({ "deprecation" })
-public class HDFSAccessByRIDAdapter extends FileSystemBasedAdapter implements IControlledAdapter{
-
-	private static final long serialVersionUID = 1L;
-	private boolean newFrame;
-	private transient ByteBuffer frameBuffer;
-	private String inputFormat;
-	private Configuration conf;
-	private transient FileSystem fs;
-	private RecordDescriptor inRecDesc;
-	private final HashMap<Integer, String> files;
-
-	public HDFSAccessByRIDAdapter(IAType atype, String inputFormat, AlgebricksPartitionConstraint clusterLocations, RecordDescriptor inRecDesc, Configuration conf, HashMap<Integer,String> files) {
-		super(atype);
-		this.inputFormat = inputFormat;
-		this.conf = conf;
-		this.inRecDesc = inRecDesc;
-		this.files = files;
-	}
-
-	@Override
-	public void configure(Map<String, Object> arguments) throws Exception {
-		this.configuration = arguments;
-		fs = FileSystem.get(conf);
-		String specifiedFormat = (String) configuration.get(KEY_FORMAT);
-		if (specifiedFormat == null) {
-			throw new IllegalArgumentException(" Unspecified data format");
-		} else if (FORMAT_DELIMITED_TEXT.equalsIgnoreCase(specifiedFormat)) {
-			parserFactory = getDelimitedDataTupleParserFactory((ARecordType) atype);
-		} else if (FORMAT_ADM.equalsIgnoreCase((String)configuration.get(KEY_FORMAT))) {
-			parserFactory = new ControlledADMTupleParserFactory((ARecordType) atype);
-		} else {
-			throw new IllegalArgumentException(" format " + configuration.get(KEY_FORMAT) + " not supported");
-		}
-	}
-
-	@Override
-	protected ITupleParserFactory getDelimitedDataTupleParserFactory(ARecordType recordType) throws AsterixException {
-		int n = recordType.getFieldTypes().length;
-		IValueParserFactory[] fieldParserFactories = new IValueParserFactory[n];
-		for (int i = 0; i < n; i++) {
-			ATypeTag tag = recordType.getFieldTypes()[i].getTypeTag();
-			IValueParserFactory vpf = typeToValueParserFactMap.get(tag);
-			if (vpf == null) {
-				throw new NotImplementedException("No value parser factory for delimited fields of type " + tag);
-			}
-			fieldParserFactories[i] = vpf;
-		}
-		String delimiterValue = (String) configuration.get(KEY_DELIMITER);
-		if (delimiterValue != null && delimiterValue.length() > 1) {
-			throw new AsterixException("improper delimiter");
-		}
-
-		Character delimiter = delimiterValue.charAt(0);
-		return new ControlledDelimitedDataTupleParserFactory(recordType, fieldParserFactories, delimiter);
-	}
-
-	@Override
-	public void start(int partition, IFrameWriter writer) throws Exception {
-		throw new NotImplementedException("Access by RID adapter doesn't support start function");
-	}
-
-	public void processNextFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException
-	{
-		frameBuffer = buffer;
-		newFrame = true;
-		((ControlledTupleParser)parser).parseNext(writer);
-	}
-
-	public void close(IFrameWriter writer) throws HyracksDataException
-	{
-		((ControlledTupleParser)parser).close(writer);
-	}
-
-	public AdapterType getAdapterType() {
-		return AdapterType.READ;
-	}
-
-	@Override
-	public void initialize(IHyracksTaskContext ctx) throws Exception {
-		this.ctx = ctx;
-		//create parser and initialize it with an instance of the inputStream
-		parser = parserFactory.createTupleParser(ctx);
-		((ControlledTupleParser)parser).initialize(getInputStream(0));
-	}
-
-	@Override
-	public InputStream getInputStream(int partition) throws IOException {
-
-		//if files map is not null, then it is optimized and we should return optimized inputStream, else return regular
-		if(files == null)
-		{	
-
-			//different input stream implementation based on the input format
-			if(inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
-			{
-				return new InputStream() {
-					private RCFile.Reader reader;
-					private int rowDifference;
-					private String lastFileName = "";
-					private String newFileName;
-					private long lastByteLocation = 0;
-					private long newByteLocation = 0;
-					private int lastRowNumber = 0;
-					private int newRowNumber = 0;
-					private LongWritable key;
-					private BytesRefArrayWritable value;
-					private int EOL = "\n".getBytes()[0];
-					private byte delimiter = 0x01;
-					private boolean pendingValue = false;
-					private int currentTupleIdx;
-					private int numberOfTuplesInCurrentFrame;
-					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
-					private ByteBufferInputStream bbis = new ByteBufferInputStream();
-					private DataInputStream dis = new DataInputStream(bbis);
-
-					@Override
-					public void close()
-					{
-						if (reader != null)
-						{
-							reader.close();
-						}
-						try {
-							super.close();
-						} catch (IOException e) {
-							e.printStackTrace();
-						}
-					}
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-						if(newFrame)
-						{
-							//first time called with this frame
-							//reset frame buffer
-							tupleAccessor.reset(frameBuffer);
-							//get number of tuples in frame
-							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
-							//set tuple index to first tuple
-							currentTupleIdx = 0;
-							//set new frame to false
-							newFrame = false;
-							pendingValue = false;
-						}
-
-						//check and see if there is a pending value
-						//Double check this
-						int numBytes = 0;
-						if (pendingValue) {
-							//last value didn't fit into buffer
-							int sizeOfNextTuple = getTupleSize(value) + 1;
-							if(sizeOfNextTuple > len)
-							{
-								return 0;
-							}
-							copyCurrentTuple(buffer, offset + numBytes);
-							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = false;
-							//move to next tuple
-							currentTupleIdx++;
-						}
-
-						//No pending value or done with pending value
-						//check if there are more tuples in the frame
-						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
-						{
-							//get 3 things from the current tuple in the frame(File name, byte location and row number)
-							//get the fileName
-							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
-							newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
-							//check if it is a new file
-							if(!lastFileName.equals(newFileName))//stringBuilder.toString()))
-							{
-								//new file
-								lastFileName = newFileName;
-								//close old file
-								if(reader != null)
-								{
-									reader.close();
-								}
-								//open new file
-								reader = new Reader(fs, new Path(lastFileName), conf);
-								//read and save byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								lastByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(lastByteLocation);
-								//read and save rowNumber
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
-								lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-								//loop until row
-								for(int i=0; i < lastRowNumber; i++)
-								{
-									//this loop perform a single I/O and move to the next record in the block which is already in memory
-									//if no more records in the current block, it perform another I/O and get the next block
-									//<this should never happen here>
-									reader.next(key);
-								}
-								//read record
-								reader.getCurrentRow(value);
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = getTupleSize(value) + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = true;
-									return numBytes;
-								}
-								copyCurrentTuple(buffer, offset + numBytes);
-								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							else
-							{
-								//same file
-								//get the byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								newByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-
-								//check if same block
-								if(lastByteLocation != newByteLocation)
-								{
-									//new block
-									lastByteLocation = newByteLocation;
-									//seek
-									reader.seek(lastByteLocation);
-									//read and save rowNumber
-									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
-									lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-									//loop until row
-									for(int i=0; i < lastRowNumber; i++)
-									{
-										reader.next(key);
-									}
-									//read record
-									reader.getCurrentRow(value);
-									//copy it to the buffer if there is enough space
-									int sizeOfNextTuple = getTupleSize(value) + 1;
-									if(sizeOfNextTuple + numBytes > len)
-									{
-										//mark waiting value
-										pendingValue = true;
-										return numBytes;
-									}
-									copyCurrentTuple(buffer, offset + numBytes);
-									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-									numBytes += sizeOfNextTuple;
-								}
-								else
-								{
-									//same block
-									//get the row number
-									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
-									newRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-
-									//calculate row difference
-									rowDifference = newRowNumber - lastRowNumber;
-
-									//update last row number
-									lastRowNumber = newRowNumber;
-
-									//move to the new row
-									for(int i=0; i < rowDifference; i++)
-									{
-										reader.next(key);
-									}
-									//read record
-									reader.getCurrentRow(value);
-
-									//copy it to the buffer if there is enough space
-									int sizeOfNextTuple = getTupleSize(value) + 1;
-									if(sizeOfNextTuple + numBytes > len)
-									{
-										//mark waiting value
-										pendingValue = true;
-										return numBytes;
-									}
-									copyCurrentTuple(buffer, offset + numBytes);
-									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-									numBytes += sizeOfNextTuple;
-								}
-							}
-							//move to next tuple
-							currentTupleIdx++;
-						}	
-						//no more tuples in frame
-						return (numBytes == 0) ? -1 : numBytes;
-					}
-
-					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
-						int rcOffset = 0;
-						for(int i=0; i< value.size(); i++)
-						{
-							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
-							rcOffset += value.get(i).getLength() + 1;
-							buffer[rcOffset - 1] = delimiter;
-						}
-					}
-
-					private int getTupleSize(BytesRefArrayWritable value2) {
-						int size=0;
-						//loop over rc column and add lengths
-						for(int i=0; i< value.size(); i++)
-						{
-							size += value.get(i).getLength();
-						}
-						//add delimeters bytes sizes
-						size += value.size() -1;
-						return size;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-				};
-			}
-			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT))
-			{
-				return new InputStream() {
-					private FSDataInputStream reader;
-					private String lastFileName = "";
-					private String newFileName;
-					private int EOL = "\n".getBytes()[0];
-					private int currentTupleIdx;
-					private int numberOfTuplesInCurrentFrame;
-					private long byteLocation;
-					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
-					private String value;
-					private String pendingValue = null;
-					private ByteBufferInputStream bbis = new ByteBufferInputStream();
-					private DataInputStream dis = new DataInputStream(bbis);
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-						if(newFrame)
-						{
-							//first time called with this frame
-							//reset frame buffer
-							tupleAccessor.reset(frameBuffer);
-							//get number of tuples in frame
-							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
-							//set tuple index to first tuple
-							currentTupleIdx = 0;
-							//set new frame to false
-							newFrame = false;
-						}
-
-						//check and see if there is a pending value
-						int numBytes = 0;
-						if (pendingValue != null) {
-							//last value didn't fit into buffer
-							int sizeOfNextTuple = pendingValue.length() + 1;
-							if(sizeOfNextTuple > len)
-							{
-								return 0;
-							}
-							//there is enough space
-							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.length());
-							buffer[offset + numBytes + pendingValue.length()] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = null;
-							//move to next tuple
-							currentTupleIdx++;
-						}
-
-						//No pending value or done with pending value
-						//check if there are more tuples in the frame
-						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
-						{
-							//get the fileName
-							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
-							newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
-							//check if it is a new file
-							if(!lastFileName.equals(newFileName))
-							{
-								//new file
-								lastFileName = newFileName;
-								//close old file
-								if(reader != null)
-								{
-									reader.close();
-								}
-								//open new file
-								reader = fs.open(new Path(lastFileName));
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								value = reader.readLine();
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.length() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
-								buffer[offset + numBytes + value.length()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							else
-							{
-								//same file, just seek and read
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								value = reader.readLine();
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.length() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
-								buffer[offset + numBytes + value.length()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							currentTupleIdx++;
-						}
-						return (numBytes == 0) ? -1 : numBytes;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-
-					@Override
-					public void close(){
-						try {
-							if (reader != null)
-							{
-								reader.close();
-							}
-							super.close();
-						} catch (IOException e) {
-							e.printStackTrace();
-						}
-					}
-
-				};
-			}
-			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))
-			{
-				return new InputStream() {
-					private SequenceFile.Reader reader;
-					private Writable key;
-					private Text value;
-					private String lastFileName = "";
-					private String newFileName;
-					private long byteLocation;
-					private int EOL = "\n".getBytes()[0];
-					private int currentTupleIdx;
-					private int numberOfTuplesInCurrentFrame;
-					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
-					private Text pendingValue = null;
-					private ByteBufferInputStream bbis = new ByteBufferInputStream();
-					private DataInputStream dis = new DataInputStream(bbis);
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-
-						if(newFrame)
-						{
-							//first time called with this frame
-							//reset frame buffer
-							tupleAccessor.reset(frameBuffer);
-							//get number of tuples in frame
-							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
-							//set tuple index to first tuple
-							currentTupleIdx = 0;
-							//set new frame to false
-							newFrame = false;
-						}
-
-						//check and see if there is a pending value
-						//Double check this
-						int numBytes = 0;
-						if (pendingValue != null) {
-							//last value didn't fit into buffer
-							int sizeOfNextTuple = pendingValue.getLength() + 1;
-							if(sizeOfNextTuple > len)
-							{
-								return 0;
-							}
-							//there is enough space
-							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-							buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = null;
-							//move to next tuple
-							currentTupleIdx++;
-						}
-
-						//No pending value or done with pending value
-						//check if there are more tuples in the frame
-						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
-						{
-							//get the fileName]
-							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
-							newFileName = ((AString) inRecDesc.getFields()[0].deserialize(dis)).getStringValue();
-							//check if it is a new file
-							if(!lastFileName.equals(newFileName))
-							{
-								//new file
-								lastFileName = newFileName;
-								//close old file
-								if(reader != null)
-								{
-									reader.close();
-								}
-								//open new file
-								reader = new SequenceFile.Reader(fs,new Path(lastFileName),conf);
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								reader.next(key, value);
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.getLength() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							else
-							{
-								//same file, just seek and read
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								reader.next(key, value);
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.getLength() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							currentTupleIdx++;
-						}
-						return (numBytes == 0) ? -1 : numBytes;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-
-					@Override
-					public void close(){
-						try {
-							if (reader != null)
-							{
-								reader.close();
-							}
-							super.close();
-						} catch (IOException e) {
-							e.printStackTrace();
-						}
-					}
-				};
-			}
-			//unknow format
-			throw new IOException("Unknown input format");
-		}
-		else
-		{
-			//optimized
-			//different input stream implementation based on the input format
-			if(inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_RC))
-			{
-				return new InputStream() {
-					private RCFile.Reader reader;
-					private int rowDifference;
-					private int lastFileNumber = -1;
-					private int newFileNumber = 0;
-					private long lastByteLocation = 0;
-					private long newByteLocation = 0;
-					private int lastRowNumber = 0;
-					private int newRowNumber = 0;
-					private LongWritable key;
-					private BytesRefArrayWritable value;
-					private int EOL = "\n".getBytes()[0];
-					private byte delimiter = 0x01;
-					private boolean pendingValue = false;
-					private int currentTupleIdx;
-					private int numberOfTuplesInCurrentFrame;
-					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
-					private ByteBufferInputStream bbis = new ByteBufferInputStream();
-					private DataInputStream dis = new DataInputStream(bbis);
-
-					@Override
-					public void close()
-					{
-						if (reader != null)
-						{
-							reader.close();
-						}
-						try {
-							super.close();
-						} catch (IOException e) {
-							e.printStackTrace();
-						}
-					}
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-						if(newFrame)
-						{
-							//first time called with this frame
-							//reset frame buffer
-							tupleAccessor.reset(frameBuffer);
-							//get number of tuples in frame
-							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
-							//set tuple index to first tuple
-							currentTupleIdx = 0;
-							//set new frame to false
-							newFrame = false;
-							pendingValue = false;
-						}
-
-						//check and see if there is a pending value
-						//Double check this
-						int numBytes = 0;
-						if (pendingValue) {
-							//last value didn't fit into buffer
-							int sizeOfNextTuple = getTupleSize(value) + 1;
-							if(sizeOfNextTuple > len)
-							{
-								return 0;
-							}
-							copyCurrentTuple(buffer, offset + numBytes);
-							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = false;
-							//move to next tuple
-							currentTupleIdx++;
-						}
-
-						//No pending value or done with pending value
-						//check if there are more tuples in the frame
-						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
-						{
-							//get 3 things from the current tuple in the frame(File name, byte location and row number)
-							//get the fileName
-							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
-							newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
-							//check if it is a new file
-							if(lastFileNumber != newFileNumber)
-							{
-								//new file
-								lastFileNumber = newFileNumber;
-								//close old file
-								if(reader != null)
-								{
-									reader.close();
-								}
-								//open new file
-								reader = new Reader(fs, new Path(files.get(newFileNumber)), conf);
-								//read and save byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								lastByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(lastByteLocation);
-								//read and save rowNumber
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
-								lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-								//loop until row
-								for(int i=0; i < lastRowNumber; i++)
-								{
-									//this loop perform a single I/O and move to the next record in the block which is already in memory
-									//if no more records in the current block, it perform another I/O and get the next block
-									//<this should never happen here>
-									reader.next(key);
-								}
-								//read record
-								reader.getCurrentRow(value);
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = getTupleSize(value) + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = true;
-									return numBytes;
-								}
-								copyCurrentTuple(buffer, offset + numBytes);
-								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							else
-							{
-								//same file
-								//get the byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								newByteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-
-								//check if same block
-								if(lastByteLocation != newByteLocation)
-								{
-									//new block
-									lastByteLocation = newByteLocation;
-									//seek
-									reader.seek(lastByteLocation);
-									//read and save rowNumber
-									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
-									lastRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-									//loop until row
-									for(int i=0; i < lastRowNumber; i++)
-									{
-										reader.next(key);
-									}
-									//read record
-									reader.getCurrentRow(value);
-									//copy it to the buffer if there is enough space
-									int sizeOfNextTuple = getTupleSize(value) + 1;
-									if(sizeOfNextTuple + numBytes > len)
-									{
-										//mark waiting value
-										pendingValue = true;
-										return numBytes;
-									}
-									copyCurrentTuple(buffer, offset + numBytes);
-									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-									numBytes += sizeOfNextTuple;
-								}
-								else
-								{
-									//same block
-									//get the row number
-									bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 2));
-									newRowNumber = ((AInt32)(inRecDesc.getFields()[2].deserialize(dis))).getIntegerValue();
-
-									//calculate row difference
-									rowDifference = newRowNumber - lastRowNumber;
-
-									//update last row number
-									lastRowNumber = newRowNumber;
-
-									//move to the new row
-									for(int i=0; i < rowDifference; i++)
-									{
-										reader.next(key);
-									}
-									//read record
-									reader.getCurrentRow(value);
-
-									//copy it to the buffer if there is enough space
-									int sizeOfNextTuple = getTupleSize(value) + 1;
-									if(sizeOfNextTuple + numBytes > len)
-									{
-										//mark waiting value
-										pendingValue = true;
-										return numBytes;
-									}
-									copyCurrentTuple(buffer, offset + numBytes);
-									buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-									numBytes += sizeOfNextTuple;
-								}
-							}
-							//move to next tuple
-							currentTupleIdx++;
-						}	
-						//no more tuples in frame
-						return (numBytes == 0) ? -1 : numBytes;
-					}
-
-					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
-						int rcOffset = 0;
-						for(int i=0; i< value.size(); i++)
-						{
-							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
-							rcOffset += value.get(i).getLength() + 1;
-							buffer[rcOffset - 1] = delimiter;
-						}
-					}
-
-					private int getTupleSize(BytesRefArrayWritable value2) {
-						int size=0;
-						//loop over rc column and add lengths
-						for(int i=0; i< value.size(); i++)
-						{
-							size += value.get(i).getLength();
-						}
-						//add delimeters bytes sizes
-						size += value.size() -1;
-						return size;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-				};
-			}
-			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_TEXT))
-			{
-				return new InputStream() {
-					private FSDataInputStream reader;
-					private int lastFileNumber = -1;
-					private int newFileNumber = 0;
-					private int EOL = "\n".getBytes()[0];
-					private int currentTupleIdx;
-					private int numberOfTuplesInCurrentFrame;
-					private long byteLocation;
-					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
-					private String value;
-					private String pendingValue = null;
-					private ByteBufferInputStream bbis = new ByteBufferInputStream();
-					private DataInputStream dis = new DataInputStream(bbis);
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-						if(newFrame)
-						{
-							//first time called with this frame
-							//reset frame buffer
-							tupleAccessor.reset(frameBuffer);
-							//get number of tuples in frame
-							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
-							//set tuple index to first tuple
-							currentTupleIdx = 0;
-							//set new frame to false
-							newFrame = false;
-						}
-
-						//check and see if there is a pending value
-						int numBytes = 0;
-						if (pendingValue != null) {
-							//last value didn't fit into buffer
-							int sizeOfNextTuple = pendingValue.length() + 1;
-							if(sizeOfNextTuple > len)
-							{
-								return 0;
-							}
-							//there is enough space
-							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.length());
-							buffer[offset + numBytes + pendingValue.length()] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = null;
-							//move to next tuple
-							currentTupleIdx++;
-						}
-
-						//No pending value or done with pending value
-						//check if there are more tuples in the frame
-						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
-						{
-							//get the file number
-							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
-							newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
-							//check if it is a new file
-							if(lastFileNumber != newFileNumber)
-							{
-								//new file
-								lastFileNumber = newFileNumber;
-								//close old file
-								if(reader != null)
-								{
-									reader.close();
-								}
-								
-								//open new file
-								reader = fs.open(new Path(files.get(newFileNumber)));
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								value = reader.readLine();
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.length() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
-								buffer[offset + numBytes + value.length()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							else
-							{
-								//same file, just seek and read
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								value = reader.readLine();
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.length() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.length());
-								buffer[offset + numBytes + value.length()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							currentTupleIdx++;
-						}
-						return (numBytes == 0) ? -1 : numBytes;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-
-					@Override
-					public void close(){
-						try {
-							if (reader != null)
-							{
-								reader.close();
-							}
-							super.close();
-						} catch (IOException e) {
-							e.printStackTrace();
-						}
-					}
-
-				};
-			}
-			else if (inputFormat.equals(HDFSAdapterFactory.INPUT_FORMAT_SEQUENCE))
-			{
-				return new InputStream() {
-					private SequenceFile.Reader reader;
-					private Writable key;
-					private Text value;
-					private int lastFileNumber = -1;
-					private int newFileNumber = 0;
-					private long byteLocation;
-					private int EOL = "\n".getBytes()[0];
-					private int currentTupleIdx;
-					private int numberOfTuplesInCurrentFrame;
-					private IFrameTupleAccessor tupleAccessor = new FrameTupleAccessor(ctx.getFrameSize(),inRecDesc);
-					private Text pendingValue = null;
-					private ByteBufferInputStream bbis = new ByteBufferInputStream();
-					private DataInputStream dis = new DataInputStream(bbis);
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-
-						if(newFrame)
-						{
-							//first time called with this frame
-							//reset frame buffer
-							tupleAccessor.reset(frameBuffer);
-							//get number of tuples in frame
-							numberOfTuplesInCurrentFrame = tupleAccessor.getTupleCount();
-							//set tuple index to first tuple
-							currentTupleIdx = 0;
-							//set new frame to false
-							newFrame = false;
-						}
-
-						//check and see if there is a pending value
-						//Double check this
-						int numBytes = 0;
-						if (pendingValue != null) {
-							//last value didn't fit into buffer
-							int sizeOfNextTuple = pendingValue.getLength() + 1;
-							if(sizeOfNextTuple > len)
-							{
-								return 0;
-							}
-							//there is enough space
-							System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-							buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = null;
-							//move to next tuple
-							currentTupleIdx++;
-						}
-
-						//No pending value or done with pending value
-						//check if there are more tuples in the frame
-						while(currentTupleIdx < numberOfTuplesInCurrentFrame)
-						{
-							//get the fileName]
-							bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 0));
-							newFileNumber = ((AInt32) inRecDesc.getFields()[0].deserialize(dis)).getIntegerValue();
-							//check if it is a new file
-							if(lastFileNumber != newFileNumber)
-							{
-								//new file
-								lastFileNumber = newFileNumber;
-								//close old file
-								if(reader != null)
-								{
-									reader.close();
-								}
-								//open new file
-								reader = new SequenceFile.Reader(fs,new Path(files.get(newFileNumber)),conf);
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								reader.next(key, value);
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.getLength() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							else
-							{
-								//same file, just seek and read
-								//read byte location
-								bbis.setByteBuffer(frameBuffer, tupleAccessor.getTupleStartOffset(currentTupleIdx) + tupleAccessor.getFieldSlotsLength() + tupleAccessor.getFieldStartOffset(currentTupleIdx, 1));
-								byteLocation = ((AInt64) inRecDesc.getFields()[1].deserialize(dis)).getLongValue();
-								//seek
-								reader.seek(byteLocation);
-								//read record
-								reader.next(key, value);
-								//copy it to the buffer if there is enough space
-								int sizeOfNextTuple = value.getLength() + 1;
-								if(sizeOfNextTuple + numBytes > len)
-								{
-									//mark waiting value
-									pendingValue = value;
-									return numBytes;
-								}
-								System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-								buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-							currentTupleIdx++;
-						}
-						return (numBytes == 0) ? -1 : numBytes;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-
-					@Override
-					public void close(){
-						try {
-							if (reader != null)
-							{
-								reader.close();
-							}
-							super.close();
-						} catch (IOException e) {
-							e.printStackTrace();
-						}
-					}
-				};
-			}
-			//unknow format
-			throw new IOException("Unknown input format");
-		}
-	}
-
-	@Override
-	public AlgebricksPartitionConstraint getPartitionConstraint()
-			throws Exception {
-		return partitionConstraint;
-	}
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
index 2b355ff..f8b381b 100644
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
+++ b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSAdapter.java
@@ -18,7 +18,6 @@
 import java.io.InputStream;
 import java.util.Map;
 
-import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.InputSplit;
@@ -27,8 +26,6 @@
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
 
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -37,348 +34,202 @@
 
 /**
  * Provides functionality for fetching external data stored in an HDFS instance.
- * Note: reader are never closed in adapters. should we make sure they are closed before returning or switching to a different reader?
  */
 @SuppressWarnings({ "deprecation", "rawtypes" })
 public class HDFSAdapter extends FileSystemBasedAdapter {
 
-	private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 1L;
 
-	private transient String[] readSchedule;
-	private transient boolean executed[];
-	private transient InputSplit[] inputSplits;
-	private transient JobConf conf;
-	private transient AlgebricksPartitionConstraint clusterLocations;
+    private transient String[] readSchedule;
+    private transient boolean executed[];
+    private transient InputSplit[] inputSplits;
+    private transient JobConf conf;
+    private transient AlgebricksPartitionConstraint clusterLocations;
 
-	private transient String nodeName;
+    private transient String nodeName;
 
-	public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
-			AlgebricksPartitionConstraint clusterLocations) {
-		super(atype);
-		this.readSchedule = readSchedule;
-		this.executed = executed;
-		this.inputSplits = inputSplits;
-		this.conf = conf;
-		this.clusterLocations = clusterLocations;
-	}
+    public HDFSAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
+            AlgebricksPartitionConstraint clusterLocations) {
+        super(atype);
+        this.readSchedule = readSchedule;
+        this.executed = executed;
+        this.inputSplits = inputSplits;
+        this.conf = conf;
+        this.clusterLocations = clusterLocations;
+    }
 
-	@Override
-	public void configure(Map<String, Object> arguments) throws Exception {
-		this.configuration = arguments;
-		configureFormat();
-	}
+    @Override
+    public void configure(Map<String, Object> arguments) throws Exception {
+        this.configuration = arguments;
+        configureFormat();
+    }
 
-	public AdapterType getAdapterType() {
-		return AdapterType.READ_WRITE;
-	}
+    public AdapterType getAdapterType() {
+        return AdapterType.READ_WRITE;
+    }
 
-	@Override
-	public void initialize(IHyracksTaskContext ctx) throws Exception {
-		this.ctx = ctx;
-		this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
-	}
+    @Override
+    public void initialize(IHyracksTaskContext ctx) throws Exception {
+        this.ctx = ctx;
+        this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
+    }
 
-	private Reporter getReporter() {
-		Reporter reporter = new Reporter() {
+    private Reporter getReporter() {
+        Reporter reporter = new Reporter() {
 
-			@Override
-			public Counter getCounter(Enum<?> arg0) {
-				return null;
-			}
+            @Override
+            public Counter getCounter(Enum<?> arg0) {
+                return null;
+            }
 
-			@Override
-			public Counter getCounter(String arg0, String arg1) {
-				return null;
-			}
+            @Override
+            public Counter getCounter(String arg0, String arg1) {
+                return null;
+            }
 
-			@Override
-			public InputSplit getInputSplit() throws UnsupportedOperationException {
-				return null;
-			}
+            @Override
+            public InputSplit getInputSplit() throws UnsupportedOperationException {
+                return null;
+            }
 
-			@Override
-			public void incrCounter(Enum<?> arg0, long arg1) {
-			}
+            @Override
+            public void incrCounter(Enum<?> arg0, long arg1) {
+            }
 
-			@Override
-			public void incrCounter(String arg0, String arg1, long arg2) {
-			}
+            @Override
+            public void incrCounter(String arg0, String arg1, long arg2) {
+            }
 
-			@Override
-			public void setStatus(String arg0) {
-			}
+            @Override
+            public void setStatus(String arg0) {
+            }
 
-			@Override
-			public void progress() {
-			}
-		};
+            @Override
+            public void progress() {
+            }
+        };
 
-		return reporter;
-	}
+        return reporter;
+    }
 
-	@Override
-	public InputStream getInputStream(int partition) throws IOException {
+    @Override
+    public InputStream getInputStream(int partition) throws IOException {
 
-		if(conf.getInputFormat() instanceof RCFileInputFormat)
-		{
-			//if hdfs input format is rc-input-format, we return a different InputStream
-			return new InputStream() {
+        return new InputStream() {
 
-				private RecordReader<LongWritable, BytesRefArrayWritable> reader;
-				private LongWritable key;
-				private BytesRefArrayWritable value;
-				private boolean hasMore = false;
-				private int EOL = "\n".getBytes()[0];
-				private byte delimiter = 0x01;
-				private boolean pendingValue = false;
-				private int currentSplitIndex = 0;
+            private RecordReader<Object, Text> reader;
+            private Object key;
+            private Text value;
+            private boolean hasMore = false;
+            private int EOL = "\n".getBytes()[0];
+            private Text pendingValue = null;
+            private int currentSplitIndex = 0;
 
-				@SuppressWarnings("unchecked")
-				private boolean moveToNext() throws IOException {
-					for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-						/**
-						 * read all the partitions scheduled to the current node
-						 */
-						if (readSchedule[currentSplitIndex].equals(nodeName)) {
-							/**
-							 * pick an unread split to read
-							 * synchronize among simultaneous partitions in the same machine
-							 */
-							synchronized (executed) {
-								if (executed[currentSplitIndex] == false) {
-									executed[currentSplitIndex] = true;
-								} else {
-									continue;
-								}
-							}
+            @SuppressWarnings("unchecked")
+            private boolean moveToNext() throws IOException {
+                for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
+                    /**
+                     * read all the partitions scheduled to the current node
+                     */
+                    if (readSchedule[currentSplitIndex].equals(nodeName)) {
+                        /**
+                         * pick an unread split to read
+                         * synchronize among simultaneous partitions in the same machine
+                         */
+                        synchronized (executed) {
+                            if (executed[currentSplitIndex] == false) {
+                                executed[currentSplitIndex] = true;
+                            } else {
+                                continue;
+                            }
+                        }
 
-							/**
-							 * read the split
-							 */
-							reader = getRecordReader(currentSplitIndex);
-							key = reader.createKey();
-							value = reader.createValue();
-							return true;
-						}
-					}
-					return false;
-				}
+                        /**
+                         * read the split
+                         */
+                        reader = getRecordReader(currentSplitIndex);
+                        key = reader.createKey();
+                        value = (Text) reader.createValue();
+                        return true;
+                    }
+                }
+                return false;
+            }
 
-				@Override
-				public int read(byte[] buffer, int offset, int len) throws IOException {
-					if (reader == null) {
-						if (!moveToNext()) {
-							//nothing to read
-							return -1;
-						}
-					}
+            @Override
+            public int read(byte[] buffer, int offset, int len) throws IOException {
+                if (reader == null) {
+                    if (!moveToNext()) {
+                        //nothing to read
+                        return -1;
+                    }
+                }
 
-					int numBytes = 0;
-					if (pendingValue) {
-						//last value didn't fit into buffer
-						int sizeOfNextTuple = getTupleSize(value) + 1;
-						if(sizeOfNextTuple > len)
-						{
-							return 0;
-						}
-						copyCurrentTuple(buffer, offset + numBytes);
-						buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-						numBytes += sizeOfNextTuple;
-						//set pending to false
-						pendingValue = false;
-					}
+                int numBytes = 0;
+                if (pendingValue != null) {
+                    System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
+                    buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
+                    numBytes += pendingValue.getLength() + 1;
+                    pendingValue = null;
+                }
 
-					while (numBytes < len) {
-						hasMore = reader.next(key, value);
-						if (!hasMore) {
-							while (moveToNext()) {
-								hasMore = reader.next(key, value);
-								if (hasMore) {
-									//move to the next non-empty split
-									break;
-								}
-							}
-						}
-						if (!hasMore) {
-							return (numBytes == 0) ? -1 : numBytes;
-						}
-						int sizeOfNextTuple = getTupleSize(value) + 1;
-						if (numBytes + sizeOfNextTuple > len) {
-							// cannot add tuple to current buffer
-							// but the reader has moved pass the fetched tuple
-							// we need to store this for a subsequent read call.
-							// and return this then.
-							pendingValue = true;
-							break;
-						} else {
-							//copy
-							copyCurrentTuple(buffer, offset + numBytes);
-							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-						}
-					}
-					return numBytes;
-				}
+                while (numBytes < len) {
+                    hasMore = reader.next(key, value);
+                    if (!hasMore) {
+                        while (moveToNext()) {
+                            hasMore = reader.next(key, value);
+                            if (hasMore) {
+                                //move to the next non-empty split
+                                break;
+                            }
+                        }
+                    }
+                    if (!hasMore) {
+                        return (numBytes == 0) ? -1 : numBytes;
+                    }
+                    int sizeOfNextTuple = value.getLength() + 1;
+                    if (numBytes + sizeOfNextTuple > len) {
+                        // cannot add tuple to current buffer
+                        // but the reader has moved pass the fetched tuple
+                        // we need to store this for a subsequent read call.
+                        // and return this then.
+                        pendingValue = value;
+                        break;
+                    } else {
+                        System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
+                        buffer[offset + numBytes + value.getLength()] = (byte) EOL;
+                        numBytes += sizeOfNextTuple;
+                    }
+                }
+                return numBytes;
+            }
 
-				private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
-					int rcOffset = 0;
-					for(int i=0; i< value.size(); i++)
-					{
-						System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
-						rcOffset += value.get(i).getLength() + 1;
-						buffer[rcOffset - 1] = delimiter;
-					}
-				}
+            @Override
+            public int read() throws IOException {
+                throw new NotImplementedException("Use read(byte[], int, int");
+            }
 
-				private int getTupleSize(BytesRefArrayWritable value2) {
-					int size=0;
-					//loop over rc column and add lengths
-					for(int i=0; i< value.size(); i++)
-					{
-						size += value.get(i).getLength();
-					}
-					//add delimeters bytes sizes
-					size += value.size() -1;
-					return size;
-				}
+            private RecordReader getRecordReader(int slitIndex) throws IOException {
+                if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
+                    SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
+                    RecordReader reader = format.getRecordReader(
+                            (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+                    return reader;
+                } else {
+                    TextInputFormat format = (TextInputFormat) conf.getInputFormat();
+                    RecordReader reader = format.getRecordReader(
+                            (org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
+                    return reader;
+                }
+            }
 
-				@Override
-				public int read() throws IOException {
-					throw new NotImplementedException("Use read(byte[], int, int");
-				}
+        };
 
-				private RecordReader getRecordReader(int slitIndex) throws IOException {
-					RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
-					RecordReader reader = format.getRecordReader(
-							(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-					return reader;
-				}
+    }
 
-			};
-		}
-		else
-		{
-			return new InputStream() {
-
-				private RecordReader<Object, Text> reader;
-				private Object key;
-				private Text value;
-				private boolean hasMore = false;
-				private int EOL = "\n".getBytes()[0];
-				private Text pendingValue = null;
-				private int currentSplitIndex = 0;
-
-				@SuppressWarnings("unchecked")
-				private boolean moveToNext() throws IOException {
-					for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-						/**
-						 * read all the partitions scheduled to the current node
-						 */
-						if (readSchedule[currentSplitIndex].equals(nodeName)) {
-							/**
-							 * pick an unread split to read
-							 * synchronize among simultaneous partitions in the same machine
-							 */
-							synchronized (executed) {
-								if (executed[currentSplitIndex] == false) {
-									executed[currentSplitIndex] = true;
-								} else {
-									continue;
-								}
-							}
-
-							/**
-							 * read the split
-							 */
-							reader = getRecordReader(currentSplitIndex);
-							key = reader.createKey();
-							value = (Text) reader.createValue();
-							return true;
-						}
-					}
-					return false;
-				}
-
-				@Override
-				public int read(byte[] buffer, int offset, int len) throws IOException {
-					if (reader == null) {
-						if (!moveToNext()) {
-							//nothing to read
-							return -1;
-						}
-					}
-
-					int numBytes = 0;
-					if (pendingValue != null) {
-						int sizeOfNextTuple = pendingValue.getLength() + 1;
-						if(sizeOfNextTuple > len)
-						{
-							return 0;
-						}
-						System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-						buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-						numBytes += pendingValue.getLength() + 1;
-						pendingValue = null;
-					}
-
-					while (numBytes < len) {
-						hasMore = reader.next(key, value);
-						if (!hasMore) {
-							while (moveToNext()) {
-								hasMore = reader.next(key, value);
-								if (hasMore) {
-									//move to the next non-empty split
-									break;
-								}
-							}
-						}
-						if (!hasMore) {
-							return (numBytes == 0) ? -1 : numBytes;
-						}
-						int sizeOfNextTuple = value.getLength() + 1;
-						if (numBytes + sizeOfNextTuple > len) {
-							// cannot add tuple to current buffer
-							// but the reader has moved pass the fetched tuple
-							// we need to store this for a subsequent read call.
-							// and return this then.
-							pendingValue = value;
-							break;
-						} else {
-							System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-							buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-						}
-					}
-					return numBytes;
-				}
-
-				@Override
-				public int read() throws IOException {
-					throw new NotImplementedException("Use read(byte[], int, int");
-				}
-
-				private RecordReader getRecordReader(int slitIndex) throws IOException {
-					if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-						SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-						RecordReader reader = format.getRecordReader(
-								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-						return reader;
-					} else {
-						TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-						RecordReader reader = format.getRecordReader(
-								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-						return reader;
-					}
-				}
-
-			};
-		}
-
-	}
-
-	@Override
-	public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-		return clusterLocations;
-	}
+    @Override
+    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
+        return clusterLocations;
+    }
 
 }
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
deleted file mode 100644
index 59b39c5..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HDFSIndexingAdapter.java
+++ /dev/null
@@ -1,1208 +0,0 @@
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
-import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.FileSplit;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.mapred.Reporter;
-import org.apache.hadoop.mapred.SequenceFileInputFormat;
-import org.apache.hadoop.mapred.TextInputFormat;
-import org.apache.hadoop.mapred.Counters.Counter;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksException;
-
-
-/**
- * Provides functionality for reading external files and attach RID info to them before they are sent to the parser
- * 
- *  Room for optimization: get numbers of indexing fields (or fields names for adm) and do a quick filtering before sending to parser
- *  
- */
-@SuppressWarnings({ "deprecation", "rawtypes" })
-public class HDFSIndexingAdapter extends FileSystemBasedAdapter {
-
-	private static final long serialVersionUID = 1L;
-	private transient String[] readSchedule;
-	private transient boolean executed[];
-	private transient InputSplit[] inputSplits;
-	private transient JobConf conf;
-	private transient AlgebricksPartitionConstraint clusterLocations;
-	private final Map<String,Integer> files;
-	private transient String nodeName;
-	public static final byte[] fileNameFieldNameWithRecOpeningBraces = "{\"_file-name\":\"".getBytes();
-	public static final byte[] fileNameFieldClosingQuotation = "\"".getBytes();
-	public static final byte[] fileNumberFieldNameWithRecOpeningBraces = "{\"_file-number\":".getBytes();
-	public static final byte[] bytelocationFieldName = ",\"_byte-location\":".getBytes();
-	public static final byte[] bytelocationValueEnd = "i64,".getBytes();
-
-	public HDFSIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
-			AlgebricksPartitionConstraint clusterLocations, Map<String,Integer> files) {
-		super(atype);
-		this.readSchedule = readSchedule;
-		this.executed = executed;
-		this.inputSplits = inputSplits;
-		this.conf = conf;
-		this.clusterLocations = clusterLocations;
-		this.files = files;
-	}
-
-	@Override
-	public void configure(Map<String, Object> arguments) throws Exception {
-		this.configuration = arguments;
-		configureFormat();
-	}
-
-	public AdapterType getAdapterType() {
-		return AdapterType.READ;
-	}
-
-	@Override
-	public void initialize(IHyracksTaskContext ctx) throws Exception {
-		this.ctx = ctx;
-		this.nodeName = ctx.getJobletContext().getApplicationContext().getNodeId();
-	}
-
-	private Reporter getReporter() {
-		Reporter reporter = new Reporter() {
-
-			@Override
-			public Counter getCounter(Enum<?> arg0) {
-				return null;
-			}
-
-			@Override
-			public Counter getCounter(String arg0, String arg1) {
-				return null;
-			}
-
-			@Override
-			public InputSplit getInputSplit() throws UnsupportedOperationException {
-				return null;
-			}
-
-			@Override
-			public void incrCounter(Enum<?> arg0, long arg1) {
-			}
-
-			@Override
-			public void incrCounter(String arg0, String arg1, long arg2) {
-			}
-
-			@Override
-			public void setStatus(String arg0) {
-			}
-
-			@Override
-			public void progress() {
-			}
-		};
-
-		return reporter;
-	}
-
-	@Override
-	public InputStream getInputStream(int partition) throws IOException {
-		if(files == null)
-		{
-			if(conf.getInputFormat() instanceof RCFileInputFormat)
-			{
-				//indexing rc input format
-				return new InputStream() {
-
-					private RecordReader<LongWritable, BytesRefArrayWritable> reader;
-					private LongWritable key;
-					private BytesRefArrayWritable value;
-					private boolean hasMore = false;
-					private int EOL = "\n".getBytes()[0];
-					private byte delimiter = 0x01;
-					private boolean pendingValue = false;
-					private int currentSplitIndex = 0;
-					private byte[] fileName;
-					private byte[] byteLocation;
-					private byte[] rowNumberBytes;
-					private long blockByteLocation;
-					private long NextblockByteLocation;
-					private int rowNumber;
-
-					@SuppressWarnings("unchecked")
-					private boolean moveToNext() throws IOException {
-						for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-							/**
-							 * read all the partitions scheduled to the current node
-							 */
-							if (readSchedule[currentSplitIndex].equals(nodeName)) {
-								/**
-								 * pick an unread split to read
-								 * synchronize among simultaneous partitions in the same machine
-								 */
-								synchronized (executed) {
-									if (executed[currentSplitIndex] == false) {
-										executed[currentSplitIndex] = true;
-									} else {
-										continue;
-									}
-								}
-
-								/**
-								 * read the split
-								 */
-								reader = getRecordReader(currentSplitIndex);
-								key = reader.createKey();
-								value = reader.createValue();
-								fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
-								blockByteLocation = reader.getPos();
-								pendingValue = reader.next(key, value);
-								NextblockByteLocation = reader.getPos();
-								rowNumber = 1;
-								byteLocation = String.valueOf(blockByteLocation).getBytes("UTF-8");
-								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-								return true;
-							}
-						}
-						return false;
-					}
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-						if (reader == null) {
-							if (!moveToNext()) {
-								//nothing to read
-								return -1;
-							}
-						}
-
-						int numBytes = 0;
-						if (pendingValue) {
-							//last value didn't fit into buffer
-							// 1 for EOL
-							int sizeOfNextTuple = getTupleSize(value) + 1;
-							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileName.length + 3 > len) {
-								return 0;
-							}
-
-							//copy filename
-							System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
-							buffer[offset + numBytes + fileName.length] = delimiter;
-							numBytes += fileName.length + 1;
-
-							//copy byte location
-							System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-							buffer[offset + numBytes + byteLocation.length] = delimiter;
-							numBytes += byteLocation.length + 1;
-
-							//copy row number
-							System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
-							buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
-							numBytes += rowNumberBytes.length + 1;
-
-							copyCurrentTuple(buffer, offset + numBytes);
-							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = false;
-						}
-
-						while (numBytes < len) {
-							hasMore = reader.next(key, value);
-							if (!hasMore) {
-								while (moveToNext()) {
-									hasMore = reader.next(key, value);
-									if (hasMore) {
-										//move to the next non-empty split
-										break;
-									}
-								}
-							}
-							if (!hasMore) {
-								return (numBytes == 0) ? -1 : numBytes;
-							}
-
-							//check if moved to next block
-							blockByteLocation = reader.getPos();
-							if(blockByteLocation != NextblockByteLocation)
-							{
-								//moved to a new block, reset stuff
-								//row number
-								rowNumber = 1;
-								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-
-								//block location
-								byteLocation = String.valueOf(NextblockByteLocation).getBytes("UTF-8");
-								NextblockByteLocation = blockByteLocation;
-							}
-							else
-							{
-								rowNumber += 1;
-								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-							}
-
-							int sizeOfNextTuple = getTupleSize(value) + 1;
-							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileName.length + 3 > len) {
-								// cannot add tuple to current buffer
-								// but the reader has moved pass the fetched tuple
-								// we need to store this for a subsequent read call.
-								// and return this then.
-								pendingValue = true;
-								break;
-							} else {
-								//copy filename
-								System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
-								buffer[offset + numBytes + fileName.length] = delimiter;
-								numBytes += fileName.length + 1;
-
-								//copy byte location
-								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-								buffer[offset + numBytes + byteLocation.length] = delimiter;
-								numBytes += byteLocation.length + 1;
-
-								//copy row number
-								System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
-								buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
-								numBytes += rowNumberBytes.length + 1;
-
-								copyCurrentTuple(buffer, offset + numBytes);
-								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-						}
-						return numBytes;
-					}
-
-					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
-						int rcOffset = 0;
-						for(int i=0; i< value.size(); i++)
-						{
-							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
-							rcOffset += value.get(i).getLength() + 1;
-							buffer[rcOffset - 1] = delimiter;
-						}
-					}
-
-					private int getTupleSize(BytesRefArrayWritable value2) {
-						int size=0;
-						//loop over rc column and add lengths
-						for(int i=0; i< value.size(); i++)
-						{
-							size += value.get(i).getLength();
-						}
-						//add delimeters bytes sizes
-						size += value.size() -1;
-						return size;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-
-					private RecordReader getRecordReader(int slitIndex) throws IOException {
-						RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
-						RecordReader reader = format.getRecordReader(
-								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-						return reader;
-					}
-
-				};
-			}
-			else
-			{
-				//get content format
-				if(configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT))
-				{
-					//reading data and RIDs for delimited text
-					return new InputStream() {
-
-						private RecordReader<Object, Text> reader;
-						private Object key;
-						private Text value;
-						private boolean hasMore = false;
-						private int EOL = "\n".getBytes()[0];
-						private Text pendingValue = null;
-						private int currentSplitIndex = 0;
-						private byte[] fileName;
-						private byte[] byteLocation;
-						private byte delimiter = ((String)configuration.get(KEY_DELIMITER)).getBytes()[0];
-
-						@SuppressWarnings("unchecked")
-						private boolean moveToNext() throws IOException {
-							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-								/**
-								 * read all the partitions scheduled to the current node
-								 */
-								if (readSchedule[currentSplitIndex].equals(nodeName)) {
-									/**
-									 * pick an unread split to read
-									 * synchronize among simultaneous partitions in the same machine
-									 */
-									synchronized (executed) {
-										if (executed[currentSplitIndex] == false) {
-											executed[currentSplitIndex] = true;
-										} else {
-											continue;
-										}
-									}
-
-									/**
-									 * read the split
-									 */
-									reader = getRecordReader(currentSplitIndex);
-									key = reader.createKey();
-									value = (Text) reader.createValue();
-									fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
-									return true;
-								}
-							}
-							return false;
-						}
-
-						@Override
-						public int read(byte[] buffer, int offset, int len) throws IOException {
-							if (reader == null) {
-								if (!moveToNext()) {
-									//nothing to read
-									return -1;
-								}
-							}
-
-							int numBytes = 0;
-							if (pendingValue != null) {
-								int sizeOfNextTuple = pendingValue.getLength() + 1;
-								if (numBytes + sizeOfNextTuple +byteLocation.length + fileName.length + 2> len)
-								{
-									return numBytes;
-								}
-								//copy filename
-								System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
-								buffer[offset + numBytes + fileName.length] = delimiter;
-								numBytes += fileName.length + 1;
-
-								//copy byte location
-								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-								buffer[offset + numBytes + byteLocation.length] = delimiter;
-								numBytes += byteLocation.length + 1;
-
-								//copy actual value
-								System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-								buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-								numBytes += pendingValue.getLength() + 1;
-								pendingValue = null;
-							}
-
-							while (numBytes < len) {
-								//get reader position before you actually read
-								byteLocation = String.valueOf(reader.getPos()).getBytes();
-								hasMore = reader.next(key, value);
-								if (!hasMore) {
-									while (moveToNext()) {
-										//get reader position before you actually read
-										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
-										hasMore = reader.next(key, value);
-										if (hasMore) {
-											//move to the next non-empty split
-											break;
-										}
-									}
-								}
-								if (!hasMore) {
-									return (numBytes == 0) ? -1 : numBytes;
-								}
-								int sizeOfNextTuple = value.getLength() + 1;
-								if (numBytes + sizeOfNextTuple +byteLocation.length + fileName.length + 2> len) {
-									// cannot add tuple to current buffer
-									// but the reader has moved pass the fetched tuple
-									// we need to store this for a subsequent read call.
-									// and return this then.
-									pendingValue = value;
-									break;
-								} else {
-									//copy filename
-									System.arraycopy(fileName, 0, buffer, offset + numBytes, fileName.length);
-									buffer[offset + numBytes + fileName.length] = delimiter;
-									numBytes += fileName.length + 1;
-
-									//copy byte location
-									System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-									buffer[offset + numBytes + byteLocation.length] = delimiter;
-									numBytes += byteLocation.length + 1;
-
-									//Copy actual value
-									System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-									buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-									numBytes += sizeOfNextTuple;
-								}
-							}
-							return numBytes;
-						}
-
-						@Override
-						public int read() throws IOException {
-							throw new NotImplementedException("Use read(byte[], int, int");
-						}
-
-						private RecordReader getRecordReader(int slitIndex) throws IOException {
-							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							} else {
-								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							}
-						}
-
-					};
-				}
-				else if((configuration.get(KEY_FORMAT).equals(FORMAT_ADM)))
-				{
-					//reading data and RIDs for adm formatted data
-					return new InputStream() {
-
-						private RecordReader<Object, Text> reader;
-						private Object key;
-						private Text value;
-						private boolean hasMore = false;
-						private int EOL = "\n".getBytes()[0];
-						private Text pendingValue = null;
-						private int currentSplitIndex = 0;
-						private byte[] fileName;
-						private byte[] byteLocation;
-
-						@SuppressWarnings("unchecked")
-						private boolean moveToNext() throws IOException {
-							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-								/**
-								 * read all the partitions scheduled to the current node
-								 */
-								if (readSchedule[currentSplitIndex].equals(nodeName)) {
-									/**
-									 * pick an unread split to read
-									 * synchronize among simultaneous partitions in the same machine
-									 */
-									synchronized (executed) {
-										if (executed[currentSplitIndex] == false) {
-											executed[currentSplitIndex] = true;
-										} else {
-											continue;
-										}
-									}
-
-									/**
-									 * read the split
-									 */
-									reader = getRecordReader(currentSplitIndex);
-									key = reader.createKey();
-									value = (Text) reader.createValue();
-									fileName = ((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath().getBytes();
-									return true;
-								}
-							}
-							return false;
-						}
-
-						@Override
-						public int read(byte[] buffer, int offset, int len) throws IOException {
-							if (reader == null) {
-								if (!moveToNext()) {
-									//nothing to read
-									return -1;
-								}
-							}
-
-							int numBytes = 0;
-							if (pendingValue != null) {
-								int firstFieldLocation = value.find("\"");
-								int admValueSize = value.getLength();
-								if(firstFieldLocation >= 0)
-								{
-									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
-									int sizeOfNextTupleAndRID = fileNameFieldNameWithRecOpeningBraces.length + fileName.length + fileNameFieldClosingQuotation.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
-									if (numBytes + sizeOfNextTupleAndRID > len) {
-										// still cannot add tuple to current buffer
-										// return 0 so parser would double the buffer size.
-										return 0;
-									} else {
-										//copy fileNameFieldNameWithRecOpeningBraces
-										System.arraycopy(fileNameFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNameFieldNameWithRecOpeningBraces.length);
-										numBytes += fileNameFieldNameWithRecOpeningBraces.length;
-										//copy fileName
-										System.arraycopy(fileName, 0, buffer, offset + numBytes,fileName.length);
-										numBytes += fileName.length;
-										//copy fileName closing quotation
-										System.arraycopy(fileNameFieldClosingQuotation, 0, buffer, offset + numBytes,fileNameFieldClosingQuotation.length);
-										numBytes += fileNameFieldClosingQuotation.length;
-										//copy bytelocationFieldName
-										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
-										numBytes += bytelocationFieldName.length;
-										//copy byte location value
-										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
-										numBytes += byteLocation.length;
-										//copy byte location field end 
-										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
-										numBytes += bytelocationValueEnd.length;
-										//copy the actual adm instance
-										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
-										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
-										numBytes += admValueSize - firstFieldLocation +1;
-									}
-								}
-								pendingValue = null;
-							}
-
-							while (numBytes < len) {
-								//get reader position before you actually read
-								byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
-								hasMore = reader.next(key, value);
-								if (!hasMore) {
-									while (moveToNext()) {
-										//get reader position before you actually read
-										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
-										hasMore = reader.next(key, value);
-										if (hasMore) {
-											//move to the next non-empty split
-											break;
-										}
-									}
-								}
-								if (!hasMore) {
-									return (numBytes == 0) ? -1 : numBytes;
-								}
-								//get the index of the first field name
-								int firstFieldLocation = value.find("\"");
-								int admValueSize = value.getLength();
-								if(firstFieldLocation >= 0)
-								{
-									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
-									int sizeOfNextTupleAndRID = fileNameFieldNameWithRecOpeningBraces.length + fileName.length + fileNameFieldClosingQuotation.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
-									if (numBytes + sizeOfNextTupleAndRID > len) {
-										// cannot add tuple to current buffer
-										// but the reader has moved pass the fetched tuple
-										// we need to store this for a subsequent read call.
-										// and return this then.
-										pendingValue = value;
-										break;
-									} else {
-										//copy fileNameFieldNameWithRecOpeningBraces
-										System.arraycopy(fileNameFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNameFieldNameWithRecOpeningBraces.length);
-										numBytes += fileNameFieldNameWithRecOpeningBraces.length;
-										//copy fileName
-										System.arraycopy(fileName, 0, buffer, offset + numBytes,fileName.length);
-										numBytes += fileName.length;
-										//copy fileName closing quotation
-										System.arraycopy(fileNameFieldClosingQuotation, 0, buffer, offset + numBytes,fileNameFieldClosingQuotation.length);
-										numBytes += fileNameFieldClosingQuotation.length;
-										//copy bytelocationFieldName
-										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
-										numBytes += bytelocationFieldName.length;
-										//copy byte location value
-										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
-										numBytes += byteLocation.length;
-										//copy byte location field end 
-										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
-										numBytes += bytelocationValueEnd.length;
-										//copy the actual adm instance
-										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
-										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
-										numBytes += admValueSize - firstFieldLocation +1;
-									}
-								}
-							}
-							return numBytes;
-						}
-
-						@Override
-						public int read() throws IOException {
-							throw new NotImplementedException("Use read(byte[], int, int");
-						}
-
-						private RecordReader getRecordReader(int slitIndex) throws IOException {
-							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							} else {
-								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							}
-						}
-
-					};
-				}
-				else
-				{
-					throw new IOException("Can't index " +configuration.get(KEY_FORMAT)+" input");
-				}
-			}
-		}
-		else
-		{
-			if(conf.getInputFormat() instanceof RCFileInputFormat)
-			{
-				//indexing rc input format
-				return new InputStream() {
-
-					private RecordReader<LongWritable, BytesRefArrayWritable> reader;
-					private LongWritable key;
-					private BytesRefArrayWritable value;
-					private boolean hasMore = false;
-					private int EOL = "\n".getBytes()[0];
-					private byte delimiter = 0x01;
-					private boolean pendingValue = false;
-					private int currentSplitIndex = 0;
-					private byte[] fileNumber;
-					private byte[] byteLocation;
-					private byte[] rowNumberBytes;
-					private Integer file;
-					private long blockByteLocation;
-					private long NextblockByteLocation;
-					private int rowNumber;
-
-					@SuppressWarnings("unchecked")
-					private boolean moveToNext() throws IOException {
-						for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-							/**
-							 * read all the partitions scheduled to the current node
-							 */
-							if (readSchedule[currentSplitIndex].equals(nodeName)) {
-								/**
-								 * pick an unread split to read
-								 * synchronize among simultaneous partitions in the same machine
-								 */
-								synchronized (executed) {
-									if (executed[currentSplitIndex] == false) {
-										executed[currentSplitIndex] = true;
-									} else {
-										continue;
-									}
-								}
-
-								/**
-								 * read the split
-								 */
-								reader = getRecordReader(currentSplitIndex);
-								key = reader.createKey();
-								value = reader.createValue();
-								//getting the file number
-								file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
-								if(file == null)
-								{
-									throw new HyracksException("a file was not found in the map while indexing");
-								}
-								fileNumber = String.valueOf(file).getBytes("UTF-8");
-								blockByteLocation = reader.getPos();
-								pendingValue = reader.next(key, value);
-								NextblockByteLocation = reader.getPos();
-								rowNumber = 1;
-								byteLocation = String.valueOf(blockByteLocation).getBytes("UTF-8");
-								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-								return true;
-							}
-						}
-						return false;
-					}
-
-					@Override
-					public int read(byte[] buffer, int offset, int len) throws IOException {
-						if (reader == null) {
-							if (!moveToNext()) {
-								//nothing to read
-								return -1;
-							}
-						}
-
-						int numBytes = 0;
-						if (pendingValue) {
-							//last value didn't fit into buffer
-							// 1 for EOL
-							int sizeOfNextTuple = getTupleSize(value) + 1;
-							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileNumber.length + 3 > len) {
-								return 0;
-							}
-							//copy file number
-							System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
-							buffer[offset + numBytes + fileNumber.length] = delimiter;
-							numBytes += fileNumber.length + 1;
-
-							//copy byte location
-							System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-							buffer[offset + numBytes + byteLocation.length] = delimiter;
-							numBytes += byteLocation.length + 1;
-
-							//copy row number
-							System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
-							buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
-							numBytes += rowNumberBytes.length + 1;
-
-							copyCurrentTuple(buffer, offset + numBytes);
-							buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-							numBytes += sizeOfNextTuple;
-							//set pending to false
-							pendingValue = false;
-						}
-
-						while (numBytes < len) {
-							hasMore = reader.next(key, value);
-							if (!hasMore) {
-								while (moveToNext()) {
-									hasMore = reader.next(key, value);
-									if (hasMore) {
-										//move to the next non-empty split
-										break;
-									}
-								}
-							}
-							if (!hasMore) {
-								return (numBytes == 0) ? -1 : numBytes;
-							}
-
-							//check if moved to next block
-							blockByteLocation = reader.getPos();
-							if(blockByteLocation != NextblockByteLocation)
-							{
-								//moved to a new block, reset stuff
-								//row number
-								rowNumber = 1;
-								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-
-								//block location
-								byteLocation = String.valueOf(NextblockByteLocation).getBytes("UTF-8");
-								NextblockByteLocation = blockByteLocation;
-							}
-							else
-							{
-								rowNumber += 1;
-								rowNumberBytes = String.valueOf(rowNumber).getBytes("UTF-8");
-							}
-
-							int sizeOfNextTuple = getTupleSize(value) + 1;
-							if (numBytes + sizeOfNextTuple +  rowNumberBytes.length + byteLocation.length + fileNumber.length + 3 > len) {
-								// cannot add tuple to current buffer
-								// but the reader has moved pass the fetched tuple
-								// we need to store this for a subsequent read call.
-								// and return this then.
-								pendingValue = true;
-								break;
-							} else {
-								//copy file number
-								System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
-								buffer[offset + numBytes + fileNumber.length] = delimiter;
-								numBytes += fileNumber.length + 1;
-
-								//copy byte location
-								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-								buffer[offset + numBytes + byteLocation.length] = delimiter;
-								numBytes += byteLocation.length + 1;
-
-								//copy row number
-								System.arraycopy(rowNumberBytes, 0, buffer, offset + numBytes, rowNumberBytes.length);
-								buffer[offset + numBytes + rowNumberBytes.length] = delimiter;
-								numBytes += rowNumberBytes.length + 1;
-
-								copyCurrentTuple(buffer, offset + numBytes);
-								buffer[offset + numBytes + sizeOfNextTuple - 1] = (byte) EOL;
-								numBytes += sizeOfNextTuple;
-							}
-						}
-						return numBytes;
-					}
-
-					private void copyCurrentTuple(byte[] buffer, int offset) throws IOException {
-						int rcOffset = 0;
-						for(int i=0; i< value.size(); i++)
-						{
-							System.arraycopy(value.get(i).getData(), value.get(i).getStart(), buffer, offset + rcOffset, value.get(i).getLength());
-							rcOffset += value.get(i).getLength() + 1;
-							buffer[rcOffset - 1] = delimiter;
-						}
-					}
-
-					private int getTupleSize(BytesRefArrayWritable value2) {
-						int size=0;
-						//loop over rc column and add lengths
-						for(int i=0; i< value.size(); i++)
-						{
-							size += value.get(i).getLength();
-						}
-						//add delimeters bytes sizes
-						size += value.size() -1;
-						return size;
-					}
-
-					@Override
-					public int read() throws IOException {
-						throw new NotImplementedException("Use read(byte[], int, int");
-					}
-
-					private RecordReader getRecordReader(int slitIndex) throws IOException {
-						RCFileInputFormat format = (RCFileInputFormat) conf.getInputFormat();
-						RecordReader reader = format.getRecordReader(
-								(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-						return reader;
-					}
-
-				};
-			}
-			else
-			{
-				//get content format
-				if(configuration.get(KEY_FORMAT).equals(FORMAT_DELIMITED_TEXT))
-				{
-					//reading data and RIDs for delimited text
-					return new InputStream() {
-
-						private RecordReader<Object, Text> reader;
-						private Object key;
-						private Text value;
-						private boolean hasMore = false;
-						private int EOL = "\n".getBytes()[0];
-						private Text pendingValue = null;
-						private int currentSplitIndex = 0;
-						private Integer file;
-						private byte[] fileNumber;
-						private byte[] byteLocation;
-						private byte delimiter = ((String)configuration.get(KEY_DELIMITER)).getBytes()[0];
-
-						@SuppressWarnings("unchecked")
-						private boolean moveToNext() throws IOException {
-							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-								/**
-								 * read all the partitions scheduled to the current node
-								 */
-								if (readSchedule[currentSplitIndex].equals(nodeName)) {
-									/**
-									 * pick an unread split to read
-									 * synchronize among simultaneous partitions in the same machine
-									 */
-									synchronized (executed) {
-										if (executed[currentSplitIndex] == false) {
-											executed[currentSplitIndex] = true;
-										} else {
-											continue;
-										}
-									}
-
-									/**
-									 * read the split
-									 */
-									reader = getRecordReader(currentSplitIndex);
-									key = reader.createKey();
-									value = (Text) reader.createValue();
-									file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
-									if(file == null)
-									{
-										throw new HyracksException("The file:"+((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath()+" was not found in the map while indexing");
-									}
-									fileNumber = String.valueOf(file).getBytes("UTF-8");
-									return true;
-								}
-							}
-							return false;
-						}
-
-						@Override
-						public int read(byte[] buffer, int offset, int len) throws IOException {
-							if (reader == null) {
-								if (!moveToNext()) {
-									//nothing to read
-									return -1;
-								}
-							}
-
-							int numBytes = 0;
-							if (pendingValue != null) {
-								int sizeOfNextTuple = pendingValue.getLength() + 1;
-								if (numBytes + sizeOfNextTuple +byteLocation.length + fileNumber.length + 2> len)
-								{
-									return numBytes;
-								}
-								//copy file number
-								System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
-								buffer[offset + numBytes + fileNumber.length] = delimiter;
-								numBytes += fileNumber.length + 1;
-
-								//copy byte location
-								System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-								buffer[offset + numBytes + byteLocation.length] = delimiter;
-								numBytes += byteLocation.length + 1;
-
-								//copy actual value
-								System.arraycopy(pendingValue.getBytes(), 0, buffer, offset + numBytes, pendingValue.getLength());
-								buffer[offset + numBytes + pendingValue.getLength()] = (byte) EOL;
-								numBytes += pendingValue.getLength() + 1;
-								pendingValue = null;
-							}
-
-							while (numBytes < len) {
-								//get reader position before you actually read
-								byteLocation = String.valueOf(reader.getPos()).getBytes();
-								hasMore = reader.next(key, value);
-								if (!hasMore) {
-									while (moveToNext()) {
-										//get reader position before you actually read
-										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
-										hasMore = reader.next(key, value);
-										if (hasMore) {
-											//move to the next non-empty split
-											break;
-										}
-									}
-								}
-								if (!hasMore) {
-									return (numBytes == 0) ? -1 : numBytes;
-								}
-								int sizeOfNextTuple = value.getLength() + 1;
-								if (numBytes + sizeOfNextTuple +byteLocation.length + fileNumber.length + 2> len) {
-									// cannot add tuple to current buffer
-									// but the reader has moved pass the fetched tuple
-									// we need to store this for a subsequent read call.
-									// and return this then.
-									pendingValue = value;
-									break;
-								} else {
-									//copy file number
-									System.arraycopy(fileNumber, 0, buffer, offset + numBytes, fileNumber.length);
-									buffer[offset + numBytes + fileNumber.length] = delimiter;
-									numBytes += fileNumber.length + 1;
-
-									//copy byte location
-									System.arraycopy(byteLocation, 0, buffer, offset + numBytes, byteLocation.length);
-									buffer[offset + numBytes + byteLocation.length] = delimiter;
-									numBytes += byteLocation.length + 1;
-
-									//Copy actual value
-									System.arraycopy(value.getBytes(), 0, buffer, offset + numBytes, value.getLength());
-									buffer[offset + numBytes + value.getLength()] = (byte) EOL;
-									numBytes += sizeOfNextTuple;
-								}
-							}
-							return numBytes;
-						}
-
-						@Override
-						public int read() throws IOException {
-							throw new NotImplementedException("Use read(byte[], int, int");
-						}
-
-						private RecordReader getRecordReader(int slitIndex) throws IOException {
-							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							} else {
-								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							}
-						}
-
-					};
-				}
-				else if((configuration.get(KEY_FORMAT).equals(FORMAT_ADM)))
-				{
-					//reading data and RIDs for adm formatted data
-					return new InputStream() {
-
-						private RecordReader<Object, Text> reader;
-						private Object key;
-						private Text value;
-						private boolean hasMore = false;
-						private int EOL = "\n".getBytes()[0];
-						private Text pendingValue = null;
-						private int currentSplitIndex = 0;
-						private Integer file;
-						private byte[] fileNumber;
-						private byte[] byteLocation;
-
-						@SuppressWarnings("unchecked")
-						private boolean moveToNext() throws IOException {
-							for (; currentSplitIndex < inputSplits.length; currentSplitIndex++) {
-								/**
-								 * read all the partitions scheduled to the current node
-								 */
-								if (readSchedule[currentSplitIndex].equals(nodeName)) {
-									/**
-									 * pick an unread split to read
-									 * synchronize among simultaneous partitions in the same machine
-									 */
-									synchronized (executed) {
-										if (executed[currentSplitIndex] == false) {
-											executed[currentSplitIndex] = true;
-										} else {
-											continue;
-										}
-									}
-
-									/**
-									 * read the split
-									 */
-									reader = getRecordReader(currentSplitIndex);
-									key = reader.createKey();
-									value = (Text) reader.createValue();
-									file = files.get(((FileSplit)(inputSplits[currentSplitIndex])).getPath().toUri().getPath());
-									if(file == null)
-									{
-										throw new HyracksException("a file was not found in the map while indexing");
-									}
-									fileNumber = String.valueOf(file).getBytes("UTF-8");
-									return true;
-								}
-							}
-							return false;
-						}
-
-						@Override
-						public int read(byte[] buffer, int offset, int len) throws IOException {
-							if (reader == null) {
-								if (!moveToNext()) {
-									//nothing to read
-									return -1;
-								}
-							}
-
-							int numBytes = 0;
-							if (pendingValue != null) {
-								int firstFieldLocation = value.find("\"");
-								int admValueSize = value.getLength();
-								if(firstFieldLocation >= 0)
-								{
-									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
-									int sizeOfNextTupleAndRID = fileNumberFieldNameWithRecOpeningBraces.length + fileNumber.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
-									if (numBytes + sizeOfNextTupleAndRID > len) {
-										// still cannot add tuple to current buffer
-										// return 0 so parser would double the buffer size.
-										return 0;
-									} else {
-										//copy fileNumberFieldNameWithRecOpeningBraces
-										System.arraycopy(fileNumberFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNumberFieldNameWithRecOpeningBraces.length);
-										numBytes += fileNumberFieldNameWithRecOpeningBraces.length;
-										//copy file Number
-										System.arraycopy(fileNumber, 0, buffer, offset + numBytes,fileNumber.length);
-										numBytes += fileNumber.length;
-										//copy bytelocationFieldName
-										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
-										numBytes += bytelocationFieldName.length;
-										//copy byte location value
-										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
-										numBytes += byteLocation.length;
-										//copy byte location field end 
-										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
-										numBytes += bytelocationValueEnd.length;
-										//copy the actual adm instance
-										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
-										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
-										numBytes += admValueSize - firstFieldLocation +1;
-									}
-								}
-								pendingValue = null;
-							}
-
-							while (numBytes < len) {
-								//get reader position before you actually read
-								byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
-								hasMore = reader.next(key, value);
-								if (!hasMore) {
-									while (moveToNext()) {
-										//get reader position before you actually read
-										byteLocation = String.valueOf(reader.getPos()).getBytes("UTF-8");
-										hasMore = reader.next(key, value);
-										if (hasMore) {
-											//move to the next non-empty split
-											break;
-										}
-									}
-								}
-								if (!hasMore) {
-									return (numBytes == 0) ? -1 : numBytes;
-								}
-								//get the index of the first field name
-								int firstFieldLocation = value.find("\"");
-								int admValueSize = value.getLength();
-								if(firstFieldLocation >= 0)
-								{
-									int sizeOfNextTuple = value.getLength() - firstFieldLocation + 1;
-									int sizeOfNextTupleAndRID = fileNumberFieldNameWithRecOpeningBraces.length + fileNumber.length + bytelocationFieldName.length  + byteLocation.length + bytelocationValueEnd.length + sizeOfNextTuple;
-									if (numBytes + sizeOfNextTupleAndRID > len) {
-										// cannot add tuple to current buffer
-										// but the reader has moved pass the fetched tuple
-										// we need to store this for a subsequent read call.
-										// and return this then.
-										pendingValue = value;
-										break;
-									} else {
-										//copy fileNumberFieldNameWithRecOpeningBraces
-										System.arraycopy(fileNumberFieldNameWithRecOpeningBraces, 0, buffer, offset + numBytes,fileNumberFieldNameWithRecOpeningBraces.length);
-										numBytes += fileNumberFieldNameWithRecOpeningBraces.length;
-										//copy fileNumber
-										System.arraycopy(fileNumber, 0, buffer, offset + numBytes,fileNumber.length);
-										numBytes += fileNumber.length;
-										//copy bytelocationFieldName
-										System.arraycopy(bytelocationFieldName, 0, buffer, offset + numBytes,bytelocationFieldName.length);
-										numBytes += bytelocationFieldName.length;
-										//copy byte location value
-										System.arraycopy(byteLocation, 0, buffer, offset + numBytes,byteLocation.length);
-										numBytes += byteLocation.length;
-										//copy byte location field end 
-										System.arraycopy(bytelocationValueEnd, 0, buffer, offset + numBytes,bytelocationValueEnd.length);
-										numBytes += bytelocationValueEnd.length;
-										//copy the actual adm instance
-										System.arraycopy(value.getBytes(), firstFieldLocation, buffer, offset + numBytes,admValueSize - firstFieldLocation);
-										buffer[offset + numBytes + admValueSize - firstFieldLocation] = (byte) EOL;
-										numBytes += admValueSize - firstFieldLocation +1;
-									}
-								}
-							}
-							return numBytes;
-						}
-
-						@Override
-						public int read() throws IOException {
-							throw new NotImplementedException("Use read(byte[], int, int");
-						}
-
-						private RecordReader getRecordReader(int slitIndex) throws IOException {
-							if (conf.getInputFormat() instanceof SequenceFileInputFormat) {
-								SequenceFileInputFormat format = (SequenceFileInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							} else {
-								TextInputFormat format = (TextInputFormat) conf.getInputFormat();
-								RecordReader reader = format.getRecordReader(
-										(org.apache.hadoop.mapred.FileSplit) inputSplits[slitIndex], conf, getReporter());
-								return reader;
-							}
-						}
-
-					};
-				}
-				else
-				{
-					throw new IOException("Can't index " +configuration.get(KEY_FORMAT)+" input");
-				}
-			}
-		}
-	}
-
-	@Override
-	public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-		return clusterLocations;
-	}
-}
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
deleted file mode 100644
index 178b106..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/HiveIndexingAdapter.java
+++ /dev/null
@@ -1,62 +0,0 @@
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.util.Map;
-
-import org.apache.hadoop.mapred.InputSplit;
-import org.apache.hadoop.mapred.JobConf;
-
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-
-/**
- * Provides the functionality of fetching data in form of ADM records from a Hive dataset.
- */
-@SuppressWarnings("deprecation")
-public class HiveIndexingAdapter extends AbstractDatasourceAdapter{
-
-    private static final long serialVersionUID = 1L;
-
-    public static final String HIVE_DATABASE = "database";
-    public static final String HIVE_TABLE = "table";
-    public static final String HIVE_HOME = "hive-home";
-    public static final String HIVE_METASTORE_URI = "metastore-uri";
-    public static final String HIVE_WAREHOUSE_DIR = "warehouse-dir";
-    public static final String HIVE_METASTORE_RAWSTORE_IMPL = "rawstore-impl";
-
-    private HDFSIndexingAdapter hdfsIndexingAdapter;
-
-    public HiveIndexingAdapter(IAType atype, String[] readSchedule, boolean[] executed, InputSplit[] inputSplits, JobConf conf,
-            AlgebricksPartitionConstraint clusterLocations, Map<String,Integer> files) {
-        this.hdfsIndexingAdapter = new HDFSIndexingAdapter(atype, readSchedule, executed, inputSplits, conf, clusterLocations, files);
-        this.atype = atype;
-    }
-
-    @Override
-    public AdapterType getAdapterType() {
-        return AdapterType.READ;
-    }
-
-    @Override
-    public void configure(Map<String, Object> arguments) throws Exception {
-        this.configuration = arguments;
-        this.hdfsIndexingAdapter.configure(arguments);
-    }
-
-    @Override
-    public void initialize(IHyracksTaskContext ctx) throws Exception {
-    	hdfsIndexingAdapter.initialize(ctx);
-    }
-
-    @Override
-    public void start(int partition, IFrameWriter writer) throws Exception {
-    	hdfsIndexingAdapter.start(partition, writer);
-    }
-
-    @Override
-    public AlgebricksPartitionConstraint getPartitionConstraint() throws Exception {
-        return hdfsIndexingAdapter.getPartitionConstraint();
-    }
-
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
deleted file mode 100644
index 253f675..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/dataset/adapter/IControlledAdapter.java
+++ /dev/null
@@ -1,17 +0,0 @@
-package edu.uci.ics.asterix.external.dataset.adapter;
-
-import java.io.Serializable;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-public interface IControlledAdapter extends Serializable{
-
-	public void initialize(IHyracksTaskContext ctx) throws Exception;
-	
-	public void processNextFrame(ByteBuffer buffer, IFrameWriter writer) throws HyracksDataException;
-	
-	public void close(IFrameWriter writer) throws HyracksDataException;
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java
deleted file mode 100644
index 47550a4..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalDataFilesMetadataProvider.java
+++ /dev/null
@@ -1,38 +0,0 @@
-package edu.uci.ics.asterix.external.util;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.StringTokenizer;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
-import edu.uci.ics.asterix.external.dataset.adapter.AbstractDatasourceAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.HDFSAdapter;
-
-public class ExternalDataFilesMetadataProvider {
-	public static ArrayList<FileStatus> getHDFSFileStatus(AbstractDatasourceAdapter adapter) throws IOException
-	{
-		ArrayList<FileStatus> files = new ArrayList<FileStatus>();
-		//Configure hadoop connection
-		Configuration conf = HDFSAdapterFactory.configureHadoopConnection(adapter.getConfiguration());
-		FileSystem fs = FileSystem.get(conf);
-		//get the list of paths from the adapter
-		StringTokenizer tokenizer = new StringTokenizer(((String)adapter.getConfiguration().get(HDFSAdapter.KEY_PATH)),",");
-		Path inputPath = null;
-		FileStatus[] fileStatuses;
-		while(tokenizer.hasMoreTokens())
-		{
-			inputPath = new Path(tokenizer.nextToken().trim());
-			fileStatuses = fs.listStatus(inputPath);
-			for(int i=0; i < fileStatuses.length; i++)
-			{
-				files.add(fileStatuses[i]);
-			}
-		}
-		return files;
-	}
-}
\ No newline at end of file
diff --git a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java b/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
deleted file mode 100644
index 29fcfb1..0000000
--- a/asterix-external-data/src/main/java/edu/uci/ics/asterix/external/util/ExternalIndexHashPartitionComputerFactory.java
+++ /dev/null
@@ -1,94 +0,0 @@
-package edu.uci.ics.asterix.external.util;
-
-import java.io.DataInputStream;
-import java.nio.ByteBuffer;
-
-import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.AInt64;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.hyracks.api.comm.IFrameTupleAccessor;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputer;
-import edu.uci.ics.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
-
-/* A class that is used to partition external data tuples when building an index over them
- * the computer it returns, computes the HDFS block value before using the actual hash partitioning 
- * function. this way we ensure that records within ranges of 64MB sizes are partitioned together to the same
- * data node.
- */
-
-public class ExternalIndexHashPartitionComputerFactory implements ITuplePartitionComputerFactory{
-	private static final long serialVersionUID = 1L;
-	private final int[] hashFields;
-	private final int bytesInHDFSBlock = 67108864;
-	private final IBinaryHashFunctionFactory[] hashFunctionFactories;
-	@SuppressWarnings("unchecked")
-	private final ISerializerDeserializer<AInt64> longSerde = AqlSerializerDeserializerProvider.INSTANCE.getNonTaggedSerializerDeserializer(BuiltinType.AINT64);
-
-	public ExternalIndexHashPartitionComputerFactory(int[] hashFields, IBinaryHashFunctionFactory[] hashFunctionFactories) {
-		this.hashFields = hashFields;
-		this.hashFunctionFactories = hashFunctionFactories;
-	}
-
-	@Override
-	public ITuplePartitionComputer createPartitioner() {
-		final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length];
-		for (int i = 0; i < hashFunctionFactories.length; ++i) {
-			hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction();
-		}
-		return new ITuplePartitionComputer() {
-			private ByteBuffer serializedLong = ByteBuffer.allocate(8);;
-			private AInt64 byteLocation;
-			private ByteBufferInputStream bbis = new ByteBufferInputStream();
-			private DataInputStream dis = new DataInputStream(bbis);
-			@Override
-			public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts){
-				if (nParts == 1) {
-					return 0;
-				}
-				int h = 0;
-				int startOffset = accessor.getTupleStartOffset(tIndex);
-				int slotLength = accessor.getFieldSlotsLength();
-				for (int j = 0; j < hashFields.length; ++j) {
-					int fIdx = hashFields[j];
-					IBinaryHashFunction hashFn = hashFunctions[j];
-					int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
-					int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
-					if(j == 1)
-					{
-						//reset the buffer
-						serializedLong.clear();
-						//read byte location
-						bbis.setByteBuffer(accessor.getBuffer() , accessor.getTupleStartOffset(tIndex) + accessor.getFieldSlotsLength() + accessor.getFieldStartOffset(tIndex, hashFields[1]));
-						try {
-							byteLocation = ((AInt64) longSerde.deserialize(dis));
-							//compute the block number, serialize it into a new array and call the hash function
-							serializedLong.putLong(byteLocation.getLongValue()/bytesInHDFSBlock);
-							//call the hash function
-							int fh = hashFn
-									.hash(serializedLong.array(), 0,serializedLong.capacity());
-									h = h * 31 + fh;
-						} catch (HyracksDataException e) {
-							System.err.println("could not serialize the long byte position value!!!");
-							e.printStackTrace();
-						}
-					}
-					else
-					{
-						int fh = hashFn
-						.hash(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd - fStart);
-						h = h * 31 + fh;
-					}
-				}
-				if (h < 0) {
-					h = -(h + 1);
-				}
-				return h % nParts;
-			}
-		};
-	}
-}
\ No newline at end of file
diff --git a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
index 92d1c8e..dae1fb1 100644
--- a/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
+++ b/asterix-maven-plugins/lexer-generator-maven-plugin/src/main/resources/Lexer.java
@@ -19,235 +19,215 @@
 
 public class [LEXER_NAME] {
 
-	public static final int
-	TOKEN_EOF = 0, TOKEN_AUX_NOT_FOUND = 1 [TOKENS_CONSTANTS];
+    public static final int
+        TOKEN_EOF = 0, TOKEN_AUX_NOT_FOUND = 1 [TOKENS_CONSTANTS];
 
-	// Human representation of tokens. Useful for debug.
-	// Is possible to convert a TOKEN_CONSTANT in its image through
-	// [LEXER_NAME].tokenKindToString(TOKEN_CONSTANT); 
-	private static final String[] tokenImage = {
-		"<EOF>", "<AUX_NOT_FOUND>" [TOKENS_IMAGES]
-	};
+    // Human representation of tokens. Useful for debug.
+    // Is possible to convert a TOKEN_CONSTANT in its image through
+    // [LEXER_NAME].tokenKindToString(TOKEN_CONSTANT); 
+    private static final String[] tokenImage = {
+            "<EOF>", "<AUX_NOT_FOUND>" [TOKENS_IMAGES]
+          };
+    
+    private static final char EOF_CHAR = 4;
+    protected java.io.Reader inputStream;
+    protected int column;
+    protected int line;
+    protected boolean prevCharIsCR;
+    protected boolean prevCharIsLF;
+    protected char[] buffer;
+    protected int bufsize;
+    protected int bufpos;
+    protected int tokenBegin;
+    protected int endOf_USED_Buffer;
+    protected int endOf_UNUSED_Buffer;
+    protected int maxUnusedBufferSize;
 
-	private static final char EOF_CHAR = 4;
-	protected java.io.Reader inputStream;
-	protected int column;
-	protected int line;
-	protected boolean prevCharIsCR;
-	protected boolean prevCharIsLF;
-	protected char[] buffer;
-	protected int bufsize;
-	protected int bufpos;
-	protected int tokenBegin;
-	protected int endOf_USED_Buffer;
-	protected int endOf_UNUSED_Buffer;
-	protected int maxUnusedBufferSize;
+// ================================================================================
+//  Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
+// ================================================================================
 
-	// ================================================================================
-	//  Auxiliary functions. Can parse the tokens used in the grammar as partial/auxiliary
-	// ================================================================================
+    [LEXER_AUXFUNCTIONS]
 
-	[LEXER_AUXFUNCTIONS]
+// ================================================================================
+//  Main method. Return a TOKEN_CONSTANT
+// ================================================================================            
+            
+    public int next() throws [LEXER_NAME]Exception, IOException{
+        char currentChar = buffer[bufpos];
+        while (currentChar == ' ' || currentChar=='\t' || currentChar == '\n' || currentChar=='\r')
+            currentChar = readNextChar(); 
+        tokenBegin = bufpos;
+        if (currentChar==EOF_CHAR) return TOKEN_EOF;
 
-			// ================================================================================
-			//  Main method. Return a TOKEN_CONSTANT
-			// ================================================================================            
+        [LEXER_LOGIC]
+    }
 
-			public int next() throws [LEXER_NAME]Exception, IOException{
-		if(bufpos < 0)
-			readNextChar();
-		char currentChar = buffer[bufpos];
-		while (currentChar == ' ' || currentChar=='\t' || currentChar == '\n' || currentChar=='\r')
-			currentChar = readNextChar(); 
-		tokenBegin = bufpos;
-		if (currentChar==EOF_CHAR) return TOKEN_EOF;
+// ================================================================================
+//  Public interface
+// ================================================================================
+    
+    public [LEXER_NAME](java.io.Reader stream) throws IOException{
+        reInit(stream);
+    }
 
-		[LEXER_LOGIC]
-	}
+    public void reInit(java.io.Reader stream) throws IOException{
+        done();
+        inputStream    = stream;
+        bufsize        = 4096;
+        line           = 1;
+        column         = 0;
+        bufpos         = -1;
+        endOf_UNUSED_Buffer = bufsize;
+        endOf_USED_Buffer = 0;
+        prevCharIsCR   = false;
+        prevCharIsLF   = false;
+        buffer         = new char[bufsize];
+        tokenBegin     = -1;
+        maxUnusedBufferSize = 4096/2;
+        readNextChar();
+    }
 
-	//used when done with stream, must be called exiplicitly now.
-	public void close()throws IOException
-	{
-		inputStream.close();
-	}
+    public String getLastTokenImage() {
+        if (bufpos >= tokenBegin)
+            return new String(buffer, tokenBegin, bufpos - tokenBegin);
+          else
+            return new String(buffer, tokenBegin, bufsize - tokenBegin) +
+                                  new String(buffer, 0, bufpos);
+    }
+    
+    public static String tokenKindToString(int token) {
+        return tokenImage[token]; 
+    }
 
-	//used before processing a new patch in the inputStream
-	public void reset(){
-		line           = 1;
-		column         = 0;
-		bufpos         = -1;
-		endOf_UNUSED_Buffer = bufsize;
-		endOf_USED_Buffer = 0;
-		prevCharIsCR   = false;
-		prevCharIsLF   = false;
-		tokenBegin     = -1;
-	}
+    public void done(){
+        buffer = null;
+    }
 
-	// ================================================================================
-	//  Public interface
-	// ================================================================================
+// ================================================================================
+//  Parse error management
+// ================================================================================    
+    
+    protected int parseError(String reason) throws [LEXER_NAME]Exception {
+        StringBuilder message = new StringBuilder();
+        message.append(reason).append("\n");
+        message.append("Line: ").append(line).append("\n");
+        message.append("Row: ").append(column).append("\n");
+        throw new [LEXER_NAME]Exception(message.toString());
+    }
 
-	public [LEXER_NAME](java.io.Reader stream) throws IOException{
-		reInit(stream);
-	}
+    protected int parseError(int ... tokens) throws [LEXER_NAME]Exception {
+        StringBuilder message = new StringBuilder();
+        message.append("Error while parsing. ");
+        message.append(" Line: ").append(line);
+        message.append(" Row: ").append(column);
+        message.append(" Expecting:");
+        for (int tokenId : tokens){
+            message.append(" ").append([LEXER_NAME].tokenKindToString(tokenId));
+        }
+        throw new [LEXER_NAME]Exception(message.toString());
+    }
+    
+    protected void updateLineColumn(char c){
+        column++;
+    
+        if (prevCharIsLF)
+        {
+            prevCharIsLF = false;
+            line += (column = 1);
+        }
+        else if (prevCharIsCR)
+        {
+            prevCharIsCR = false;
+            if (c == '\n')
+            {
+                prevCharIsLF = true;
+            }
+            else
+            {
+                line += (column = 1);
+            }
+        }
+        
+        if (c=='\r') {
+            prevCharIsCR = true;
+        } else if(c == '\n') {
+            prevCharIsLF = true;
+        }
+    }
+    
+// ================================================================================
+//  Read data, buffer management. It uses a circular (and expandable) buffer
+// ================================================================================    
 
-	public void reInit(java.io.Reader stream) throws IOException{
-		done();
-		inputStream    = stream;
-		bufsize        = 4096;
-		line           = 1;
-		column         = 0;
-		bufpos         = -1;
-		endOf_UNUSED_Buffer = bufsize;
-		endOf_USED_Buffer = 0;
-		prevCharIsCR   = false;
-		prevCharIsLF   = false;
-		buffer         = new char[bufsize];
-		tokenBegin     = -1;
-		maxUnusedBufferSize = 4096/2;
-	}
+    protected char readNextChar() throws IOException {
+        if (++bufpos >= endOf_USED_Buffer)
+            fillBuff();
+        char c = buffer[bufpos];
+        updateLineColumn(c);
+        return c;
+    }
 
-	public String getLastTokenImage() {
-		if (bufpos >= tokenBegin)
-			return new String(buffer, tokenBegin, bufpos - tokenBegin);
-		else
-			return new String(buffer, tokenBegin, bufsize - tokenBegin) +
-					new String(buffer, 0, bufpos);
-	}
-
-	public static String tokenKindToString(int token) {
-		return tokenImage[token]; 
-	}
-
-	public void done(){
-		buffer = null;
-	}
-
-	// ================================================================================
-			//  Parse error management
-			// ================================================================================    
-
-	protected int parseError(String reason) throws [LEXER_NAME]Exception {
-		StringBuilder message = new StringBuilder();
-		message.append(reason).append("\n");
-		message.append("Line: ").append(line).append("\n");
-		message.append("Row: ").append(column).append("\n");
-		throw new [LEXER_NAME]Exception(message.toString());
-	}
-
-	protected int parseError(int ... tokens) throws [LEXER_NAME]Exception {
-		StringBuilder message = new StringBuilder();
-		message.append("Error while parsing. ");
-		message.append(" Line: ").append(line);
-		message.append(" Row: ").append(column);
-		message.append(" Expecting:");
-		for (int tokenId : tokens){
-			message.append(" ").append([LEXER_NAME].tokenKindToString(tokenId));
-		}
-		throw new [LEXER_NAME]Exception(message.toString());
-	}
-
-	protected void updateLineColumn(char c){
-		column++;
-
-		if (prevCharIsLF)
-		{
-			prevCharIsLF = false;
-			line += (column = 1);
-		}
-		else if (prevCharIsCR)
-		{
-			prevCharIsCR = false;
-			if (c == '\n')
-			{
-				prevCharIsLF = true;
-			}
-			else
-			{
-				line += (column = 1);
-			}
-		}
-
-		if (c=='\r') {
-			prevCharIsCR = true;
-		} else if(c == '\n') {
-			prevCharIsLF = true;
-		}
-	}
-
-	// ================================================================================
-	//  Read data, buffer management. It uses a circular (and expandable) buffer
-	// ================================================================================    
-
-	protected char readNextChar() throws IOException {
-		if (++bufpos >= endOf_USED_Buffer)
-			fillBuff();
-		char c = buffer[bufpos];
-		updateLineColumn(c);
-		return c;
-	}
-
-	protected boolean fillBuff() throws IOException {
-		if (endOf_UNUSED_Buffer == endOf_USED_Buffer) // If no more unused buffer space 
-		{
-			if (endOf_UNUSED_Buffer == bufsize)         // -- If the previous unused space was
-			{                                           // -- at the end of the buffer
-				if (tokenBegin > maxUnusedBufferSize)     // -- -- If the first N bytes before
-				{                                         //       the current token are enough
-					bufpos = endOf_USED_Buffer = 0;         // -- -- -- setup buffer to use that fragment 
-					endOf_UNUSED_Buffer = tokenBegin;
-				}
-				else if (tokenBegin < 0)                  // -- -- If no token yet
-					bufpos = endOf_USED_Buffer = 0;         // -- -- -- reuse the whole buffer
-				else
-					ExpandBuff(false);                      // -- -- Otherwise expand buffer after its end
-			}
-			else if (endOf_UNUSED_Buffer > tokenBegin)  // If the endOf_UNUSED_Buffer is after the token
-				endOf_UNUSED_Buffer = bufsize;            // -- set endOf_UNUSED_Buffer to the end of the buffer
-			else if ((tokenBegin - endOf_UNUSED_Buffer) < maxUnusedBufferSize)
-			{                                           // If between endOf_UNUSED_Buffer and the token
-				ExpandBuff(true);                         // there is NOT enough space expand the buffer                          
-			}                                           // reorganizing it
-			else 
-				endOf_UNUSED_Buffer = tokenBegin;         // Otherwise there is enough space at the start
-		}                                             // so we set the buffer to use that fragment
-		int i;
-		if ((i = inputStream.read(buffer, endOf_USED_Buffer, endOf_UNUSED_Buffer - endOf_USED_Buffer)) == -1)
-		{
-			//moved outside
-			//inputStream.close();
-			buffer[endOf_USED_Buffer]=(char)EOF_CHAR;
-			endOf_USED_Buffer++;
-			return false;
-		}
-		else
-			endOf_USED_Buffer += i;
-		return true;
-	}
+    protected boolean fillBuff() throws IOException {
+        if (endOf_UNUSED_Buffer == endOf_USED_Buffer) // If no more unused buffer space 
+        {
+          if (endOf_UNUSED_Buffer == bufsize)         // -- If the previous unused space was
+          {                                           // -- at the end of the buffer
+            if (tokenBegin > maxUnusedBufferSize)     // -- -- If the first N bytes before
+            {                                         //       the current token are enough
+              bufpos = endOf_USED_Buffer = 0;         // -- -- -- setup buffer to use that fragment 
+              endOf_UNUSED_Buffer = tokenBegin;
+            }
+            else if (tokenBegin < 0)                  // -- -- If no token yet
+              bufpos = endOf_USED_Buffer = 0;         // -- -- -- reuse the whole buffer
+            else
+              ExpandBuff(false);                      // -- -- Otherwise expand buffer after its end
+          }
+          else if (endOf_UNUSED_Buffer > tokenBegin)  // If the endOf_UNUSED_Buffer is after the token
+            endOf_UNUSED_Buffer = bufsize;            // -- set endOf_UNUSED_Buffer to the end of the buffer
+          else if ((tokenBegin - endOf_UNUSED_Buffer) < maxUnusedBufferSize)
+          {                                           // If between endOf_UNUSED_Buffer and the token
+            ExpandBuff(true);                         // there is NOT enough space expand the buffer                          
+          }                                           // reorganizing it
+          else 
+            endOf_UNUSED_Buffer = tokenBegin;         // Otherwise there is enough space at the start
+        }                                             // so we set the buffer to use that fragment
+        int i;
+        if ((i = inputStream.read(buffer, endOf_USED_Buffer, endOf_UNUSED_Buffer - endOf_USED_Buffer)) == -1)
+        {
+            inputStream.close();
+            buffer[endOf_USED_Buffer]=(char)EOF_CHAR;
+            endOf_USED_Buffer++;
+            return false;
+        }
+            else
+                endOf_USED_Buffer += i;
+        return true;
+    }
 
 
-	protected void ExpandBuff(boolean wrapAround)
-	{
-		char[] newbuffer = new char[bufsize + maxUnusedBufferSize];
+    protected void ExpandBuff(boolean wrapAround)
+    {
+      char[] newbuffer = new char[bufsize + maxUnusedBufferSize];
 
-		try {
-			if (wrapAround) {
-				System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
-				System.arraycopy(buffer, 0, newbuffer, bufsize - tokenBegin, bufpos);
-				buffer = newbuffer;
-				endOf_USED_Buffer = (bufpos += (bufsize - tokenBegin));
-			}
-			else {
-				System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
-				buffer = newbuffer;
-				endOf_USED_Buffer = (bufpos -= tokenBegin);
-			}
-		} catch (Throwable t) {
-			throw new Error(t.getMessage());
-		}
+      try {
+        if (wrapAround) {
+          System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+          System.arraycopy(buffer, 0, newbuffer, bufsize - tokenBegin, bufpos);
+          buffer = newbuffer;
+          endOf_USED_Buffer = (bufpos += (bufsize - tokenBegin));
+        }
+        else {
+          System.arraycopy(buffer, tokenBegin, newbuffer, 0, bufsize - tokenBegin);
+          buffer = newbuffer;
+          endOf_USED_Buffer = (bufpos -= tokenBegin);
+        }
+      } catch (Throwable t) {
+          throw new Error(t.getMessage());
+      }
 
-		bufsize += maxUnusedBufferSize;
-		endOf_UNUSED_Buffer = bufsize;
-		tokenBegin = 0;
-	}    
+      bufsize += maxUnusedBufferSize;
+      endOf_UNUSED_Buffer = bufsize;
+      tokenBegin = 0;
+    }    
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index 2de5d78..8f0eedb 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -31,7 +31,6 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
@@ -269,39 +268,6 @@
         }
         return dataset;
     }
-    
-    @Override
-   	public List<ExternalFile> getDatasetExternalFiles(
-   			MetadataTransactionContext mdTxnCtx, Dataset dataset)
-   			throws MetadataException {
-       	List<ExternalFile> externalFiles;
-           try {
-           	externalFiles = metadataNode.getExternalDatasetFiles(mdTxnCtx.getJobId(), dataset);
-           } catch (RemoteException e) {
-               throw new MetadataException(e);
-           }
-           return externalFiles;
-   	}
-    
-    @Override
-	public void addExternalFile(MetadataTransactionContext mdTxnCtx,
-			ExternalFile externalFile) throws MetadataException {
-    	try {
-            metadataNode.addExternalDatasetFile(mdTxnCtx.getJobId(), externalFile);
-        } catch (RemoteException e) {
-            throw new MetadataException(e);
-        }
-	}
-    
-    @Override
-	public void dropExternalFile(MetadataTransactionContext mdTxnCtx,
-			ExternalFile externalFile) throws MetadataException {
-        try {
-            metadataNode.dropExternalFile(mdTxnCtx.getJobId(), externalFile.getDataverseName(), externalFile.getDatasetName(), externalFile.getFileNumber());
-        } catch (RemoteException e) {
-            throw new MetadataException(e);
-        }
-	}
 
     @Override
     public List<Index> getDatasetIndexes(MetadataTransactionContext ctx, String dataverseName, String datasetName)
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index 81ce4f8..ee0791b 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -41,8 +41,6 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -52,7 +50,6 @@
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DatasourceAdapterTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DatatypeTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.DataverseTupleTranslator;
-import edu.uci.ics.asterix.metadata.entitytupletranslators.ExternalFileTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.FunctionTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.IndexTupleTranslator;
 import edu.uci.ics.asterix.metadata.entitytupletranslators.NodeGroupTupleTranslator;
@@ -62,7 +59,6 @@
 import edu.uci.ics.asterix.metadata.valueextractors.MetadataEntityValueExtractor;
 import edu.uci.ics.asterix.metadata.valueextractors.NestedDatatypeNameValueExtractor;
 import edu.uci.ics.asterix.metadata.valueextractors.TupleCopyValueExtractor;
-import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AMutableString;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -119,14 +115,15 @@
 
     @Override
     public void commitTransaction(JobId jobId) throws RemoteException, ACIDException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         transactionSubsystem.getTransactionManager().commitTransaction(txnCtx, new DatasetId(-1), -1);
     }
 
     @Override
     public void abortTransaction(JobId jobId) throws RemoteException, ACIDException {
         try {
-            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId,
+                    false);
             transactionSubsystem.getTransactionManager().abortTransaction(txnCtx, new DatasetId(-1), -1);
         } catch (ACIDException e) {
             e.printStackTrace();
@@ -136,13 +133,13 @@
 
     @Override
     public void lock(JobId jobId, byte lockMode) throws ACIDException, RemoteException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         transactionSubsystem.getLockManager().lock(METADATA_DATASET_ID, -1, lockMode, txnCtx);
     }
 
     @Override
     public void unlock(JobId jobId) throws ACIDException, RemoteException {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         transactionSubsystem.getLockManager().unlock(METADATA_DATASET_ID, -1, txnCtx);
     }
 
@@ -173,56 +170,48 @@
                 Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
                         dataset.getDatasetName(), IndexType.BTREE, id.getPrimaryKey(), true, dataset.getPendingOp());
 
-				addIndex(jobId, primaryIndex);
-				ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
-						dataset.getDatasetName());
-				insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
-			}
-			else
-			{
-				//added for external data
-				ExternalDatasetDetails id = (ExternalDatasetDetails) dataset.getDatasetDetails();
-				ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
-						dataset.getDatasetName());
-				insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
-			}
-			// Add entry in datatype secondary index.
-			ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
-					dataset.getDatasetName());
-			insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
-		} catch (TreeIndexDuplicateKeyException e) {
-			throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
-					+ " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-	}
+                addIndex(jobId, primaryIndex);
+                ITupleReference nodeGroupTuple = createTuple(id.getNodeGroupName(), dataset.getDataverseName(),
+                        dataset.getDatasetName());
+                insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
+            }
+            // Add entry in datatype secondary index.
+            ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
+                    dataset.getDatasetName());
+            insertTupleIntoIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+        } catch (TreeIndexDuplicateKeyException e) {
+            throw new MetadataException("A dataset with this name " + dataset.getDatasetName()
+                    + " already exists in dataverse '" + dataset.getDataverseName() + "'.", e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
 
-	@Override
-	public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
-		try {
-			IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
-			ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
-			insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
-		} catch (TreeIndexDuplicateKeyException e) {
-			throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-	}
+    @Override
+    public void addIndex(JobId jobId, Index index) throws MetadataException, RemoteException {
+        try {
+            IndexTupleTranslator tupleWriter = new IndexTupleTranslator(true);
+            ITupleReference tuple = tupleWriter.getTupleFromMetadataEntity(index);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.INDEX_DATASET, tuple);
+        } catch (TreeIndexDuplicateKeyException e) {
+            throw new MetadataException("An index with name '" + index.getIndexName() + "' already exists.", e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
 
-	@Override
-	public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
-		try {
-			NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
-			ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
-			insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
-		} catch (TreeIndexDuplicateKeyException e) {
-			throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-	}
+    @Override
+    public void addNode(JobId jobId, Node node) throws MetadataException, RemoteException {
+        try {
+            NodeTupleTranslator tupleReaderWriter = new NodeTupleTranslator(true);
+            ITupleReference tuple = tupleReaderWriter.getTupleFromMetadataEntity(node);
+            insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.NODE_DATASET, tuple);
+        } catch (TreeIndexDuplicateKeyException e) {
+            throw new MetadataException("A node with name '" + node.getNodeName() + "' already exists.", e);
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
 
     @Override
     public void addNodeGroup(JobId jobId, NodeGroup nodeGroup) throws MetadataException, RemoteException {
@@ -285,7 +274,7 @@
 
         ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         txnCtx.setWriteTxn(true);
         txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                 metadataIndex.isPrimaryIndex());
@@ -298,7 +287,7 @@
 
     private IModificationOperationCallback createIndexModificationCallback(JobId jobId, long resourceId,
             IMetadataIndex metadataIndex, ILSMIndex lsmIndex, IndexOperation indexOp) throws Exception {
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
 
         if (metadataIndex.isPrimaryIndex()) {
             return new PrimaryIndexModificationOperationCallback(metadataIndex.getDatasetId().getId(),
@@ -396,60 +385,47 @@
                 // artifacts.
             }
 
-			// Delete entry from secondary index 'group'.
-			if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
-				InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
-				ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
-				// Searches the index for the tuple to be deleted. Acquires an S
-				// lock on the GROUPNAME_ON_DATASET_INDEX index.
-				try {
-					ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
-							MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
-					deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
-				} catch (TreeIndexException tie) {
-					// ignore this exception and continue deleting all relevant
-					// artifacts.
-				}
-			}
-			else
-			{
-				ExternalDatasetDetails id = (ExternalDatasetDetails) dataset.getDatasetDetails();
-				ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
-				// Searches the index for the tuple to be deleted. Acquires an S
-				// lock on the GROUPNAME_ON_DATASET_INDEX index.
-				try {
-					ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
-							MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
-					deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
-				} catch (TreeIndexException tie) {
-					// ignore this exception and continue deleting all relevant
-					// artifacts.
-				}
-			}
-			// Delete entry from secondary index 'type'.
-			ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getItemTypeName(), datasetName);
-			// Searches the index for the tuple to be deleted. Acquires an S
-			// lock on the DATATYPENAME_ON_DATASET_INDEX index.
-			try {
-				ITupleReference dataTypeTuple = getTupleToBeDeleted(jobId,
-						MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
-				deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
-			} catch (TreeIndexException tie) {
-				// ignore this exception and continue deleting all relevant
-				// artifacts.
-			}
+            // Delete entry from secondary index 'group'.
+            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+                InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
+                ITupleReference groupNameSearchKey = createTuple(id.getNodeGroupName(), dataverseName, datasetName);
+                // Searches the index for the tuple to be deleted. Acquires an S
+                // lock on the GROUPNAME_ON_DATASET_INDEX index.
+                try {
+                    ITupleReference groupNameTuple = getTupleToBeDeleted(jobId,
+                            MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameSearchKey);
+                    deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
+                } catch (TreeIndexException tie) {
+                    // ignore this exception and continue deleting all relevant
+                    // artifacts.
+                }
+            }
+            // Delete entry from secondary index 'type'.
+            ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getItemTypeName(), datasetName);
+            // Searches the index for the tuple to be deleted. Acquires an S
+            // lock on the DATATYPENAME_ON_DATASET_INDEX index.
+            try {
+                ITupleReference dataTypeTuple = getTupleToBeDeleted(jobId,
+                        MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
+                deleteTupleFromIndex(jobId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
+            } catch (TreeIndexException tie) {
+                // ignore this exception and continue deleting all relevant
+                // artifacts.
+            }
 
-			// Delete entry(s) from the 'indexes' dataset.
-			List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
-			if (datasetIndexes != null) {
-				for (Index index : datasetIndexes) {
-					dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
-				}
-			}
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-	}
+            // Delete entry(s) from the 'indexes' dataset.
+            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+                List<Index> datasetIndexes = getDatasetIndexes(jobId, dataverseName, datasetName);
+                if (datasetIndexes != null) {
+                    for (Index index : datasetIndexes) {
+                        dropIndex(jobId, dataverseName, datasetName, index.getIndexName());
+                    }
+                }
+            }
+        } catch (Exception e) {
+            throw new MetadataException(e);
+        }
+    }
 
     @Override
     public void dropIndex(JobId jobId, String dataverseName, String datasetName, String indexName)
@@ -606,7 +582,7 @@
                 lsmIndex, IndexOperation.DELETE);
         ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(modCallback, NoOpOperationCallback.INSTANCE);
 
-        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId, false);
         txnCtx.setWriteTxn(true);
         txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
                 metadataIndex.isPrimaryIndex());
@@ -1160,116 +1136,8 @@
         }
     }
 
-	@Override
-	public List<ExternalFile> getExternalDatasetFiles(JobId jobId,
-			Dataset dataset) throws MetadataException, RemoteException {
-		try {
-			ITupleReference searchKey = createTuple(dataset.getDataverseName(),dataset.getDatasetName());
-			ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
-			IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
-					tupleReaderWriter);
-			List<ExternalFile> results = new ArrayList<ExternalFile>();
-			searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
-			return results;
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-	}
-
-	@SuppressWarnings("unchecked")
-	public ITupleReference createExternalFileSearchTuple(String dataverseName, String datasetName, int fileNumber) throws HyracksDataException {
-		ISerializerDeserializer<AString> stringSerde = AqlSerializerDeserializerProvider.INSTANCE
-				.getSerializerDeserializer(BuiltinType.ASTRING);
-		ISerializerDeserializer<AInt32> intSerde = AqlSerializerDeserializerProvider.INSTANCE
-				.getSerializerDeserializer(BuiltinType.AINT32);
-
-		AMutableString aString = new AMutableString("");
-		ArrayTupleBuilder tupleBuilder = new ArrayTupleBuilder(3);
-
-		//dataverse field
-		aString.setValue(dataverseName);
-		stringSerde.serialize(aString, tupleBuilder.getDataOutput());
-		tupleBuilder.addFieldEndOffset();
-
-		//dataset field
-		aString.setValue(datasetName);
-		stringSerde.serialize(aString, tupleBuilder.getDataOutput());
-		tupleBuilder.addFieldEndOffset();
-
-		//file number field
-		intSerde.serialize(new AInt32(fileNumber), tupleBuilder.getDataOutput());
-		tupleBuilder.addFieldEndOffset();
-
-		ArrayTupleReference tuple = new ArrayTupleReference();
-		tuple.reset(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray());
-		return tuple;
-		}
-
-	public ExternalFile getExternalDatasetFile(JobId jobId,String dataverseName, String datasetName,
-			int fileNumber) throws MetadataException, RemoteException {
-		try {
-			//create the search key
-			ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
-			ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(false);
-			IValueExtractor<ExternalFile> valueExtractor = new MetadataEntityValueExtractor<ExternalFile>(
-					tupleReaderWriter);
-			List<ExternalFile> results = new ArrayList<ExternalFile>();
-			searchIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey, valueExtractor, results);
-			return results.get(0);
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-	}
-	
-	@Override
-	public void dropExternalFile(JobId jobId, String dataverseName,
-			String datasetName, int fileNumber) throws MetadataException,
-			RemoteException {
-		ExternalFile externalFile;
-		try {
-			externalFile = getExternalDatasetFile(jobId, dataverseName, datasetName,fileNumber);
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-		if (externalFile == null) {
-			throw new MetadataException("Cannot drop external file because it doesn't exist.");
-		}
-		try {
-			// Delete entry from the 'ExternalFile' dataset.
-			ITupleReference searchKey = createExternalFileSearchTuple(dataverseName, datasetName, fileNumber);
-			// Searches the index for the tuple to be deleted. Acquires an S
-			// lock on the 'ExternalFile' dataset.
-			ITupleReference datasetTuple = getTupleToBeDeleted(jobId,
-					MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, searchKey);
-			deleteTupleFromIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, datasetTuple);
-
-		} catch (TreeIndexException e) {
-			throw new MetadataException("Couldn't drop externalFile.", e);
-		} catch (Exception e) {
-			throw new MetadataException(e);
-		}
-	}
-	
-	@Override
-    public void addExternalDatasetFile(JobId jobId, ExternalFile externalFile)
-                    throws MetadataException, RemoteException {
-            try {
-                    // Insert into the 'externalFiles' dataset.
-                    ExternalFileTupleTranslator tupleReaderWriter = new ExternalFileTupleTranslator(true);
-                    ITupleReference externalFileTuple = tupleReaderWriter.getTupleFromMetadataEntity(externalFile);
-                    insertTupleIntoIndex(jobId, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET, externalFileTuple);
-            } catch (TreeIndexDuplicateKeyException e) {
-                    throw new MetadataException("An externalFile with this number " + externalFile.getFileNumber()
-                                    + " already exists in dataset '" + externalFile.getDatasetName() + "' in dataverse '"+externalFile.getDataverseName()+"'.", e);
-            } catch (Exception e) {
-                    throw new MetadataException(e);
-            }
+    @Override
+    public int getMostRecentDatasetId() throws MetadataException, RemoteException {
+        return DatasetIdFactory.getMostRecentDatasetId();
     }
-
-
-	@Override
-	public int getMostRecentDatasetId() throws MetadataException, RemoteException {
-		return DatasetIdFactory.getMostRecentDatasetId();
-	}
 }
-
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
index 53f72dd..22c5e46 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataManager.java
@@ -26,7 +26,6 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
@@ -440,36 +439,6 @@
      */
     public List<Function> getDataverseFunctions(MetadataTransactionContext ctx, String dataverseName)
             throws MetadataException;
-    
-    /**
-     * @param mdTxnCtx
-     *            MetadataTransactionContext of an active metadata transaction.
-     * @param externalFile
-     *            An instance of type ExternalFile that represents the external file being
-     *            added
-     * @throws MetadataException
-     */
-    public void addExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
-    
-    /**
-     * @param mdTxnCtx
-     *            MetadataTransactionContext of an active metadata transaction.
-     * @param dataset
-     *            An instance of type Dataset that represents the "external" dataset 
-     * @return A list of external files belonging to the dataset
-     * @throws MetadataException
-     */
-    public List<ExternalFile> getDatasetExternalFiles(MetadataTransactionContext mdTxnCtx, Dataset dataset) throws MetadataException;
-
-    /**
-     * @param mdTxnCtx
-     *            MetadataTransactionContext of an active metadata transaction.
-     * @param externalFile
-     *            An instance of type ExternalFile that represents the external file being
-     *            dropped
-     * @throws MetadataException
-     */
-    public void dropExternalFile(MetadataTransactionContext mdTxnCtx, ExternalFile externalFile) throws MetadataException;
 
     public void initializeDatasetIdFactory(MetadataTransactionContext ctx) throws MetadataException;
     
@@ -484,5 +453,4 @@
     public void releaseReadLatch();
 
 
-
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
index 206ef8a..d1e63e1 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IMetadataNode.java
@@ -28,7 +28,6 @@
 import edu.uci.ics.asterix.metadata.entities.DatasourceAdapter;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.Node;
@@ -472,45 +471,6 @@
      * @throws RemoteException
      */
     public void addAdapter(JobId jobId, DatasourceAdapter adapter) throws MetadataException, RemoteException;
-    
-    /**
-     * @param jobId
-     *            A globally unique id for an active metadata transaction.
-     * @param externalFile
-     * 			  An object representing the external file entity
-     * @throws MetadataException
-     *             for example, if the file already exists.
-     * @throws RemoteException
-     */
-	public void addExternalDatasetFile(JobId jobId, ExternalFile externalFile) throws MetadataException, RemoteException;
-	
-	/**
-     * @param jobId
-     *            A globally unique id for an active metadata transaction.
-     * @param dataset
-     *            A dataset the files belongs to.
-     * @throws MetadataException
-     * @throws RemoteException
-     */
-	public List<ExternalFile> getExternalDatasetFiles(JobId jobId, Dataset dataset
-			) throws MetadataException, RemoteException;
-	
-	/**
-     * Deletes an externalFile , acquiring local locks on behalf of the given
-     * transaction id.
-     * 
-     * @param jobId
-     *            A globally unique id for an active metadata transaction.
-     * @param dataverseName
-     *            dataverse asociated with the external dataset that owns the file to be deleted.
-     * @param datasetName
-     *            Name of dataset owning the file to be deleted.
-     * @param fileNumber
-     * 			  the id number for the file to be deleted
-     * @throws RemoteException
-     */
-    public void dropExternalFile(JobId jobId, String dataverseName, String datasetName, int fileNumber) throws MetadataException,
-            RemoteException;
 
     public void initializeDatasetIdFactory(JobId jobId) throws MetadataException, RemoteException;
     
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index ed89cfc..7be4e8e 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -60,7 +60,6 @@
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -117,7 +116,7 @@
                 MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
                 MetadataPrimaryIndexes.INDEX_DATASET, MetadataPrimaryIndexes.NODE_DATASET,
                 MetadataPrimaryIndexes.NODEGROUP_DATASET, MetadataPrimaryIndexes.FUNCTION_DATASET,
-                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET, MetadataPrimaryIndexes.EXTERNAL_FILE_DATASET };
+                MetadataPrimaryIndexes.DATASOURCE_ADAPTER_DATASET };
         secondaryIndexes = new IMetadataIndex[] { MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX,
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
@@ -333,14 +332,14 @@
         int[] bloomFilterKeyFields = index.getBloomFilterKeyFields();
         LSMBTree lsmBtree = null;
         long resourceID = -1;
-        AsterixRuntimeComponentsProvider rtcProvider = AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER;
         ILSMOperationTracker opTracker = index.isPrimaryIndex() ? runtimeContext.getLSMBTreeOperationTracker(index
                 .getDatasetId().getId()) : new BaseOperationTracker((DatasetLifecycleManager) indexLifecycleManager,
-                LSMBTreeIOOperationCallbackFactory.INSTANCE, index.getDatasetId().getId());
+                index.getDatasetId().getId());
         if (create) {
             lsmBtree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, bufferCache, fileMapProvider, typeTraits,
                     comparatorFactories, bloomFilterKeyFields, runtimeContext.getBloomFilterFalsePositiveRate(),
-                    runtimeContext.getLSMMergePolicy(), opTracker, runtimeContext.getLSMIOScheduler(), rtcProvider);
+                    runtimeContext.getLSMMergePolicy(), opTracker, runtimeContext.getLSMIOScheduler(),
+                    LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
             lsmBtree.create();
             resourceID = runtimeContext.getResourceIdFactory().createId();
             ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
@@ -359,7 +358,7 @@
                         typeTraits, comparatorFactories, bloomFilterKeyFields,
                         runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
                         opTracker, runtimeContext.getLSMIOScheduler(),
-                        AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER);
+                        LSMBTreeIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
                 indexLifecycleManager.register(resourceID, lsmBtree);
             }
         }
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 078b340d..4909e21 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -18,14 +18,11 @@
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.logging.Logger;
 
-import org.apache.hadoop.fs.FileStatus;
-
 import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
@@ -47,17 +44,13 @@
 import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
 import edu.uci.ics.asterix.external.adapter.factory.ITypedDatasetAdapterFactory;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataAccessByRIDOperatorDescriptor;
-import edu.uci.ics.asterix.external.data.operator.ExternalDataIndexingOperatorDescriptor;
 import edu.uci.ics.asterix.external.data.operator.ExternalDataScanOperatorDescriptor;
 import edu.uci.ics.asterix.external.data.operator.FeedIntakeOperatorDescriptor;
 import edu.uci.ics.asterix.external.data.operator.FeedMessageOperatorDescriptor;
-import edu.uci.ics.asterix.external.dataset.adapter.AbstractDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.ITypedDatasourceAdapter;
 import edu.uci.ics.asterix.external.feed.lifecycle.FeedId;
 import edu.uci.ics.asterix.external.feed.lifecycle.IFeedMessage;
-import edu.uci.ics.asterix.external.util.ExternalDataFilesMetadataProvider;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
@@ -71,7 +64,6 @@
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.ExternalFile;
 import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
@@ -168,7 +160,6 @@
     private boolean asyncResults;
     private ResultSetId resultSetId;
     private IResultSerializerFactoryProvider resultSerializerFactoryProvider;
-    private static boolean optimizeExternalIndexes = false;
 
     private final Dataverse defaultDataverse;
     private JobId jobId;
@@ -178,6 +169,22 @@
     private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
     private static Scheduler hdfsScheduler;
 
+    public String getPropertyValue(String propertyName) {
+        return config.get(propertyName);
+    }
+
+    public void setConfig(Map<String, String> config) {
+        this.config = config;
+    }
+
+    public Map<String, String[]> getAllStores() {
+        return stores;
+    }
+
+    public Map<String, String> getConfig() {
+        return config;
+    }
+
     public AqlMetadataProvider(Dataverse defaultDataverse) {
         this.defaultDataverse = defaultDataverse;
         this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
@@ -257,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 {
@@ -419,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,
@@ -730,111 +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.RUNTIME_PROVIDER;
-				BTreeSearchOperatorDescriptor btreeSearchOp = new BTreeSearchOperatorDescriptor(jobSpec, outputRecDesc,
-	                    appContext.getStorageManagerInterface(), appContext.getIndexLifecycleManagerProvider(), spPc.first,
-	                    typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
-	                    lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
-	                            new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider,
-	                            isSecondary ? new SecondaryIndexOperationTrackerProvider(
-	                                    LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId())
-	                                    : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
-	                            rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
-	                    searchCallbackFactory);
-				return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
-			} catch (MetadataException me) {
-				throw new AlgebricksException(me);
-			}
-		}
-		else
-		{
-			try {
-				Index primaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-						dataset.getDatasetName(), dataset.getDatasetName());
-				if (primaryIndex != null) {
-					isSecondary = !indexName.equals(primaryIndex.getIndexName());
-				}
-				int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
-				RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-				int numKeys = numPrimaryKeys;
-				int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
-				ITypeTraits[] typeTraits = null;
-				int[] bloomFilterKeyFields;
-				if (isSecondary) {
-					Index secondaryIndex = MetadataManager.INSTANCE.getIndex(mdTxnCtx, dataset.getDataverseName(),
-							dataset.getDatasetName(), indexName);
-					int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
-					numKeys += numSecondaryKeys;
-					keysStartIndex = outputVars.size() - numKeys;
-					typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys, typeEnv, context);
-					bloomFilterKeyFields = new int[numSecondaryKeys];
-					for (int i = 0; i < numSecondaryKeys; i++) {
-						bloomFilterKeyFields[i] = i;
-					}
-				} else {
-					typeTraits = JobGenHelper.variablesToTypeTraits(outputVars, keysStartIndex, numKeys + 1, typeEnv,
-							context);
-					bloomFilterKeyFields = new int[numPrimaryKeys];
-					for (int i = 0; i < numPrimaryKeys; i++) {
-						bloomFilterKeyFields[i] = i;
-					}
-				}
-				IBinaryComparatorFactory[] comparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
-						outputVars, keysStartIndex, numKeys, typeEnv, context);
+            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();
@@ -852,20 +579,18 @@
                     typeTraits, comparatorFactories, bloomFilterKeyFields, lowKeyFields, highKeyFields,
                     lowKeyInclusive, highKeyInclusive, new LSMBTreeDataflowHelperFactory(
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()), rtcProvider,
-                            isSecondary ? new SecondaryIndexOperationTrackerProvider(
-                                    LSMBTreeIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId())
+                            isSecondary ? new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId())
                                     : new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()), rtcProvider,
-                            rtcProvider, storageProperties.getBloomFilterFalsePositiveRate()), retainInput,
-                    searchCallbackFactory);
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
+                            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)
@@ -923,11 +648,11 @@
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            new SecondaryIndexOperationTrackerProvider(LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, proposeLinearizer(
-                                    nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
+                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
+                                    comparatorFactories.length), storageProperties.getBloomFilterFalsePositiveRate()),
+                    retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
 
         } catch (MetadataException me) {
@@ -1083,7 +808,7 @@
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE,
                             storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -1151,7 +876,7 @@
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
                             new PrimaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
                                     .getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory, true);
 
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
@@ -1345,9 +1070,9 @@
                     comparatorFactories, bloomFilterKeyFields, fieldPermutation, indexOp,
                     new LSMBTreeDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            new SecondaryIndexOperationTrackerProvider(LSMBTreeIOOperationCallbackFactory.INSTANCE,
-                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            LSMBTreeIOOperationCallbackFactory.INSTANCE, storageProperties
                                     .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory,
                     false);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
@@ -1473,10 +1198,9 @@
                     invListsTypeTraits, invListComparatorFactories, tokenizerFactory, fieldPermutation, indexOp,
                     new LSMInvertedIndexDataflowHelperFactory(new AsterixVirtualBufferCacheProvider(datasetId),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            new SecondaryIndexOperationTrackerProvider(
-                                    LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, dataset.getDatasetId()),
+                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, storageProperties
+                            LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, storageProperties
                                     .getBloomFilterFalsePositiveRate()), filterFactory, modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
                     splitsAndConstraint.second);
@@ -1568,12 +1292,11 @@
                             valueProviderFactories, RTreePolicyType.RTREE, primaryComparatorFactories,
                             new AsterixVirtualBufferCacheProvider(dataset.getDatasetId()),
                             AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            new SecondaryIndexOperationTrackerProvider(LSMRTreeIOOperationCallbackFactory.INSTANCE,
-                                    dataset.getDatasetId()), AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
-                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER, proposeLinearizer(
-                                    nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
-                    modificationCallbackFactory, false);
+                            new SecondaryIndexOperationTrackerProvider(dataset.getDatasetId()),
+                            AsterixRuntimeComponentsProvider.RUNTIME_PROVIDER,
+                            LSMRTreeIOOperationCallbackFactory.INSTANCE, proposeLinearizer(nestedKeyType.getTypeTag(),
+                                    comparatorFactories.length), storageProperties.getBloomFilterFalsePositiveRate()),
+                    filterFactory, modificationCallbackFactory, false);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
         } catch (MetadataException | IOException e) {
             throw new AlgebricksException(e);
@@ -1621,20 +1344,9 @@
         }
 
         int numPartitions = 0;
-        List<String> nodeGroup = null;
-        if(dataset.getDatasetType() == DatasetType.EXTERNAL)
-		{
-			ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-			nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
-					.getNodeNames();
-		}
-		else
-		{
-			InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
-			nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
-					.getNodeNames();
-		}
-
+        InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+        List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
+                .getNodeNames();
         for (String nd : nodeGroup) {
             numPartitions += AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
         }
@@ -1652,12 +1364,6 @@
         return splitProviderAndPartitionConstraints(splits);
     }
 
-    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForExternalDataset(
-			String dataverseName, String datasetName, String targetIdxName) throws AlgebricksException {
-		FileSplit[] splits = splitsForExternalDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName);
-		return splitProviderAndPartitionConstraints(splits);
-	}
-    
     public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataverse(
             String dataverse) {
         FileSplit[] splits = splitsForDataverse(mdTxnCtx, dataverse);
@@ -1748,56 +1454,6 @@
         }
     }
 
-    private FileSplit[] splitsForExternalDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
-			String datasetName, String targetIdxName) throws AlgebricksException {
-
-		try {
-			File relPathFile = new File(getRelativePath(dataverseName, datasetName + "_idx_" + targetIdxName));
-			Dataset dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx, dataverseName, datasetName);
-			if (dataset.getDatasetType() != DatasetType.EXTERNAL) {
-				throw new AlgebricksException("Not an external dataset");
-			}
-			ExternalDatasetDetails datasetDetails = (ExternalDatasetDetails) dataset.getDatasetDetails();
-			List<String> nodeGroup = MetadataManager.INSTANCE.getNodegroup(mdTxnCtx, datasetDetails.getNodeGroupName())
-					.getNodeNames();
-			if (nodeGroup == null) {
-				throw new AlgebricksException("Couldn't find node group " + datasetDetails.getNodeGroupName());
-			}
-
-			List<FileSplit> splitArray = new ArrayList<FileSplit>();
-			for (String nd : nodeGroup) {
-				String[] nodeStores = stores.get(nd);
-				if (nodeStores == null) {
-					LOGGER.warning("Node " + nd + " has no stores.");
-					throw new AlgebricksException("Node " + nd + " has no stores.");
-				} else {
-					int numIODevices;
-					if (datasetDetails.getNodeGroupName().compareTo(MetadataConstants.METADATA_NODEGROUP_NAME) == 0) {
-						numIODevices = 1;
-					} else {
-						numIODevices = AsterixClusterProperties.INSTANCE.getNumberOfIODevices(nd);
-					}
-					String[] ioDevices = AsterixClusterProperties.INSTANCE.getIODevices(nd);
-					for (int j = 0; j < nodeStores.length; j++) {
-						for (int k = 0; k < numIODevices; k++) {
-							File f = new File(ioDevices[k] + File.separator + nodeStores[j] + File.separator
-									+ relPathFile);
-							splitArray.add(new FileSplit(nd, new FileReference(f), k));
-						}
-					}
-				}
-			}
-			FileSplit[] splits = new FileSplit[splitArray.size()];
-			int i = 0;
-			for (FileSplit fs : splitArray) {
-				splits[i++] = fs;
-			}
-			return splits;
-		} catch (MetadataException me) {
-			throw new AlgebricksException(me);
-		}
-	}
-    
     private static Map<String, String> initializeAdapterFactoryMapping() {
         Map<String, String> adapterFactoryMapping = new HashMap<String, String>();
         adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter",
@@ -1810,8 +1466,6 @@
                 "edu.uci.ics.asterix.external.dataset.adapter..RSSFeedAdapterFactory");
         adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapter",
                 "edu.uci.ics.asterix.external.dataset.adapter.CNNFeedAdapterFactory");
-        adapterFactoryMapping.put("edu.uci.ics.asterix.external.dataset.adapter.HiveAdapter",
-				"edu.uci.ics.asterix.external.adapter.factory.HiveAdapterFactory");
         return adapterFactoryMapping;
     }
 
@@ -1910,4 +1564,3 @@
     }
 
 }
-
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/evaluators/common/SimilarityJaccardCheckEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
index a4f0da2..d6fae85 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/SimilarityJaccardCheckEvaluator.java
@@ -71,7 +71,7 @@
         // Probe phase: Probe items from second list, and compute intersection size.
         int intersectionSize = 0;
         int probeListCount = 0;
-        int minUnionSize = probeListSize;
+        int minUnionSize = buildListSize;
         while (probeIter.hasNext()) {
             probeListCount++;
             byte[] buf = probeIter.getData();
@@ -97,7 +97,7 @@
                 // Could not find element in other set. Increase min union size by 1.
                 minUnionSize++;
                 // Check whether jaccThresh can still be satisfied if there was a mismatch.
-                int maxIntersectionSize = intersectionSize + (probeListSize - probeListCount);
+                int maxIntersectionSize = Math.min(buildListSize, intersectionSize + (probeListSize - probeListCount));
                 int lowerBound = (int) Math.floor(jaccThresh * minUnionSize);
                 if (maxIntersectionSize < lowerBound) {
                     // Cannot satisfy jaccThresh.
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
index b44755c..4f4eba2 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/job/listener/JobEventListenerFactory.java
@@ -49,7 +49,7 @@
                 try {
                     ITransactionManager txnManager = ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext()
                             .getApplicationObject()).getTransactionSubsystem().getTransactionManager();
-                    ITransactionContext txnContext = txnManager.getTransactionContext(jobId);
+                    ITransactionContext txnContext = txnManager.getTransactionContext(jobId, false);
                     txnContext.setWriteTxn(transactionalWrite);
                     txnManager.completedTransaction(txnContext, new DatasetId(-1), -1,
                             !(jobStatus == JobStatus.FAILURE));
@@ -62,7 +62,7 @@
             public void jobletStart() {
                 try {
                     ((IAsterixAppRuntimeContext) jobletContext.getApplicationContext().getApplicationObject())
-                            .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId);
+                            .getTransactionSubsystem().getTransactionManager().getTransactionContext(jobId, true);
                 } catch (ACIDException e) {
                     throw new Error(e);
                 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
index e08c509..b8c8659 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
@@ -92,16 +92,6 @@
             throw new AsterixException(e);
         }
     }
-    
-    @Override
-    public void close() throws IOException{
-    	admLexer.close();
-    }
-    
-    @Override
-    public void reset(){
-    	admLexer.reset();
-    }
 
     protected boolean parseAdmInstance(IAType objectType, boolean datasetRec, DataOutput out) throws AsterixException,
             IOException {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
deleted file mode 100644
index a4f1691..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractControlledTupleParser.java
+++ /dev/null
@@ -1,123 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import edu.uci.ics.hyracks.dataflow.common.comm.io.FrameTupleAppender;
-import edu.uci.ics.hyracks.dataflow.common.comm.util.FrameUtils;
-
-/**
- * An Abstract class implementation for IControlledTupleParser. It provides common
- * functionality involved in parsing data in an external format in a pipelined manner and packing
- * frames with formed tuples.
- * (DONE)
- */
-public abstract class AbstractControlledTupleParser extends ControlledTupleParser{
-
-	protected ArrayTupleBuilder tb = new ArrayTupleBuilder(1);
-	protected transient DataOutput dos;
-	protected final FrameTupleAppender appender;
-	protected final ByteBuffer frame;
-	protected final ARecordType recType;
-	protected final IHyracksTaskContext ctx;
-	protected IDataParser parser;
-	
-	public AbstractControlledTupleParser(IHyracksTaskContext ctx, ARecordType recType) throws HyracksDataException {
-		appender = new FrameTupleAppender(ctx.getFrameSize());
-		frame = ctx.allocateFrame();
-		this.recType = recType;
-		this.ctx = ctx;
-		dos = tb.getDataOutput();
-	}
-
-	public abstract IDataParser getDataParser();
-
-	@Override
-	public void parse(InputStream in, IFrameWriter writer)
-			throws HyracksDataException {
-		//This function when used works as non-pipelined parser
-		//This whole parser interface needs revisiting. 
-		appender.reset(frame, true);
-		parser = getDataParser();
-		try {
-			parser.initialize(in, recType, true);
-			while (true) {
-				tb.reset();
-				if (!parser.parse(tb.getDataOutput())) {
-					parser.reset();
-					break;
-				}
-				tb.addFieldEndOffset();
-				addTupleToFrame(writer);
-			}
-			parser.close();
-			if (appender.getTupleCount() > 0) {
-				FrameUtils.flushFrame(frame, writer);
-			}
-		} catch (Exception e) {
-			throw new HyracksDataException("Failed to initialize data parser");
-		}
-	}
-
-	@Override
-	public void initialize(InputStream in) throws HyracksDataException {
-		appender.reset(frame, true);
-		parser = getDataParser();
-		try {
-			parser.initialize(in, recType, true);
-
-		} catch (Exception e) {
-			throw new HyracksDataException("Failed to initialize data parser");
-		}
-	}
-
-	@Override
-	public void parseNext(IFrameWriter writer) throws HyracksDataException {
-		try {
-			while (true) {
-				tb.reset();
-				if (!parser.parse(tb.getDataOutput())) {
-					parser.reset();
-					break;
-				}
-				tb.addFieldEndOffset();
-				addTupleToFrame(writer);
-			}
-		} catch (AsterixException ae) {
-			throw new HyracksDataException(ae);
-		} catch (IOException ioe) {
-			throw new HyracksDataException(ioe);
-		}
-	}
-
-	@Override
-	public void close(IFrameWriter writer) throws HyracksDataException {
-		try{	
-			parser.close();
-			if (appender.getTupleCount() > 0) {
-				FrameUtils.flushFrame(frame, writer);
-			}
-		} catch (IOException ioe) {
-			throw new HyracksDataException(ioe);
-		}
-	}
-
-	protected void addTupleToFrame(IFrameWriter writer) throws HyracksDataException {
-		if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-			FrameUtils.flushFrame(frame, writer);
-			appender.reset(frame, true);
-			if (!appender.append(tb.getFieldEndOffsets(), tb.getByteArray(), 0, tb.getSize())) {
-				throw new IllegalStateException();
-			}
-		}
-
-	}
-
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
index 78159f5..2322338 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AbstractTupleParser.java
@@ -62,7 +62,6 @@
             while (true) {
                 tb.reset();
                 if (!parser.parse(tb.getDataOutput())) {
-                	parser.close();
                     break;
                 }
                 tb.addFieldEndOffset();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
index 34f75e6..7cb9bb0 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/AdmTupleParser.java
@@ -20,7 +20,7 @@
 
 /**
  * An extension of AbstractTupleParser that provides functionality for
- * parsing adm formatted input files.
+ * parsing delimited files.
  */
 public class AdmTupleParser extends AbstractTupleParser {
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
deleted file mode 100644
index aa2d9ba..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParser.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-
-/**
- * An extension of AbstractControlledTupleParser that provides functionality for
- * parsing Adm formatted input.
- */
-public class ControlledADMTupleParser extends AbstractControlledTupleParser{
-
-	public ControlledADMTupleParser(IHyracksTaskContext ctx, ARecordType recType)
-			throws HyracksDataException {
-		super(ctx, recType);
-	}
-
-	@Override
-	public IDataParser getDataParser() {
-		return new ADMDataParser();
-	}
-
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
deleted file mode 100644
index d72366c..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledADMTupleParserFactory.java
+++ /dev/null
@@ -1,28 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-
-/**
- * A Controlled tuple parser factory for creating a tuple parser capable of parsing
- * ADM data.
- */
-public class ControlledADMTupleParserFactory implements ITupleParserFactory{
-	private static final long serialVersionUID = 1L;
-
-    protected ARecordType recType;
-    
-    public ControlledADMTupleParserFactory(ARecordType recType){
-    	this.recType = recType;
-    }
-
-	@Override
-	public ITupleParser createTupleParser(IHyracksTaskContext ctx)
-			throws HyracksDataException {
-		return new ControlledADMTupleParser(ctx, recType);
-	}
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
deleted file mode 100644
index 15643d4..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParser.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-
-public class ControlledDelimitedDataTupleParser extends AbstractControlledTupleParser{
-
-	private final DelimitedDataParser dataParser;
-	
-	public ControlledDelimitedDataTupleParser(IHyracksTaskContext ctx,
-			ARecordType recType,  IValueParserFactory[] valueParserFactories, char fieldDelimter) throws HyracksDataException {
-		super(ctx, recType);
-		dataParser = new DelimitedDataParser(recType, valueParserFactories, fieldDelimter);
-	}
-
-	@Override
-	public IDataParser getDataParser() {
-		return dataParser;
-	}
-
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
deleted file mode 100644
index ced33ef..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledDelimitedDataTupleParserFactory.java
+++ /dev/null
@@ -1,29 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParserFactory;
-
-public class ControlledDelimitedDataTupleParserFactory implements ITupleParserFactory{
-	private static final long serialVersionUID = 1L;
-	private IValueParserFactory[] valueParserFactories;
-	private char fieldDelimiter;
-	protected ARecordType recordType;
-
-
-	public ControlledDelimitedDataTupleParserFactory(ARecordType recordType, IValueParserFactory[] fieldParserFactories, char fieldDelimiter) {
-		this.recordType = recordType;
-		this.valueParserFactories = fieldParserFactories;
-		this.fieldDelimiter = fieldDelimiter;
-	}
-	
-
-	@Override
-	public ITupleParser createTupleParser(IHyracksTaskContext ctx)
-			throws HyracksDataException {
-		return new ControlledDelimitedDataTupleParser(ctx, recordType, valueParserFactories, fieldDelimiter);
-	}
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
deleted file mode 100644
index 10b09f5..0000000
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ControlledTupleParser.java
+++ /dev/null
@@ -1,29 +0,0 @@
-package edu.uci.ics.asterix.runtime.operators.file;
-
-import java.io.InputStream;
-
-import edu.uci.ics.hyracks.api.comm.IFrameWriter;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.dataflow.std.file.ITupleParser;
-
-/**
- * This interface is to be implemented by parsers used in a pipelined hyracks job where input is not ready all at once
- */
-public abstract class ControlledTupleParser implements ITupleParser{
-
-	/**
-	 * This function associate an input stream with the parser
-	 */
-	public abstract void initialize(InputStream in) throws HyracksDataException;
-	
-	/**
-	 * This function should flush the tuples setting in the frame writer buffer
-	 * and free all resources
-	 */
-	public abstract void close(IFrameWriter writer) throws HyracksDataException;
-
-	/**
-	 * This function is called when there are more data ready for parsing in the input stream
-	 */
-	public abstract void parseNext(IFrameWriter writer) throws HyracksDataException;
-}
\ No newline at end of file
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
index 22c43ec..5a639dc 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/DelimitedDataParser.java
@@ -100,17 +100,6 @@
     }
 
     @Override
-	public void close() {
-		cursor.close();
-	}
-
-    @Override
-    public void reset()
-    {
-    	cursor.reset();
-    }
-    
-    @Override
     public boolean parse(DataOutput out) throws AsterixException, IOException {
         while (cursor.nextRecord()) {
             recBuilder.reset(recordType);
@@ -176,7 +165,7 @@
         private static final int INITIAL_BUFFER_SIZE = 4096;
         private static final int INCREMENT = 4096;
 
-        private Reader in;
+        private final Reader in;
 
         private char[] buffer;
         private int start;
@@ -193,21 +182,6 @@
             end = 0;
             state = State.INIT;
         }
-        
-        public void close(){
-				try {
-					in.close();
-				} catch (IOException e) {
-					// TODO Auto-generated catch block
-					e.printStackTrace();
-				}
-        }
-        
-        public void reset(){
-        	start = 0;
-            end = 0;
-            state = State.INIT;
-        }
 
         public boolean nextRecord() throws IOException {
             while (true) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
index cd9ae2e..46d281b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/IDataParser.java
@@ -52,17 +52,4 @@
      * @throws IOException
      */
     public boolean parse(DataOutput out) throws AsterixException, IOException;
-
-    /**
-     * Close the underlying inputStream object.
-     * 
-     */
-    public void close() throws IOException;
-    
-    /**
-     * Reset the parser before processing a new patch of input in the input stream
-     */
-    public void reset();
 }
-
-
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
index ffd9edd..e037ec6 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapter.java
@@ -224,8 +224,7 @@
             while (continueIngestion) {
                 tb.reset();
                 if (!parser.parse(tb.getDataOutput())) {
-                	parser.close();
-                	break;
+                    break;
                 }
                 tb.addFieldEndOffset();
                 if (delayConfigured) {
diff --git a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
index c576cf8..bf3c086 100644
--- a/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
+++ b/asterix-tools/src/main/java/edu/uci/ics/asterix/tools/external/data/RateControlledFileSystemBasedAdapterFactory.java
@@ -14,17 +14,14 @@
  */
 package edu.uci.ics.asterix.tools.external.data;
 
-import java.util.HashMap;
 import java.util.Map;
 
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.external.adapter.factory.IGenericDatasetAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.FileSystemBasedAdapter;
-import edu.uci.ics.asterix.external.dataset.adapter.IControlledAdapter;
 import edu.uci.ics.asterix.external.dataset.adapter.IDatasourceAdapter;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 
 /**
  * Factory class for creating @see{RateControllerFileSystemBasedAdapter} The
@@ -70,12 +67,6 @@
     public String getName() {
         return "file_feed";
     }
-    
-    @Override
-	public IDatasourceAdapter createIndexingAdapter(
-			Map<String, Object> configuration, IAType atype, Map<String,Integer> files) throws Exception {
-		throw new NotImplementedException("Rate Controlled Indexing Adapter is not implemented for feeds");
-	}
 
     private void checkRequiredArgs(Map<String, Object> configuration) throws Exception {
         if (configuration.get(KEY_FILE_SYSTEM) == null) {
@@ -92,10 +83,4 @@
         }
     }
 
-	@Override
-	public IControlledAdapter createAccessByRIDAdapter(
-			Map<String, Object> configuration, IAType atype, HashMap<Integer,String> files) throws Exception {
-		throw new NotImplementedException("Rate Controlled Access by RID Adapter is not implemented for feeds");
-	}
-
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
index a6537cd..00f0c4a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexInstantSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
             throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             return new PrimaryIndexInstantSearchOperationCallback(datasetId, primaryKeyFields,
                     txnSubsystem.getLockManager(), txnCtx);
         } catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index abeec62..07daee4 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -58,7 +58,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
                     primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
                     indexOp);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
index b59eb75..01cb725 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexSearchOperationCallbackFactory.java
@@ -41,7 +41,7 @@
             throws HyracksDataException {
         ITransactionSubsystem txnSubsystem = txnSubsystemProvider.getTransactionSubsystem(ctx);
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             return new PrimaryIndexSearchOperationCallback(datasetId, primaryKeyFields, txnSubsystem.getLockManager(),
                     txnCtx);
         } catch (ACIDException e) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index d5bc877..563e9b7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -55,7 +55,7 @@
         }
 
         try {
-            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
+            ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId, false);
             IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
                     txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
             txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
index aec378b..140a8dd 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexOperationTrackerProvider.java
@@ -18,7 +18,6 @@
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackFactory;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTracker;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerProvider;
 
@@ -27,10 +26,8 @@
     private static final long serialVersionUID = 1L;
 
     private final int datasetID;
-    private final ILSMIOOperationCallbackFactory ioOpCallbackFactory;
 
-    public SecondaryIndexOperationTrackerProvider(ILSMIOOperationCallbackFactory ioOpCallbackFactory, int datasetID) {
-        this.ioOpCallbackFactory = ioOpCallbackFactory;
+    public SecondaryIndexOperationTrackerProvider(int datasetID) {
         this.datasetID = datasetID;
     }
 
@@ -38,7 +35,7 @@
     public ILSMOperationTracker getOperationTracker(IHyracksTaskContext ctx) {
         DatasetLifecycleManager dslcManager = (DatasetLifecycleManager) ((IAsterixAppRuntimeContext) ctx
                 .getJobletContext().getApplicationContext().getApplicationObject()).getIndexLifecycleManager();
-        return new BaseOperationTracker(dslcManager, ioOpCallbackFactory, datasetID);
+        return new BaseOperationTracker(dslcManager, datasetID);
     }
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index d243dd2..cf60182 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -55,10 +55,11 @@
         LSMBTree lsmBTree = LSMBTreeUtils.createLSMTree(virtualBufferCaches, file, runtimeContextProvider
                 .getBufferCache(), runtimeContextProvider.getFileMapManager(), typeTraits, cmpFactories,
                 bloomFilterKeyFields, runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
-                        .getLSMMergePolicy(), isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID)
-                        : new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
-                                LSMBTreeIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
-                        .getLSMIOScheduler(), runtimeContextProvider.getLSMBTreeIOOperationCallbackProvider(isPrimary));
+                        .getLSMMergePolicy(),
+                isPrimary ? runtimeContextProvider.getLSMBTreeOperationTracker(datasetID) : new BaseOperationTracker(
+                        (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
+                runtimeContextProvider.getLSMIOScheduler(), LSMBTreeIOOperationCallbackFactory.INSTANCE
+                        .createIOOperationCallback());
         return lsmBTree;
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index 8482172..5ed7019 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -59,25 +59,37 @@
         List<IVirtualBufferCache> virtualBufferCaches = runtimeContextProvider.getVirtualBufferCaches(datasetID);
         try {
             if (isPartitioned) {
-                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(virtualBufferCaches, runtimeContextProvider
-                        .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
-                        runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
-                                .getLSMMergePolicy(), new BaseOperationTracker(
-                                (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
-                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
-                                .getLSMIOScheduler(), runtimeContextProvider
-                                .getLSMInvertedIndexIOOperationCallbackProvider());
+                return InvertedIndexUtils.createPartitionedLSMInvertedIndex(
+                        virtualBufferCaches,
+                        runtimeContextProvider.getFileMapManager(),
+                        invListTypeTraits,
+                        invListCmpFactories,
+                        tokenTypeTraits,
+                        tokenCmpFactories,
+                        tokenizerFactory,
+                        runtimeContextProvider.getBufferCache(),
+                        filePath,
+                        runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+                        runtimeContextProvider.getLSMMergePolicy(),
+                        new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider
+                                .getIndexLifecycleManager(), datasetID), runtimeContextProvider.getLSMIOScheduler(),
+                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
             } else {
-                return InvertedIndexUtils.createLSMInvertedIndex(virtualBufferCaches, runtimeContextProvider
-                        .getFileMapManager(), invListTypeTraits, invListCmpFactories, tokenTypeTraits,
-                        tokenCmpFactories, tokenizerFactory, runtimeContextProvider.getBufferCache(), filePath,
-                        runtimeContextProvider.getBloomFilterFalsePositiveRate(), runtimeContextProvider
-                                .getLSMMergePolicy(), new BaseOperationTracker(
-                                (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
-                                LSMInvertedIndexIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
-                                .getLSMIOScheduler(), runtimeContextProvider
-                                .getLSMInvertedIndexIOOperationCallbackProvider());
+                return InvertedIndexUtils.createLSMInvertedIndex(
+                        virtualBufferCaches,
+                        runtimeContextProvider.getFileMapManager(),
+                        invListTypeTraits,
+                        invListCmpFactories,
+                        tokenTypeTraits,
+                        tokenCmpFactories,
+                        tokenizerFactory,
+                        runtimeContextProvider.getBufferCache(),
+                        filePath,
+                        runtimeContextProvider.getBloomFilterFalsePositiveRate(),
+                        runtimeContextProvider.getLSMMergePolicy(),
+                        new BaseOperationTracker((DatasetLifecycleManager) runtimeContextProvider
+                                .getIndexLifecycleManager(), datasetID), runtimeContextProvider.getLSMIOScheduler(),
+                        LSMInvertedIndexIOOperationCallbackFactory.INSTANCE.createIOOperationCallback());
             }
         } catch (IndexException e) {
             throw new HyracksDataException(e);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index bc1e889..dc6b30b 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -66,10 +66,9 @@
                     runtimeContextProvider.getFileMapManager(), typeTraits, rtreeCmpFactories, btreeCmpFactories,
                     valueProviderFactories, rtreePolicyType, runtimeContextProvider.getBloomFilterFalsePositiveRate(),
                     runtimeContextProvider.getLSMMergePolicy(), new BaseOperationTracker(
-                            (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(),
-                            LSMRTreeIOOperationCallbackFactory.INSTANCE, datasetID), runtimeContextProvider
-                            .getLSMIOScheduler(), runtimeContextProvider.getLSMRTreeIOOperationCallbackProvider(),
-                    linearizeCmpFactory);
+                            (DatasetLifecycleManager) runtimeContextProvider.getIndexLifecycleManager(), datasetID),
+                    runtimeContextProvider.getLSMIOScheduler(), LSMRTreeIOOperationCallbackFactory.INSTANCE
+                            .createIOOperationCallback(), linearizeCmpFactory);
         } catch (TreeIndexException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index 6d86f70..c7df2f2 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -665,8 +665,6 @@
 
         latchLockTable();
         try {
-            validateJob(txnContext);
-
             if (IS_DEBUG_MODE) {
                 trackLockRequest("Requested", RequestType.UNLOCK, datasetId, entityHashValue, (byte) 0, txnContext,
                         dLockInfo, eLockInfo);
@@ -2212,14 +2210,14 @@
                 if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
                     tempDatasetIdObj.setId(logRecord.getDatasetId());
                     tempJobIdObj.setId(logRecord.getJobId());
-                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
                     unlock(tempDatasetIdObj, logRecord.getPKHashValue(), txnCtx);
                     txnCtx.notifyOptracker(false);
-                } else if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+                } else if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
                     tempJobIdObj.setId(logRecord.getJobId());
-                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj, false);
                     txnCtx.notifyOptracker(true);
-                    ((LogPage) logPage).notifyJobCommitter();
+                    ((LogPage) logPage).notifyJobTerminator();
                 }
                 logRecord = logPageReader.next();
             }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 4f0bb59..933afcd 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -38,6 +38,7 @@
 import edu.uci.ics.asterix.common.transactions.ILogReader;
 import edu.uci.ics.asterix.common.transactions.ILogRecord;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
 import edu.uci.ics.asterix.common.transactions.MutableLong;
 import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
@@ -74,16 +75,16 @@
         logDir = logManagerProperties.getLogDir();
         logFilePrefix = logManagerProperties.getLogFilePrefix();
         flushLSN = new MutableLong();
-        initializeLogManager();
+        initializeLogManager(0);
     }
 
-    private void initializeLogManager() {
+    private void initializeLogManager(long nextLogFileId) {
         emptyQ = new LinkedBlockingQueue<LogPage>(numLogPages);
         flushQ = new LinkedBlockingQueue<LogPage>(numLogPages);
         for (int i = 0; i < numLogPages; i++) {
             emptyQ.offer(new LogPage((LockManager) txnSubsystem.getLockManager(), logPageSize, flushLSN));
         }
-        appendLSN = initializeLogAnchor();
+        appendLSN = initializeLogAnchor(nextLogFileId);
         flushLSN.set(appendLSN);
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("LogManager starts logging in LSN: " + appendLSN);
@@ -95,12 +96,13 @@
     }
 
     @Override
-    public void log(ILogRecord logRecord) {
+    public void log(ILogRecord logRecord) throws ACIDException {
         if (logRecord.getLogSize() > logPageSize) {
             throw new IllegalStateException();
         }
         syncLog(logRecord);
-        if (logRecord.getLogType() == LogType.JOB_COMMIT && !logRecord.isFlushed()) {
+        if ((logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT)
+                && !logRecord.isFlushed()) {
             synchronized (logRecord) {
                 while (!logRecord.isFlushed()) {
                     try {
@@ -113,13 +115,16 @@
         }
     }
 
-    private synchronized void syncLog(ILogRecord logRecord) {
+    private synchronized void syncLog(ILogRecord logRecord) throws ACIDException {
         ITransactionContext txnCtx = logRecord.getTxnCtx();
+        if (txnCtx.getTxnState() == ITransactionManager.ABORTED && logRecord.getLogType() != LogType.ABORT) {
+            throw new ACIDException("Aborted job(" + txnCtx.getJobId() + ") tried to write non-abort type log record.");
+        }
         if (getLogFileOffset(appendLSN) + logRecord.getLogSize() > logFileSize) {
             prepareNextLogFile();
             appendPage.isFull(true);
             getAndInitNewPage();
-        } else if (!appendPage.hasSpace(logRecord.getLogSize(), getLogFileOffset(appendLSN))) {
+        } else if (!appendPage.hasSpace(logRecord.getLogSize())) {
             appendPage.isFull(true);
             getAndInitNewPage();
         }
@@ -141,7 +146,6 @@
         }
         appendPage.reset();
         appendPage.setFileChannel(appendChannel);
-        appendPage.setInitialFlushOffset(getLogFileOffset(appendLSN));
         flushQ.offer(appendPage);
     }
 
@@ -229,7 +233,7 @@
         return flushLSN;
     }
 
-    private long initializeLogAnchor() {
+    private long initializeLogAnchor(long nextLogFileId) {
         long fileId = 0;
         long offset = 0;
         File fileLogDir = new File(logDir);
@@ -237,9 +241,10 @@
             if (fileLogDir.exists()) {
                 List<Long> logFileIds = getLogFileIds();
                 if (logFileIds == null) {
-                    createFileIfNotExists(getLogFilePath(0));
+                    fileId = nextLogFileId;
+                    createFileIfNotExists(getLogFilePath(fileId));
                     if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("created a log file: " + getLogFilePath(0));
+                        LOGGER.info("created a log file: " + getLogFilePath(fileId));
                     }
                 } else {
                     fileId = logFileIds.get(logFileIds.size() - 1);
@@ -247,13 +252,14 @@
                     offset = logFile.length();
                 }
             } else {
+                fileId = nextLogFileId;
                 createNewDirectory(logDir);
                 if (LOGGER.isLoggable(Level.INFO)) {
                     LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
                 }
-                createFileIfNotExists(getLogFilePath(0));
+                createFileIfNotExists(getLogFilePath(fileId));
                 if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created a log file: " + getLogFilePath(0));
+                    LOGGER.info("created a log file: " + getLogFilePath(fileId));
                 }
             }
         } catch (IOException ioe) {
@@ -267,8 +273,8 @@
 
     public void renewLogFiles() {
         terminateLogFlusher();
-        deleteAllLogFiles();
-        initializeLogManager();
+        long lastMaxLogFileId = deleteAllLogFiles();
+        initializeLogManager(lastMaxLogFileId + 1);
     }
 
     private void terminateLogFlusher() {
@@ -290,7 +296,7 @@
         }
     }
 
-    private void deleteAllLogFiles() {
+    private long deleteAllLogFiles() {
         if (appendChannel != null) {
             try {
                 appendChannel.close();
@@ -305,6 +311,7 @@
                 throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
             }
         }
+        return logFileIds.get(logFileIds.size() - 1);
     }
 
     private List<Long> getLogFileIds() {
@@ -384,10 +391,16 @@
         }
         return newFileChannel;
     }
+
+    public long getReadableSmallestLSN() {
+        List<Long> logFileIds = getLogFileIds();
+        return logFileIds.get(0) * logFileSize;
+    }
 }
 
 class LogFlusher implements Callable<Boolean> {
-    private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_COMMIT_LOG_SIZE, null);
+    private static final Logger LOGGER = Logger.getLogger(LogFlusher.class.getName());
+    private final static LogPage POISON_PILL = new LogPage(null, ILogRecord.JOB_TERMINATE_LOG_SIZE, null);
     private final LogManager logMgr;//for debugging
     private final LinkedBlockingQueue<LogPage> emptyQ;
     private final LinkedBlockingQueue<LogPage> flushQ;
@@ -402,13 +415,13 @@
         flushPage = null;
         isStarted = new AtomicBoolean(false);
         terminateFlag = new AtomicBoolean(false);
-        
+
     }
 
     public void terminate() {
         //make sure the LogFlusher thread started before terminating it.
         synchronized (isStarted) {
-            while(!isStarted.get()) {
+            while (!isStarted.get()) {
                 try {
                     isStarted.wait();
                 } catch (InterruptedException e) {
@@ -416,7 +429,7 @@
                 }
             }
         }
-        
+
         terminateFlag.set(true);
         if (flushPage != null) {
             synchronized (flushPage) {
@@ -432,24 +445,34 @@
 
     @Override
     public Boolean call() {
-        synchronized(isStarted) {
+        synchronized (isStarted) {
             isStarted.set(true);
             isStarted.notify();
         }
-        while (true) {
-            flushPage = null;
-            try {
-                flushPage = flushQ.take();
-                if (flushPage == POISON_PILL || terminateFlag.get()) {
-                    return true;
+        try {
+            while (true) {
+                flushPage = null;
+                try {
+                    flushPage = flushQ.take();
+                    if (flushPage == POISON_PILL || terminateFlag.get()) {
+                        return true;
+                    }
+                } catch (InterruptedException e) {
+                    if (flushPage == null) {
+                        continue;
+                    }
                 }
-            } catch (InterruptedException e) {
-                if (flushPage == null) {
-                    continue;
-                }
+                flushPage.flush();
+                emptyQ.offer(flushPage);
             }
-            flushPage.flush();
-            emptyQ.offer(flushPage);
+        } catch (Exception e) {
+            if (LOGGER.isLoggable(Level.INFO)) {
+                LOGGER.info("-------------------------------------------------------------------------");
+                LOGGER.info("LogFlusher is terminating abnormally. System is in unusalbe state.");
+                LOGGER.info("-------------------------------------------------------------------------");
+            }
+            e.printStackTrace();
+            throw e;
         }
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
index edfec69..a3f42a7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
@@ -58,7 +58,7 @@
         appendOffset = 0;
         flushOffset = 0;
         isLastPage = false;
-        syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_COMMIT_LOG_SIZE);
+        syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.JOB_TERMINATE_LOG_SIZE);
     }
 
     ////////////////////////////////////
@@ -68,15 +68,14 @@
     @Override
     public void append(ILogRecord logRecord, long appendLSN) {
         logRecord.writeLogRecord(appendBuffer);
-        if (logRecord.getLogType() == LogType.UPDATE) {
-            logRecord.getTxnCtx().setLastLSN(logRecord.getResourceId(), appendLSN);
-        }
+        logRecord.getTxnCtx().setLastLSN(logRecord.getLogType() == LogType.UPDATE ? logRecord.getResourceId() : -1,
+                appendLSN);
         synchronized (this) {
             appendOffset += logRecord.getLogSize();
             if (IS_DEBUG_MODE) {
                 LOGGER.info("append()| appendOffset: " + appendOffset);
             }
-            if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+            if (logRecord.getLogType() == LogType.JOB_COMMIT || logRecord.getLogType() == LogType.ABORT) {
                 logRecord.isFlushed(false);
                 syncCommitQ.offer(logRecord);
             }
@@ -105,7 +104,7 @@
         this.isLastPage = isLastPage;
     }
 
-    public boolean hasSpace(int logSize, long logFileOffset) {
+    public boolean hasSpace(int logSize) {
         return appendOffset + logSize <= logPageSize;
     }
 
@@ -192,7 +191,7 @@
         }
     }
 
-    public void notifyJobCommitter() {
+    public void notifyJobTerminator() {
         ILogRecord logRecord = null;
         while (logRecord == null) {
             try {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
index 4b0e1f2..dd81df7 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
@@ -34,10 +34,9 @@
  * LogType(1)
  * JobId(4)
  * ---------------------------
- * [Header2] (16 bytes + PKValueSize) : for entity_commit and update log types 
+ * [Header2] (12 bytes + PKValueSize) : for entity_commit and update log types 
  * DatasetId(4) //stored in dataset_dataset in Metadata Node
  * PKHashValue(4)
- * PKFieldCnt(4)
  * PKValueSize(4)
  * PKValue(PKValueSize)
  * ---------------------------
@@ -61,10 +60,10 @@
  * ---------------------------
  * = LogSize =
  * 1) JOB_COMMIT_LOG_SIZE: 13 bytes (5 + 8)
- * 2) ENTITY_COMMIT: 29 + PKSize (5 + 16 + PKSize + 8)
- *    --> ENTITY_COMMIT_LOG_BASE_SIZE = 29
- * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 16 + PKSize + 21 + 14 + New/OldValueSize + 8)
- *    --> UPDATE_LOG_BASE_SIZE = 64
+ * 2) ENTITY_COMMIT: 25 + PKSize (5 + 12 + PKSize + 8)
+ *    --> ENTITY_COMMIT_LOG_BASE_SIZE = 25
+ * 3) UPDATE: 64 + PKSize + New/OldValueSize (5 + 12 + PKSize + 21 + 14 + New/OldValueSize + 8)
+ *    --> UPDATE_LOG_BASE_SIZE = 60
  */
 public class LogRecord implements ILogRecord {
 
@@ -73,7 +72,6 @@
     private int jobId;
     private int datasetId;
     private int PKHashValue;
-    private int PKFieldCnt;
     private int PKValueSize;
     private ITupleReference PKValue;
     private long prevLSN;
@@ -90,12 +88,13 @@
     private long checksum;
     //------------- fields in a log record (end) --------------//
 
+    private int PKFieldCnt;
     private static final int CHECKSUM_SIZE = 8;
     private ITransactionContext txnCtx;
     private long LSN;
     private final AtomicBoolean isFlushed;
     private final SimpleTupleWriter tupleWriter;
-    private final SimpleTupleReference readPKValue;
+    private final PrimaryKeyTupleReference readPKValue;
     private final SimpleTupleReference readNewValue;
     private final SimpleTupleReference readOldValue;
     private final CRC32 checksumGen;
@@ -104,7 +103,7 @@
     public LogRecord() {
         isFlushed = new AtomicBoolean(false);
         tupleWriter = new SimpleTupleWriter();
-        readPKValue = (SimpleTupleReference) tupleWriter.createTupleReference();
+        readPKValue = new PrimaryKeyTupleReference();
         readNewValue = (SimpleTupleReference) tupleWriter.createTupleReference();
         readOldValue = (SimpleTupleReference) tupleWriter.createTupleReference();
         checksumGen = new CRC32();
@@ -115,10 +114,9 @@
         int beginOffset = buffer.position();
         buffer.put(logType);
         buffer.putInt(jobId);
-        if (logType != LogType.JOB_COMMIT) {
+        if (logType == LogType.UPDATE || logType == LogType.ENTITY_COMMIT) {
             buffer.putInt(datasetId);
             buffer.putInt(PKHashValue);
-            buffer.putInt(PKFieldCnt);
             if (PKValueSize <= 0) {
                 throw new IllegalStateException("Primary Key Size is less than or equal to 0");
             }
@@ -173,13 +171,12 @@
         try {
             logType = buffer.get();
             jobId = buffer.getInt();
-            if (logType == LogType.JOB_COMMIT) {
+            if (logType == LogType.JOB_COMMIT || logType == LogType.ABORT) {
                 datasetId = -1;
                 PKHashValue = -1;
             } else {
-                datasetId = buffer.getInt();    
+                datasetId = buffer.getInt();
                 PKHashValue = buffer.getInt();
-                PKFieldCnt = buffer.getInt();
                 PKValueSize = buffer.getInt();
                 if (PKValueSize <= 0) {
                     throw new IllegalStateException("Primary Key Size is less than or equal to 0");
@@ -217,12 +214,20 @@
         }
         return true;
     }
-    
+
     private ITupleReference readPKValue(ByteBuffer buffer) {
-        return readTuple(buffer, readPKValue, PKFieldCnt, PKValueSize);
+        if (buffer.position() + PKValueSize > buffer.limit()) {
+            throw new BufferUnderflowException();
+        }
+        readPKValue.reset(buffer.array(), buffer.position(), PKValueSize);
+        buffer.position(buffer.position() + PKValueSize);
+        return readPKValue;
     }
 
     private ITupleReference readTuple(ByteBuffer srcBuffer, SimpleTupleReference destTuple, int fieldCnt, int size) {
+        if (srcBuffer.position() + size > srcBuffer.limit()) {
+            throw new BufferUnderflowException();
+        }
         destTuple.setFieldCount(fieldCnt);
         destTuple.resetByTupleOffset(srcBuffer, srcBuffer.position());
         srcBuffer.position(srcBuffer.position() + size);
@@ -230,9 +235,9 @@
     }
 
     @Override
-    public void formJobCommitLogRecord(ITransactionContext txnCtx) {
+    public void formJobTerminateLogRecord(ITransactionContext txnCtx, boolean isCommit) {
         this.txnCtx = txnCtx;
-        this.logType = LogType.JOB_COMMIT;
+        this.logType = isCommit ? LogType.JOB_COMMIT : LogType.ABORT;
         this.jobId = txnCtx.getJobId().getId();
         this.datasetId = -1;
         this.PKHashValue = -1;
@@ -281,7 +286,8 @@
                 setUpdateLogSize();
                 break;
             case LogType.JOB_COMMIT:
-                logSize = JOB_COMMIT_LOG_SIZE;
+            case LogType.ABORT:
+                logSize = JOB_TERMINATE_LOG_SIZE;
                 break;
             case LogType.ENTITY_COMMIT:
                 logSize = ENTITY_COMMIT_LOG_BASE_SIZE + PKValueSize;
@@ -298,7 +304,7 @@
         builder.append(" LogType : ").append(LogType.toString(logType));
         builder.append(" LogSize : ").append(logSize);
         builder.append(" JobId : ").append(jobId);
-        if (logType != LogType.JOB_COMMIT) {
+        if (logType == LogType.ENTITY_COMMIT || logType == LogType.UPDATE) {
             builder.append(" DatasetId : ").append(datasetId);
             builder.append(" PKHashValue : ").append(PKHashValue);
             builder.append(" PKFieldCnt : ").append(PKFieldCnt);
@@ -496,17 +502,17 @@
     public void setLSN(long LSN) {
         this.LSN = LSN;
     }
-    
+
     @Override
     public int getPKValueSize() {
         return PKValueSize;
     }
-    
+
     @Override
     public ITupleReference getPKValue() {
         return PKValue;
     }
-    
+
     @Override
     public void setPKFields(int[] primaryKeyFields) {
         PKFields = primaryKeyFields;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
index 823c8d3..f9e9304 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
@@ -19,11 +19,14 @@
     public static final byte UPDATE = 0;
     public static final byte JOB_COMMIT = 1;
     public static final byte ENTITY_COMMIT = 2;
+    public static final byte ABORT = 3;
     private static final String STRING_UPDATE = "UPDATE";
     private static final String STRING_JOB_COMMIT = "JOB_COMMIT";
     private static final String STRING_ENTITY_COMMIT = "ENTITY_COMMIT";
+    private static final String STRING_ABORT = "ABORT";
     private static final String STRING_INVALID_LOG_TYPE = "INVALID_LOG_TYPE";
 
+
     public static String toString(byte logType) {
         switch (logType) {
             case LogType.UPDATE:
@@ -32,6 +35,8 @@
                 return STRING_JOB_COMMIT;
             case LogType.ENTITY_COMMIT:
                 return STRING_ENTITY_COMMIT;
+            case LogType.ABORT:
+                return STRING_ABORT;
             default:
                 return STRING_INVALID_LOG_TYPE;
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
new file mode 100644
index 0000000..d45b209
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/PrimaryKeyTupleReference.java
@@ -0,0 +1,36 @@
+package edu.uci.ics.asterix.transaction.management.service.logging;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class PrimaryKeyTupleReference implements ITupleReference {
+    private byte[] fieldData;
+    private int start;
+    private int length;
+
+    public void reset(byte[] fieldData, int start, int length) {
+        this.fieldData = fieldData;
+        this.start = start;
+        this.length = length;
+    }
+    
+    @Override
+    public int getFieldCount() {
+        return 1;
+    }
+
+    @Override
+    public byte[] getFieldData(int fIdx) {
+        return fieldData;
+    }
+
+    @Override
+    public int getFieldStart(int fIdx) {
+        return start;
+    }
+
+    @Override
+    public int getFieldLength(int fIdx) {
+        return length;
+    }
+
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
index 6f6da4a..dca14d8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointThread.java
@@ -16,8 +16,8 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
@@ -70,7 +70,8 @@
         long firstLSN;
         if (openIndexList.size() > 0) {
             for (IIndex index : openIndexList) {
-                firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                firstLSN = ((AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback())
+                        .getFirstLSN();
                 minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
             }
         } else {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index 2ad3055..95ee767 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -38,7 +38,6 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
@@ -77,6 +76,7 @@
     private final TransactionSubsystem txnSubsystem;
     private final LogManager logMgr;
     private final int checkpointHistory;
+    private final long SHARP_CHECKPOINT_LSN = -1;
 
     /**
      * A file at a known location that contains the LSN of the last log record
@@ -115,18 +115,23 @@
             return state;
         }
 
-        //#. if minMCTFirstLSN is equal to -1 && 
-        //   checkpointLSN in the checkpoint file is equal to the lastLSN in the log file,
-        //   then return healthy state. Otherwise, return corrupted.
-        if ((checkpointObject.getMinMCTFirstLsn() == -2 && logMgr.getAppendLSN() == 0)
-                || (checkpointObject.getMinMCTFirstLsn() == -1 && checkpointObject.getCheckpointLsn() == logMgr
-                        .getAppendLSN())) {
+        long readableSmallestLSN = logMgr.getReadableSmallestLSN();
+        if (logMgr.getAppendLSN() == readableSmallestLSN) {
+            if (checkpointObject.getMinMCTFirstLsn() != SHARP_CHECKPOINT_LSN) {
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("[Warning] ---------------------------------------------------");
+                    LOGGER.info("[Warning] Some(or all) of transaction log files are lost.");
+                    LOGGER.info("[Warning] ---------------------------------------------------");
+                    //No choice but continuing when the log files are lost. 
+                }
+            }
+            state = SystemState.HEALTHY;
+            return state;
+        } else if (checkpointObject.getCheckpointLsn() == logMgr.getAppendLSN()
+                && checkpointObject.getMinMCTFirstLsn() == SHARP_CHECKPOINT_LSN) {
             state = SystemState.HEALTHY;
             return state;
         } else {
-            if (logMgr.getAppendLSN() == 0) {
-                throw new IllegalStateException("Transaction log files are lost.");
-            }
             state = SystemState.CORRUPTED;
             return state;
         }
@@ -138,6 +143,7 @@
         int entityCommitLogCount = 0;
         int jobCommitLogCount = 0;
         int redoCount = 0;
+        int abortLogCount = 0;
         int jobId = -1;
 
         state = SystemState.RECOVERING;
@@ -154,10 +160,11 @@
         TxnId winnerEntity = null;
 
         //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
+        long readableSmallestLSN = logMgr.getReadableSmallestLSN();
         CheckpointObject checkpointObject = readCheckpoint();
-        long lowWaterMarkLsn = checkpointObject.getMinMCTFirstLsn();
-        if (lowWaterMarkLsn == -1 || lowWaterMarkLsn == -2) {
-            lowWaterMarkLsn = 0;
+        long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLsn();
+        if (lowWaterMarkLSN < readableSmallestLSN) {
+            lowWaterMarkLSN = readableSmallestLSN;
         }
         int maxJobId = checkpointObject.getMaxJobId();
 
@@ -171,11 +178,11 @@
 
         //#. set log reader to the lowWaterMarkLsn
         ILogReader logReader = logMgr.getLogReader(true);
-        logReader.initializeScan(lowWaterMarkLsn);
+        logReader.initializeScan(lowWaterMarkLSN);
         ILogRecord logRecord = logReader.next();
         while (logRecord != null) {
             if (IS_DEBUG_MODE) {
-                System.out.println(logRecord.getLogRecordForDisplay());
+                LOGGER.info(logRecord.getLogRecordForDisplay());
             }
             //update max jobId
             if (logRecord.getJobId() > maxJobId) {
@@ -183,16 +190,12 @@
             }
             switch (logRecord.getLogType()) {
                 case LogType.UPDATE:
-                    if (IS_DEBUG_MODE) {
-                        updateLogCount++;
-                    }
+                    updateLogCount++;
                     break;
                 case LogType.JOB_COMMIT:
                     winnerJobSet.add(Integer.valueOf(logRecord.getJobId()));
                     jobId2WinnerEntitiesMap.remove(Integer.valueOf(logRecord.getJobId()));
-                    if (IS_DEBUG_MODE) {
-                        jobCommitLogCount++;
-                    }
+                    jobCommitLogCount++;
                     break;
                 case LogType.ENTITY_COMMIT:
                     jobId = logRecord.getJobId();
@@ -205,9 +208,10 @@
                         winnerEntitySet = jobId2WinnerEntitiesMap.get(Integer.valueOf(jobId));
                     }
                     winnerEntitySet.add(winnerEntity);
-                    if (IS_DEBUG_MODE) {
-                        entityCommitLogCount++;
-                    }
+                    entityCommitLogCount++;
+                    break;
+                case LogType.ABORT:
+                    abortLogCount++;
                     break;
                 default:
                     throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
@@ -239,11 +243,11 @@
         ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
 
         //#. set log reader to the lowWaterMarkLsn again.
-        logReader.initializeScan(lowWaterMarkLsn);
+        logReader.initializeScan(lowWaterMarkLSN);
         logRecord = logReader.next();
         while (logRecord != null) {
-            if (LogManager.IS_DEBUG_MODE) {
-                System.out.println(logRecord.getLogRecordForDisplay());
+            if (IS_DEBUG_MODE) {
+                LOGGER.info(logRecord.getLogRecordForDisplay());
             }
             LSN = logRecord.getLSN();
             jobId = logRecord.getJobId();
@@ -284,6 +288,7 @@
                              * log record.
                              *******************************************************************/
                             if (localResource == null) {
+                                logRecord = logReader.next();
                                 continue;
                             }
                             /*******************************************************************/
@@ -297,10 +302,8 @@
 
                             //#. get maxDiskLastLSN
                             ILSMIndex lsmIndex = (ILSMIndex) index;
-                            BaseOperationTracker indexOpTracker = (BaseOperationTracker) lsmIndex.getOperationTracker();
-                            AbstractLSMIOOperationCallback abstractLSMIOCallback = (AbstractLSMIOOperationCallback) indexOpTracker
-                                    .getIOOperationCallback();
-                            maxDiskLastLsn = abstractLSMIOCallback.getComponentLSN(index.getImmutableComponents());
+                            maxDiskLastLsn = ((AbstractLSMIOOperationCallback) lsmIndex.getIOOperationCallback())
+                                    .getComponentLSN(lsmIndex.getImmutableComponents());
 
                             //#. set resourceId and maxDiskLastLSN to the map
                             resourceId2MaxLSNMap.put(Long.valueOf(resourceId), Long.valueOf(maxDiskLastLsn));
@@ -310,15 +313,14 @@
 
                         if (LSN > maxDiskLastLsn) {
                             redo(logRecord);
-                            if (IS_DEBUG_MODE) {
-                                redoCount++;
-                            }
+                            redoCount++;
                         }
                     }
                     break;
 
                 case LogType.JOB_COMMIT:
                 case LogType.ENTITY_COMMIT:
+                case LogType.ABORT:
                     //do nothing
                     break;
 
@@ -338,10 +340,8 @@
 
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("[RecoveryMgr] recovery is completed.");
-        }
-        if (IS_DEBUG_MODE) {
-            System.out.println("[RecoveryMgr] Count: Update/EntityCommit/JobCommit/Redo = " + updateLogCount + "/"
-                    + entityCommitLogCount + "/" + jobCommitLogCount + "/" + redoCount);
+            LOGGER.info("[RecoveryMgr's recovery log count] update/entityCommit/jobCommit/abort/redo = " + updateLogCount + "/"
+                    + entityCommitLogCount + "/" + jobCommitLogCount + "/" + abortLogCount + "/" + redoCount);
         }
     }
 
@@ -375,9 +375,8 @@
                 ILSMIndex lsmIndex = (ILSMIndex) index;
                 ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(NoOpOperationCallback.INSTANCE,
                         NoOpOperationCallback.INSTANCE);
-                BaseOperationTracker indexOpTracker = (BaseOperationTracker) lsmIndex.getOperationTracker();
                 BlockingIOOperationCallbackWrapper cb = new BlockingIOOperationCallbackWrapper(
-                        indexOpTracker.getIOOperationCallback());
+                        lsmIndex.getIOOperationCallback());
                 callbackList.add(cb);
                 try {
                     indexAccessor.scheduleFlush(cb);
@@ -393,17 +392,18 @@
                     throw new ACIDException(e);
                 }
             }
-            minMCTFirstLSN = -2;
+            minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
         } else {
             long firstLSN;
             minMCTFirstLSN = Long.MAX_VALUE;
             if (openIndexList.size() > 0) {
                 for (IIndex index : openIndexList) {
-                    firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                    firstLSN = ((AbstractLSMIOOperationCallback) ((ILSMIndex) index).getIOOperationCallback())
+                            .getFirstLSN();
                     minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
                 }
             } else {
-                minMCTFirstLSN = -1;
+                minMCTFirstLSN = SHARP_CHECKPOINT_LSN;
             }
         }
         CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
@@ -580,7 +580,7 @@
                 break;
             } else {
                 if (IS_DEBUG_MODE) {
-                    System.out.println(logRecord.getLogRecordForDisplay());
+                    LOGGER.info(logRecord.getLogRecordForDisplay());
                 }
                 currentLSN = logRecord.getLSN();
             }
@@ -600,9 +600,9 @@
                         loserTxnTable.put(loserEntity, undoLSNSet);
                     }
                     undoLSNSet.add(Long.valueOf(currentLSN));
+                    updateLogCount++;
                     if (IS_DEBUG_MODE) {
-                        updateLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
+                        LOGGER.info("" + Thread.currentThread().getId() + "======> update[" + currentLSN + "]:"
                                 + tempKeyTxnId);
                     }
                     break;
@@ -611,14 +611,21 @@
                     throw new ACIDException("Unexpected LogType(" + logRecord.getLogType() + ") during abort.");
 
                 case LogType.ENTITY_COMMIT:
-                    loserTxnTable.remove(tempKeyTxnId);
+                    undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
+                    if (undoLSNSet == null) {
+                        undoLSNSet = loserTxnTable.remove(tempKeyTxnId);
+                    }
+                    entityCommitLogCount++;
                     if (IS_DEBUG_MODE) {
-                        entityCommitLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
+                        LOGGER.info("" + Thread.currentThread().getId() + "======> entity_commit[" + currentLSN + "]"
                                 + tempKeyTxnId);
                     }
                     break;
 
+                case LogType.ABORT:
+                    //ignore
+                    break;
+
                 default:
                     throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
@@ -650,23 +657,17 @@
                     throw new ACIDException("IllegalState exception during abort( " + txnContext.getJobId() + ")");
                 }
                 if (IS_DEBUG_MODE) {
-                    System.out.println(logRecord.getLogRecordForDisplay());
+                    LOGGER.info(logRecord.getLogRecordForDisplay());
                 }
                 undo(logRecord);
-                if (IS_DEBUG_MODE) {
-                    undoCount++;
-                }
+                undoCount++;
             }
         }
-
         logReader.close();
-
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(" undone loser transaction's effect");
-        }
-        if (IS_DEBUG_MODE) {
-            System.out.println("UpdateLogCount/CommitLogCount/UndoCount:" + updateLogCount + "/" + entityCommitLogCount
-                    + "/" + undoCount);
+            LOGGER.info("[RecoveryManager's rollback log count] update/entityCommit/undo:" + updateLogCount + "/" + entityCommitLogCount + "/"
+                    + undoCount);
         }
     }
 
@@ -721,7 +722,7 @@
             } else {
                 throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
             }
-            ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(logRecord.getLSN());
+            ((AbstractLSMIOOperationCallback) index.getIOOperationCallback()).updateLastLSN(logRecord.getLSN());
         } catch (Exception e) {
             throw new IllegalStateException("Failed to redo", e);
         }
@@ -765,6 +766,7 @@
         this.datasetId = datasetId;
         this.pkHashValue = pkHashValue;
         this.tupleReferencePKValue = pkValue;
+        this.pkSize = pkSize;
         isByteArrayPKValue = false;
     }
 
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
index 59a8363..5d4806d 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/AsterixRuntimeComponentsProvider.java
@@ -15,15 +15,11 @@
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerProvider;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicyProvider;
 import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
@@ -33,18 +29,12 @@
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 
 public class AsterixRuntimeComponentsProvider implements IIndexLifecycleManagerProvider, IStorageManagerInterface,
-        ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider,
-        ILSMIOOperationCallbackProvider {
+        ILSMIOOperationSchedulerProvider, ILSMMergePolicyProvider {
     private static final long serialVersionUID = 1L;
 
     public static final AsterixRuntimeComponentsProvider RUNTIME_PROVIDER = new AsterixRuntimeComponentsProvider();
-    
-    private AsterixRuntimeComponentsProvider() {
-    }
 
-    @Override
-    public ILSMIOOperationCallback getIOOperationCallback(ILSMIndex index) {
-        return ((BaseOperationTracker) index.getOperationTracker()).getIOOperationCallback();
+    private AsterixRuntimeComponentsProvider() {
     }
 
     @Override
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index 678956b..6fb91c8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -21,9 +21,9 @@
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 
-import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.asterix.common.context.PrimaryIndexOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager;
@@ -71,7 +71,7 @@
 
     //indexMap is concurrently accessed by multiple threads, 
     //so those threads are synchronized on indexMap object itself
-    private Map<MutableLong, BaseOperationTracker> indexMap;
+    private Map<MutableLong, AbstractLSMIOOperationCallback> indexMap;
 
     //TODO: fix ComponentLSNs' issues. 
     //primaryIndex, primaryIndexCallback, and primaryIndexOptracker will be modified accordingly
@@ -97,7 +97,7 @@
         isTimeout = false;
         isWriteTxn = new AtomicBoolean(false);
         isMetadataTxn = false;
-        indexMap = new HashMap<MutableLong, BaseOperationTracker>();
+        indexMap = new HashMap<MutableLong, AbstractLSMIOOperationCallback>();
         primaryIndex = null;
         tempResourceIdForRegister = new MutableLong();
         tempResourceIdForSetLSN = new MutableLong();
@@ -114,7 +114,8 @@
             }
             tempResourceIdForRegister.set(resourceId);
             if (!indexMap.containsKey(tempResourceIdForRegister)) {
-                indexMap.put(new MutableLong(resourceId), ((BaseOperationTracker) index.getOperationTracker()));
+                indexMap.put(new MutableLong(resourceId),
+                        ((AbstractLSMIOOperationCallback) index.getIOOperationCallback()));
             }
         }
     }
@@ -122,16 +123,17 @@
     //[Notice] 
     //This method is called sequentially by the LogAppender threads. 
     //However, the indexMap is concurrently read and modified through this method and registerIndexAndCallback()
-    //TODO: fix issues - 591, 609, 612, and 614.
     @Override
     public void setLastLSN(long resourceId, long LSN) {
         synchronized (indexMap) {
             firstLSN.compareAndSet(-1, LSN);
             lastLSN.set(Math.max(lastLSN.get(), LSN));
-            tempResourceIdForSetLSN.set(resourceId);
-            //TODO; create version number tracker and keep LSNs there. 
-            BaseOperationTracker opTracker = indexMap.get(tempResourceIdForSetLSN);
-            opTracker.updateLastLSN(LSN);
+            if (resourceId != -1) {
+                //Non-update log's resourceId is -1.
+                tempResourceIdForSetLSN.set(resourceId);
+                AbstractLSMIOOperationCallback ioOpCallback = indexMap.get(tempResourceIdForSetLSN);
+                ioOpCallback.updateLastLSN(LSN);
+            }
         }
     }
 
@@ -221,4 +223,10 @@
     public LogRecord getLogRecord() {
         return logRecord;
     }
+
+    public void cleanupForAbort() {
+        if (primaryIndexOpTracker != null) {
+            primaryIndexOpTracker.cleanupNumActiveOperationsForAbortedJob(primaryIndexCallback);
+        }
+    }
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
index 01b38c2..07fc152 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -53,7 +53,12 @@
             txnCtx.setTxnState(ITransactionManager.ABORTED);
         }
         try {
-            txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+            if (txnCtx.isWriteTxn()) {
+                LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
+                logRecord.formJobTerminateLogRecord(txnCtx, false);
+                txnSubsystem.getLogManager().log(logRecord);
+                txnSubsystem.getRecoveryManager().rollbackTransaction(txnCtx);
+            }
         } catch (Exception ae) {
             String msg = "Could not complete rollback! System is in an inconsistent state";
             if (LOGGER.isLoggable(Level.SEVERE)) {
@@ -62,6 +67,7 @@
             ae.printStackTrace();
             throw new ACIDException(msg, ae);
         } finally {
+            ((TransactionContext) txnCtx).cleanupForAbort();
             txnSubsystem.getLockManager().releaseLocks(txnCtx);
             transactionContextRepository.remove(txnCtx.getJobId());
         }
@@ -69,20 +75,24 @@
 
     @Override
     public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
-        return getTransactionContext(jobId);
+        return getTransactionContext(jobId, true);
     }
 
     @Override
-    public ITransactionContext getTransactionContext(JobId jobId) throws ACIDException {
+    public ITransactionContext getTransactionContext(JobId jobId, boolean createIfNotExist) throws ACIDException {
         setMaxJobId(jobId.getId());
         ITransactionContext txnCtx = transactionContextRepository.get(jobId);
         if (txnCtx == null) {
-            synchronized (this) {
-                txnCtx = transactionContextRepository.get(jobId);
-                if (txnCtx == null) {
-                    txnCtx = new TransactionContext(jobId, txnSubsystem);
-                    transactionContextRepository.put(jobId, txnCtx);
+            if (createIfNotExist) {
+                synchronized (this) {
+                    txnCtx = transactionContextRepository.get(jobId);
+                    if (txnCtx == null) {
+                        txnCtx = new TransactionContext(jobId, txnSubsystem);
+                        transactionContextRepository.put(jobId, txnCtx);
+                    }
                 }
+            } else {
+                throw new ACIDException("TransactionContext of " + jobId + " doesn't exist.");
             }
         }
         return txnCtx;
@@ -94,7 +104,7 @@
         try {
             if (txnCtx.isWriteTxn()) {
                 LogRecord logRecord = ((TransactionContext) txnCtx).getLogRecord();
-                logRecord.formJobCommitLogRecord(txnCtx);
+                logRecord.formJobTerminateLogRecord(txnCtx, true);
                 txnSubsystem.getLogManager().log(logRecord);
             }
         } catch (Exception ae) {