Fixed issue 96. Got rid of a bunch of AqlCompiledSomethings that have been replaced by the corresponding persistent metadata entities.

git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_stabilization@454 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
index 6b305e2..dad8d0a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/BTreeSearchPOperator.java
@@ -3,17 +3,14 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 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.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.optimizer.rules.am.BTreeJobGenParams;
 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.exceptions.NotImplementedException;
 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;
@@ -61,21 +58,11 @@
         }
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams();
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
-        if (jobGenParams.getIndexKind() != IndexKind.BTREE) {
-            throw new NotImplementedException(jobGenParams.getIndexKind() + " indexes are not implemented.");
-        }
         int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
         int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
         AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
-        AqlCompiledDatasetDecl datasetDecl = metadata.findDataset(jobGenParams.getDatasetName());
-        if (datasetDecl == null) {
-            throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
-        }
-        if (datasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("Trying to run btree search over external dataset ("
-                    + jobGenParams.getDatasetName() + ").");
-        }
+        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
         List<LogicalVariable> outputVars = unnestMap.getVariables();
         if (jobGenParams.getRetainInput()) {
@@ -84,7 +71,7 @@
         }
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = AqlMetadataProvider.buildBtreeRuntime(
                 builder.getJobSpec(), outputVars, opSchema, typeEnv, metadata, context, jobGenParams.getRetainInput(),
-                jobGenParams.getDatasetName(), datasetDecl, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
+                jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
                 jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive());
         builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
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 f6c49ab..ae2559b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -3,14 +3,12 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
-import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 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;
 import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -52,7 +50,7 @@
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
- * Contributes the runtime operator for an unnest-map representing an inverted-index search. 
+ * Contributes the runtime operator for an unnest-map representing an inverted-index search.
  */
 public class InvertedIndexPOperator extends IndexSearchPOperator {
     public InvertedIndexPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
@@ -61,15 +59,13 @@
 
     @Override
     public PhysicalOperatorTag getOperatorTag() {
-        // TODO: Currently, I'm relying on my own version of Algebricks, not the released one.
-        // Need to add this tag in Algebricks.
         return PhysicalOperatorTag.INVERTED_INDEX_SEARCH;
     }
 
     @Override
     public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
             IOperatorSchema opSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
-                    throws AlgebricksException {
+            throws AlgebricksException {
         UnnestMapOperator unnestMapOp = (UnnestMapOperator) op;
         ILogicalExpression unnestExpr = unnestMapOp.getExpressionRef().getValue();
         if (unnestExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
@@ -81,48 +77,40 @@
         }
         InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams();
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
-        
+
         AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
-        AqlCompiledDatasetDecl datasetDecl = metadata.findDataset(jobGenParams.getDatasetName());
-        if (datasetDecl == null) {
-            throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
-        }
-        if (datasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("Trying to run inverted index search over external dataset (" + jobGenParams.getDatasetName() + ").");
-        }
+        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
-        
+
         // Build runtime.
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch = buildInvertedIndexRuntime(metadata,
                 context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
-                jobGenParams.getDatasetName(), datasetDecl, jobGenParams.getIndexName(),
-                jobGenParams.getSearchKeyType(), keyIndexes, jobGenParams.getSearchModifierType(),
-                jobGenParams.getSimilarityThreshold());
+                jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(),
+                keyIndexes, jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold());
         // Contribute operator in hyracks job.
         builder.contributeHyracksOperator(unnestMapOp, invIndexSearch.first);
-        builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);        
+        builder.contributeAlgebricksPartitionConstraint(invIndexSearch.first, invIndexSearch.second);
         ILogicalOperator srcExchange = unnestMapOp.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMapOp, 0);
     }
-    
+
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
-            AqlCompiledMetadataDeclarations metadata, JobGenContext context,
-            JobSpecification jobSpec, UnnestMapOperator unnestMap,
-            IOperatorSchema opSchema, boolean retainInput, String datasetName,
-            AqlCompiledDatasetDecl datasetDecl, String indexName,
-            ATypeTag searchKeyType, int[] keyFields, SearchModifierType searchModifierType,
-            IAlgebricksConstantValue similarityThreshold) throws AlgebricksException {
-        IAObject simThresh = ((AsterixConstantValue)similarityThreshold).getObject();
-        String itemTypeName = datasetDecl.getItemTypeName();
+            AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
+            UnnestMapOperator unnestMap, IOperatorSchema opSchema, boolean retainInput, String datasetName,
+            Dataset dataset, String indexName, ATypeTag searchKeyType, int[] keyFields,
+            SearchModifierType searchModifierType, IAlgebricksConstantValue similarityThreshold)
+            throws AlgebricksException {
+        IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
+        String itemTypeName = dataset.getItemTypeName();
         IAType itemType = metadata.findType(itemTypeName);
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
-        AqlCompiledIndexDecl index = DatasetUtils.findSecondaryIndexByName(datasetDecl, indexName);
-        if (index == null) {
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        if (secondaryIndex == null) {
             throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
                     + datasetName);
         }
-        List<String> secondaryKeyFields = index.getFieldExprs();
+        List<String> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
         int numSecondaryKeys = secondaryKeyFields.size();
         if (numSecondaryKeys != 1) {
             throw new AlgebricksException(
@@ -134,7 +122,7 @@
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recordType = (ARecordType) itemType;
-        Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recordType);
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recordType);
         IAType secondaryKeyType = keyPairType.first;
         if (secondaryKeyType == null) {
             throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
@@ -148,7 +136,7 @@
             tokenComparatorFactories[i] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
             tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
         }
-        
+
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
         List<LogicalVariable> outputVars = unnestMap.getVariables();
         if (retainInput) {
@@ -156,37 +144,33 @@
             VariableUtilities.getLiveVariables(unnestMap, outputVars);
         }
         RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
-        
+
         int start = outputRecDesc.getFieldCount() - numPrimaryKeys;
-        IBinaryComparatorFactory[] invListsComparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(outputVars, start, numPrimaryKeys, typeEnv, context);
-        ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys, typeEnv, context);
-        
-        IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();        
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint;
-        try {
-            secondarySplitsAndConstraint = metadata
-                    .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
-                            datasetName, indexName);
-        } catch (MetadataException e) {
-            throw new AlgebricksException(e);
-        }
-        
+        IBinaryComparatorFactory[] invListsComparatorFactories = JobGenHelper.variablesToAscBinaryComparatorFactories(
+                outputVars, start, numPrimaryKeys, typeEnv, context);
+        ITypeTraits[] invListsTypeTraits = JobGenHelper.variablesToTypeTraits(outputVars, start, numPrimaryKeys,
+                typeEnv, context);
+
+        IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> secondarySplitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders = metadata
                 .getInvertedIndexFileSplitProviders(secondarySplitsAndConstraint.first);
-        
+
         // TODO: Here we assume there is only one search key field.
         int queryField = keyFields[0];
         // Get tokenizer and search modifier factories.
-        IInvertedIndexSearchModifierFactory searchModifierFactory = InvertedIndexAccessMethod.getSearchModifierFactory(searchModifierType, simThresh, index);
-        IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(searchModifierType, searchKeyType, index);
-		InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(
-				jobSpec, queryField, appContext.getStorageManagerInterface(),
-				fileSplitProviders.first, fileSplitProviders.second,
-				appContext.getIndexRegistryProvider(), tokenTypeTraits,
-				tokenComparatorFactories, invListsTypeTraits,
-				invListsComparatorFactories, new BTreeDataflowHelperFactory(),
-				queryTokenizerFactory, searchModifierFactory, outputRecDesc,
-				retainInput, NoOpOperationCallbackProvider.INSTANCE);        
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp, secondarySplitsAndConstraint.second);
+        IInvertedIndexSearchModifierFactory searchModifierFactory = InvertedIndexAccessMethod.getSearchModifierFactory(
+                searchModifierType, simThresh, secondaryIndex);
+        IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(
+                searchModifierType, searchKeyType, secondaryIndex);
+        InvertedIndexSearchOperatorDescriptor invIndexSearchOp = new InvertedIndexSearchOperatorDescriptor(jobSpec,
+                queryField, appContext.getStorageManagerInterface(), fileSplitProviders.first,
+                fileSplitProviders.second, appContext.getIndexRegistryProvider(), tokenTypeTraits,
+                tokenComparatorFactories, invListsTypeTraits, invListsComparatorFactories,
+                new BTreeDataflowHelperFactory(), queryTokenizerFactory, searchModifierFactory, outputRecDesc,
+                retainInput, NoOpOperationCallbackProvider.INSTANCE);
+        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(invIndexSearchOp,
+                secondarySplitsAndConstraint.second);
     }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
index 1832343..27a477c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/RTreeSearchPOperator.java
@@ -1,16 +1,13 @@
 package edu.uci.ics.asterix.algebra.operators.physical;
 
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 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.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.optimizer.rules.am.RTreeJobGenParams;
 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.exceptions.NotImplementedException;
 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;
@@ -56,23 +53,16 @@
 
         RTreeJobGenParams jobGenParams = new RTreeJobGenParams();
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
-        if (jobGenParams.getIndexKind() != IndexKind.RTREE) {
-            throw new NotImplementedException(jobGenParams.getIndexKind() + " indexes are not implemented.");
-        }
 
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(jobGenParams.getDatasetName());
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
         }
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
-            throw new AlgebricksException("Trying to run rtree search over external dataset ("
-                    + jobGenParams.getDatasetName() + ").");
-        }
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = AqlMetadataProvider.buildRtreeRuntime(
-                metadata, context, builder.getJobSpec(), jobGenParams.getDatasetName(), adecl,
+                metadata, context, builder.getJobSpec(), jobGenParams.getDatasetName(), dataset,
                 jobGenParams.getIndexName(), keyIndexes);
         builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index dc5d659..a9de14f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -8,14 +8,13 @@
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlIndex;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -80,8 +79,8 @@
             AssignOperator assignOp = (AssignOperator) op2;
             ILogicalExpression assignExpr = assignOp.getExpressions().get(0).getValue();
             if (assignExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions()
-                        .get(0).getValue();
+                ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions().get(0)
+                        .getValue();
                 fid = funcExpr.getFunctionIdentifier();
             }
         }
@@ -92,30 +91,32 @@
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
         String datasetName = datasetSource.getId().getDatasetName();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return false;
         }
 
         List<LogicalVariable> projectVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getUsedVariables(op1, projectVars);
         // Create operators for secondary index insert/delete.
-        String itemTypeName = adecl.getItemTypeName();
+        String itemTypeName = dataset.getItemTypeName();
         IAType itemType = metadata.findType(itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
-        List<AqlCompiledIndexDecl> secondaryIndexes = DatasetUtils.getSecondaryIndexes(adecl);
-        if (secondaryIndexes.isEmpty()) {
-            return false;
-        }
+        List<Index> indexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         ILogicalOperator currentTop = op1;
-        for (AqlCompiledIndexDecl index : secondaryIndexes) {
-            List<String> secondaryKeyFields = index.getFieldExprs();
+        boolean hasSecondaryIndex = false;
+        for (Index index : indexes) {
+            if (!index.isSecondaryIndex()) {
+                continue;
+            }
+            hasSecondaryIndex = true;
+            List<String> secondaryKeyFields = index.getKeyFieldNames();
             List<LogicalVariable> secondaryKeyVars = new ArrayList<LogicalVariable>();
             List<Mutable<ILogicalExpression>> expressions = new ArrayList<Mutable<ILogicalExpression>>();
             List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<Mutable<ILogicalExpression>>();
@@ -146,7 +147,7 @@
             project.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
             context.computeAndSetTypeEnvironmentForOperator(project);
             context.computeAndSetTypeEnvironmentForOperator(assign);
-            if (index.getKind() == IndexKind.BTREE) {
+            if (index.getIndexType() == IndexType.BTREE) {
                 for (LogicalVariable secondaryKeyVar : secondaryKeyVars) {
                     secondaryExpressions.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
                             secondaryKeyVar)));
@@ -160,9 +161,9 @@
                 indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                 currentTop = indexUpdate;
                 context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
-            } else if (index.getKind() == IndexKind.RTREE) {
-                Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                        secondaryKeyFields.get(0), recType);
+            } else if (index.getIndexType() == IndexType.RTREE) {
+                Pair<IAType, Boolean> keyPairType = Index
+                        .getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
                 IAType spatialType = keyPairType.first;
                 int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
                 int numKeys = dimension * 2;
@@ -199,11 +200,13 @@
                 IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
                         insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
                         insertOp.getOperation());
-                indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));                
-                currentTop = indexUpdate;                
+                indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assignCoordinates));
+                currentTop = indexUpdate;
                 context.computeAndSetTypeEnvironmentForOperator(indexUpdate);
             }
-
+        }
+        if (!hasSecondaryIndex) {
+            return false;
         }
         op0.getInputs().clear();
         op0.getInputs().add(new MutableObject<ILogicalOperator>(currentTop));
@@ -219,7 +222,7 @@
             IAType secondaryKeyType = (IAType) typeEnv.getVarType(secondaryKeyVar);
             if (!isNullableType(secondaryKeyType) && !forceFilter) {
                 continue;
-            }            
+            }
             ScalarFunctionCallExpression isNullFuncExpr = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.IS_NULL),
                     new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
@@ -242,10 +245,10 @@
         }
         return filterExpression;
     }
-    
+
     private boolean isNullableType(IAType type) {
         if (type.getTypeTag() == ATypeTag.UNION) {
-            return ((AUnionType)type).isNullableType();
+            return ((AUnionType) type).isNullableType();
         }
         return false;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
index 5a3d3b8..de0ee38 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushFieldAccessRule.java
@@ -11,11 +11,11 @@
 import edu.uci.ics.asterix.algebra.base.AsterixOperatorAnnotations;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixRuntimeException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 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;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
@@ -55,7 +55,8 @@
     }
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         if (context.checkIfInDontApplySet(this, op)) {
             return false;
@@ -116,11 +117,11 @@
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
         AqlSourceId asid = ((IDataSource<AqlSourceId>) scan.getDataSource()).getId();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(asid.getDatasetName());
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(asid.getDatasetName());
+        if (dataset == null) {
             throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
         }
-        if (adecl.getDatasetType() != DatasetType.INTERNAL && adecl.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
             return false;
         }
         ILogicalExpression e1 = accessFun.getArguments().get(1).getValue();
@@ -134,7 +135,7 @@
             fldName = ((AString) obj).getStringValue();
         } else {
             int pos = ((AInt32) obj).getIntegerValue();
-            String tName = adecl.getItemTypeName();
+            String tName = dataset.getItemTypeName();
             IAType t = metadata.findType(tName);
             if (t.getTypeTag() != ATypeTag.RECORD) {
                 return false;
@@ -146,9 +147,16 @@
             fldName = rt.getFieldNames()[pos];
         }
 
-        List<AqlCompiledIndexDecl> idxList = DatasetUtils.findSecondaryIndexesByOneOfTheKeys(adecl, fldName);
+        List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        boolean hasSecondaryIndex = false;
+        for (Index index : datasetIndexes) {
+            if (index.isSecondaryIndex()) {
+                hasSecondaryIndex = true;
+                break;
+            }
+        }
 
-        return idxList != null && !idxList.isEmpty();
+        return hasSecondaryIndex;
     }
 
     private boolean tryingToPushThroughSelectionWithSameDataSource(AssignOperator access, AbstractLogicalOperator op2) {
@@ -285,12 +293,12 @@
                             AqlSourceId asid = dataSource.getId();
                             AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
                             AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                            AqlCompiledDatasetDecl adecl = metadata.findDataset(asid.getDatasetName());
-                            if (adecl == null) {
+                            Dataset dataset = metadata.findDataset(asid.getDatasetName());
+                            if (dataset == null) {
                                 throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
                             }
-                            if (adecl.getDatasetType() != DatasetType.INTERNAL
-                                    && adecl.getDatasetType() != DatasetType.FEED) {
+                            if (dataset.getDatasetType() != DatasetType.INTERNAL
+                                    && dataset.getDatasetType() != DatasetType.FEED) {
                                 setAsFinal(access, context, finalAnnot);
                                 return false;
                             }
@@ -301,7 +309,7 @@
                                 fldName = ((AString) obj).getStringValue();
                             } else {
                                 int pos = ((AInt32) obj).getIntegerValue();
-                                String tName = adecl.getItemTypeName();
+                                String tName = dataset.getItemTypeName();
                                 IAType t = metadata.findType(tName);
                                 if (t.getTypeTag() != ATypeTag.RECORD) {
                                     return false;
@@ -313,7 +321,7 @@
                                 }
                                 fldName = rt.getFieldNames()[pos];
                             }
-                            int p = DatasetUtils.getPositionOfPartitioningKeyField(adecl, fldName);
+                            int p = DatasetUtils.getPositionOfPartitioningKeyField(dataset, fldName);
                             if (p < 0) { // not one of the partitioning fields
                                 setAsFinal(access, context, finalAnnot);
                                 return false;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index d7ebdc4..74f790f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -9,7 +9,7 @@
 import edu.uci.ics.asterix.algebra.operators.physical.BTreeSearchPOperator;
 import edu.uci.ics.asterix.algebra.operators.physical.InvertedIndexPOperator;
 import edu.uci.ics.asterix.algebra.operators.physical.RTreeSearchPOperator;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+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.om.functions.AsterixBuiltinFunctions;
@@ -157,9 +157,9 @@
                             throw new AlgebricksException("Could not find index " + jobGenParams.getIndexName()
                                     + " for dataset " + dataSourceId);
                         }
-                        IndexKind indexKind = jobGenParams.getIndexKind();
+                        IndexType indexType = jobGenParams.getIndexType();
                         boolean requiresBroadcast = jobGenParams.getRequiresBroadcast();
-                        switch (indexKind) {
+                        switch (indexType) {
                             case BTREE: {
                                 op.setPhysicalOperator(new BTreeSearchPOperator(dsi, requiresBroadcast));
                                 break;
@@ -177,7 +177,7 @@
                                 break;
                             }
                             default: {
-                                throw new NotImplementedException(indexKind + " indexes are not implemented.");
+                                throw new NotImplementedException(indexType + " indexes are not implemented.");
                             }
                         }
                     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
index fb0c656..f25a671 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/UnnestToDataScanRule.java
@@ -6,13 +6,12 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledFeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 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.declared.ExternalFeedDataSource;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -78,8 +77,8 @@
 
                 AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
                 AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
-                if (acdd == null) {
+                Dataset dataset = metadata.findDataset(datasetName);
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
 
@@ -87,9 +86,8 @@
 
                 ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
 
-                if (acdd.getDatasetType() == DatasetType.INTERNAL || acdd.getDatasetType() == DatasetType.FEED) {
-
-                    int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(acdd).size();
+                if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
+                    int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
                     for (int i = 0; i < numPrimaryKeys; i++) {
                         v.add(context.newVar());
                     }
@@ -105,7 +103,7 @@
 
                 return true;
             }
-            
+
             if (fid.equals(AsterixBuiltinFunctions.FEED_INGEST)) {
                 if (unnest.getPositionalVariable() != null) {
                     throw new AlgebricksException("No positional variables are allowed over datasets.");
@@ -127,14 +125,14 @@
 
                 AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
                 AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
+                Dataset dataset = metadata.findDataset(datasetName);
 
-                if (acdd == null) {
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
 
-                if (acdd.getDatasetType() != DatasetType.FEED) {
-                    throw new IllegalArgumentException("invalid dataset type:" + acdd.getDatasetType());
+                if (dataset.getDatasetType() != DatasetType.FEED) {
+                    throw new IllegalArgumentException("invalid dataset type:" + dataset.getDatasetType());
                 }
 
                 AqlSourceId asid = new AqlSourceId(metadata.getDataverseName(), datasetName);
@@ -149,7 +147,7 @@
 
                 v.add(unnest.getVariable());
 
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, acdd,
+                DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, dataset,
                         metadata));
 
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
@@ -161,36 +159,22 @@
                 return true;
             }
         }
-        
 
         return false;
     }
 
-    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, AqlCompiledDatasetDecl acdd,
+    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
             AqlCompiledMetadataDeclarations metadata) throws AlgebricksException {
-
-        AqlCompiledFeedDatasetDetails feedDetails = (AqlCompiledFeedDatasetDetails) acdd.getAqlCompiledDatasetDetails();
-
         if (!aqlId.getDataverseName().equals(metadata.getDataverseName())) {
             return null;
         }
-
-        String tName = acdd.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(tName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
-        AqlCompiledDatasetDecl dummySourceDecl = new AqlCompiledDatasetDecl(acdd.getName(), tName,
-                DatasetType.EXTERNAL, feedDetails);
-
-        ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dummySourceDecl, itemType,
+        String tName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(tName);
+        ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dataset, itemType,
                 AqlDataSource.AqlDataSourceType.EXTERNAL_FEED);
         return extDataSource;
     }
-    
+
     public void addPrimaryKey(List<LogicalVariable> scanVariables, IOptimizationContext context) {
         int n = scanVariables.size();
         List<LogicalVariable> head = new ArrayList<LogicalVariable>(scanVariables.subList(0, n - 1));
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index e4f795b..b378ed7 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -7,14 +7,17 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+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.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
 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.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -33,10 +36,13 @@
  * Class that embodies the commonalities between rewrite rules for access methods.
  */
 public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
-    
+
+    private AqlCompiledMetadataDeclarations metadata;
+
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
-    
-    protected static void registerAccessMethod(IAccessMethod accessMethod, Map<FunctionIdentifier, List<IAccessMethod>> accessMethods) {
+
+    protected static void registerAccessMethod(IAccessMethod accessMethod,
+            Map<FunctionIdentifier, List<IAccessMethod>> accessMethods) {
         List<FunctionIdentifier> funcs = accessMethod.getOptimizableFunctions();
         for (FunctionIdentifier funcIdent : funcs) {
             List<IAccessMethod> l = accessMethods.get(funcIdent);
@@ -47,16 +53,23 @@
             l.add(accessMethod);
         }
     }
-    
+
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         return false;
     }
-    
-    protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+
+    protected void setMetadataDeclarations(IOptimizationContext context) {
+        AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+        metadata = metadataProvider.getMetadataDeclarations();
+    }
+
+    protected void fillSubTreeIndexExprs(OptimizableOperatorSubTree subTree,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) throws AlgebricksException {
         // The assign may be null if there is only a filter on the primary index key.
         // Match variables from lowest assign which comes directly after the dataset scan.
-        List<LogicalVariable> varList = (!subTree.assigns.isEmpty()) ? subTree.assigns.get(subTree.assigns.size() - 1).getVariables() : subTree.dataSourceScan.getVariables();
+        List<LogicalVariable> varList = (!subTree.assigns.isEmpty()) ? subTree.assigns.get(subTree.assigns.size() - 1)
+                .getVariables() : subTree.dataSourceScan.getVariables();
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         // Check applicability of indexes by access method type.
         while (amIt.hasNext()) {
@@ -80,26 +93,25 @@
             }
         }
     }
-    
+
     /**
      * Simply picks the first index that it finds.
      * TODO: Improve this decision process by making it more systematic.
      */
-    protected Pair<IAccessMethod, AqlCompiledIndexDecl> chooseIndex(
-            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+    protected Pair<IAccessMethod, Index> chooseIndex(Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         while (amIt.hasNext()) {
             Map.Entry<IAccessMethod, AccessMethodAnalysisContext> amEntry = amIt.next();
             AccessMethodAnalysisContext analysisCtx = amEntry.getValue();
-            Iterator<Map.Entry<AqlCompiledIndexDecl, List<Integer>>> indexIt = analysisCtx.indexExprs.entrySet().iterator();
+            Iterator<Map.Entry<Index, List<Integer>>> indexIt = analysisCtx.indexExprs.entrySet().iterator();
             if (indexIt.hasNext()) {
-                Map.Entry<AqlCompiledIndexDecl, List<Integer>> indexEntry = indexIt.next();
-                return new Pair<IAccessMethod, AqlCompiledIndexDecl>(amEntry.getKey(), indexEntry.getKey());
+                Map.Entry<Index, List<Integer>> indexEntry = indexIt.next();
+                return new Pair<IAccessMethod, Index>(amEntry.getKey(), indexEntry.getKey());
             }
         }
         return null;
     }
-    
+
     /**
      * Removes irrelevant access methods candidates, based on whether the
      * expressions in the query match those in the index. For example, some
@@ -107,18 +119,17 @@
      * only require a match on a prefix of fields to be applicable. This methods
      * removes all index candidates indexExprs that are definitely not
      * applicable according to the expressions involved.
-     * 
      */
     public void pruneIndexCandidates(IAccessMethod accessMethod, AccessMethodAnalysisContext analysisCtx) {
-        Iterator<Map.Entry<AqlCompiledIndexDecl, List<Integer>>> it = analysisCtx.indexExprs.entrySet().iterator();
+        Iterator<Map.Entry<Index, List<Integer>>> it = analysisCtx.indexExprs.entrySet().iterator();
         while (it.hasNext()) {
-            Map.Entry<AqlCompiledIndexDecl, List<Integer>> entry = it.next();
-            AqlCompiledIndexDecl index = entry.getKey();
+            Map.Entry<Index, List<Integer>> entry = it.next();
+            Index index = entry.getKey();
             Iterator<Integer> exprsIter = entry.getValue().iterator();
             boolean allUsed = true;
             int lastFieldMatched = -1;
-            for (int i = 0; i < index.getFieldExprs().size(); i++) {
-                String keyField = index.getFieldExprs().get(i);
+            for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
+                String keyField = index.getKeyFieldNames().get(i);
                 boolean foundKeyField = false;
                 while (exprsIter.hasNext()) {
                     Integer ix = exprsIter.next();
@@ -140,7 +151,7 @@
                 if (!foundKeyField) {
                     allUsed = false;
                     break;
-                }                
+                }
             }
             // If the access method requires all exprs to be matched but they are not, remove this candidate.
             if (!allUsed && accessMethod.matchAllIndexExprs()) {
@@ -154,37 +165,39 @@
             }
         }
     }
-    
+
     /**
      * Analyzes the given selection condition, filling analyzedAMs with applicable access method types.
      * At this point we are not yet consulting the metadata whether an actual index exists or not.
      */
-    protected boolean analyzeCondition(ILogicalExpression cond, List<AssignOperator> assigns, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+    protected boolean analyzeCondition(ILogicalExpression cond, List<AssignOperator> assigns,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
         AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) cond;
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
         // Don't consider optimizing a disjunctive condition with an index (too complicated for now).
         if (funcIdent == AlgebricksBuiltinFunctions.OR) {
             return false;
         }
-        boolean found = analyzeFunctionExpr(funcExpr, assigns, analyzedAMs);        
+        boolean found = analyzeFunctionExpr(funcExpr, assigns, analyzedAMs);
         for (Mutable<ILogicalExpression> arg : funcExpr.getArguments()) {
             ILogicalExpression argExpr = arg.getValue();
             if (argExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
                 continue;
             }
             AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
-            boolean matchFound = analyzeFunctionExpr(argFuncExpr, assigns, analyzedAMs); 
+            boolean matchFound = analyzeFunctionExpr(argFuncExpr, assigns, analyzedAMs);
             found = found || matchFound;
         }
         return found;
     }
-    
+
     /**
      * Finds applicable access methods for the given function expression based
      * on the function identifier, and an analysis of the function's arguments.
      * Updates the analyzedAMs accordingly.
      */
-    protected boolean analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
+    protected boolean analyzeFunctionExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs) {
         FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
         if (funcIdent == AlgebricksBuiltinFunctions.AND) {
             return false;
@@ -197,7 +210,7 @@
         boolean atLeastOneMatchFound = false;
         // Place holder for a new analysis context in case we need one.
         AccessMethodAnalysisContext newAnalysisCtx = new AccessMethodAnalysisContext();
-        for(IAccessMethod accessMethod : relevantAMs) {
+        for (IAccessMethod accessMethod : relevantAMs) {
             AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(accessMethod);
             // Use the current place holder.
             if (analysisCtx == null) {
@@ -216,38 +229,38 @@
         }
         return atLeastOneMatchFound;
     }
-    
+
     /**
-     * Finds secondary indexes whose keys include fieldName, and adds a mapping in analysisCtx.indexEsprs 
+     * Finds secondary indexes whose keys include fieldName, and adds a mapping in analysisCtx.indexEsprs
      * from that index to the a corresponding optimizable function expression.
      * 
      * @return true if a candidate index was added to foundIndexExprs, false
      *         otherwise
+     * @throws AlgebricksException
      */
-    protected boolean fillIndexExprs(String fieldName, int matchedFuncExprIndex,
-            AqlCompiledDatasetDecl datasetDecl, AccessMethodAnalysisContext analysisCtx) {
-        AqlCompiledIndexDecl primaryIndexDecl = DatasetUtils.getPrimaryIndex(datasetDecl);
-        List<String> primaryIndexFields = primaryIndexDecl.getFieldExprs();     
-        List<AqlCompiledIndexDecl> indexCandidates = DatasetUtils.findSecondaryIndexesByOneOfTheKeys(datasetDecl, fieldName);
-        // Check whether the primary index is a candidate. If so, add it to the list.
-        if (primaryIndexFields.contains(fieldName)) {
-            if (indexCandidates == null) {
-                indexCandidates = new ArrayList<AqlCompiledIndexDecl>(1);
+    protected boolean fillIndexExprs(String fieldName, int matchedFuncExprIndex, Dataset dataset,
+            AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+        List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        List<Index> indexCandidates = new ArrayList<Index>();
+        // Add an index to the candidates if one of the indexed fields is fieldName.
+        for (Index index : datasetIndexes) {
+            if (index.getKeyFieldNames().contains(fieldName)) {
+                indexCandidates.add(index);
             }
-            indexCandidates.add(primaryIndexDecl);
         }
         // No index candidates for fieldName.
-        if (indexCandidates == null) {
+        if (indexCandidates.isEmpty()) {
             return false;
         }
         // Go through the candidates and fill indexExprs.
-        for (AqlCompiledIndexDecl index : indexCandidates) {
-            analysisCtx.addIndexExpr(datasetDecl, index, matchedFuncExprIndex);
+        for (Index index : indexCandidates) {
+            analysisCtx.addIndexExpr(dataset, index, matchedFuncExprIndex);
         }
         return true;
     }
 
-    protected void fillAllIndexExprs(List<LogicalVariable> varList, OptimizableOperatorSubTree subTree, AccessMethodAnalysisContext analysisCtx) {
+    protected void fillAllIndexExprs(List<LogicalVariable> varList, OptimizableOperatorSubTree subTree,
+            AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
         for (int optFuncExprIndex = 0; optFuncExprIndex < analysisCtx.matchedFuncExprs.size(); optFuncExprIndex++) {
             for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
                 LogicalVariable var = varList.get(varIndex);
@@ -263,7 +276,8 @@
                     // Get the fieldName corresponding to the assigned variable at varIndex
                     // from the assign operator right above the datasource scan.
                     // If the expr at varIndex is not a fieldAccess we get back null.
-                    fieldName = getFieldNameOfFieldAccess(subTree.assigns.get(subTree.assigns.size() - 1), subTree.recordType, varIndex);
+                    fieldName = getFieldNameOfFieldAccess(subTree.assigns.get(subTree.assigns.size() - 1),
+                            subTree.recordType, varIndex);
                     if (fieldName == null) {
                         continue;
                     }
@@ -274,24 +288,24 @@
                         break;
                     }
                     // The variable value is one of the partitioning fields.
-                    fieldName = DatasetUtils.getPartitioningExpressions(subTree.datasetDecl).get(varIndex);
+                    fieldName = DatasetUtils.getPartitioningKeys(subTree.dataset).get(varIndex);
                 }
                 // Set the fieldName in the corresponding matched function expression, and remember matching subtree.
                 optFuncExpr.setFieldName(funcVarIndex, fieldName);
                 optFuncExpr.setOptimizableSubTree(funcVarIndex, subTree);
-                fillIndexExprs(fieldName, optFuncExprIndex, subTree.datasetDecl, analysisCtx);
+                fillIndexExprs(fieldName, optFuncExprIndex, subTree.dataset, analysisCtx);
             }
         }
     }
-    
+
     /**
      * Returns the field name corresponding to the assigned variable at varIndex.
      * Returns null if the expr at varIndex is not a field access function.
      */
     protected String getFieldNameOfFieldAccess(AssignOperator assign, ARecordType recordType, int varIndex) {
         // Get expression corresponding to var at varIndex.
-        AbstractLogicalExpression assignExpr = (AbstractLogicalExpression) assign.getExpressions()
-                .get(varIndex).getValue();
+        AbstractLogicalExpression assignExpr = (AbstractLogicalExpression) assign.getExpressions().get(varIndex)
+                .getValue();
         if (assignExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return null;
         }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index b977e29..fc45063 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -4,35 +4,35 @@
 import java.util.HashMap;
 import java.util.List;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
 
 /**
  * Context for analyzing the applicability of a single access method.
  */
 public class AccessMethodAnalysisContext {
-    
+
     public List<IOptimizableFuncExpr> matchedFuncExprs = new ArrayList<IOptimizableFuncExpr>();
-    
+
     // Contains candidate indexes and a list of integers that index into matchedFuncExprs.
     // We are mapping from candidate indexes to a list of function expressions 
     // that match one of the index's expressions.
-    public HashMap<AqlCompiledIndexDecl, List<Integer>> indexExprs = new HashMap<AqlCompiledIndexDecl, List<Integer>>();
-    
+    public HashMap<Index, List<Integer>> indexExprs = new HashMap<Index, List<Integer>>();
+
     // Maps from index to the dataset it is indexing.
-    public HashMap<AqlCompiledIndexDecl, AqlCompiledDatasetDecl> indexDatasetMap = new HashMap<AqlCompiledIndexDecl, AqlCompiledDatasetDecl>();
-    
-    public void addIndexExpr(AqlCompiledDatasetDecl dataset,  AqlCompiledIndexDecl index, Integer exprIndex) {
-    	List<Integer> exprs = indexExprs.get(index);
-    	if (exprs == null) {
-    	    exprs = new ArrayList<Integer>();
-    		indexExprs.put(index, exprs);
-    	}
-    	exprs.add(exprIndex);
-    	indexDatasetMap.put(index, dataset);
+    public HashMap<Index, Dataset> indexDatasetMap = new HashMap<Index, Dataset>();
+
+    public void addIndexExpr(Dataset dataset, Index index, Integer exprIndex) {
+        List<Integer> exprs = indexExprs.get(index);
+        if (exprs == null) {
+            exprs = new ArrayList<Integer>();
+            indexExprs.put(index, exprs);
+        }
+        exprs.add(exprIndex);
+        indexDatasetMap.put(index, dataset);
     }
-    
-    public List<Integer> getIndexExprs(AqlCompiledIndexDecl index) {
+
+    public List<Integer> getIndexExprs(Index index) {
         return indexExprs.get(index);
     }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
index 22ed47a..af30163 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/AccessMethodJobGenParams.java
@@ -5,7 +5,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -19,27 +19,28 @@
  */
 public class AccessMethodJobGenParams {
     protected String indexName;
-    protected IndexKind indexKind;
+    protected IndexType indexType;
     protected String datasetName;
     protected boolean retainInput;
     protected boolean requiresBroadcast;
-    
+
     private final int NUM_PARAMS = 5;
-    
+
     public AccessMethodJobGenParams() {
     }
-    
-    public AccessMethodJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput, boolean requiresBroadcast) {
+
+    public AccessMethodJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+            boolean requiresBroadcast) {
         this.indexName = indexName;
-        this.indexKind = indexKind;
+        this.indexType = indexType;
         this.datasetName = datasetName;
         this.retainInput = retainInput;
         this.requiresBroadcast = requiresBroadcast;
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(indexName)));
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(indexKind.ordinal())));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(indexType.ordinal())));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(datasetName)));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(retainInput)));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(requiresBroadcast)));
@@ -47,18 +48,18 @@
 
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         indexName = AccessMethodUtils.getStringConstant(funcArgs.get(0));
-        indexKind = IndexKind.values()[AccessMethodUtils.getInt32Constant(funcArgs.get(1))];
+        indexType = IndexType.values()[AccessMethodUtils.getInt32Constant(funcArgs.get(1))];
         datasetName = AccessMethodUtils.getStringConstant(funcArgs.get(2));
         retainInput = AccessMethodUtils.getBooleanConstant(funcArgs.get(3));
         requiresBroadcast = AccessMethodUtils.getBooleanConstant(funcArgs.get(4));
     }
-    
+
     public String getIndexName() {
         return indexName;
     }
 
-    public IndexKind getIndexKind() {
-        return indexKind;
+    public IndexType getIndexType() {
+        return indexType;
     }
 
     public String getDatasetName() {
@@ -72,7 +73,7 @@
     public boolean getRequiresBroadcast() {
         return requiresBroadcast;
     }
-    
+
     protected void writeVarList(List<LogicalVariable> varList, List<Mutable<ILogicalExpression>> funcArgs) {
         Mutable<ILogicalExpression> numKeysRef = new MutableObject<ILogicalExpression>(new ConstantExpression(
                 new AsterixConstantValue(new AInt32(varList.size()))));
@@ -83,7 +84,7 @@
             funcArgs.add(keyVarRef);
         }
     }
-    
+
     protected int readVarList(List<Mutable<ILogicalExpression>> funcArgs, int index, List<LogicalVariable> varList) {
         int numLowKeys = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
         if (numLowKeys > 0) {
@@ -95,7 +96,7 @@
         }
         return index + numLowKeys + 1;
     }
-    
+
     protected int getNumParams() {
         return NUM_PARAMS;
     }
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 305b858..cd3712d 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
@@ -7,9 +7,9 @@
 import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+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;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.ABoolean;
 import edu.uci.ics.asterix.om.base.AInt32;
@@ -22,7 +22,6 @@
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -40,55 +39,51 @@
 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;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 
 /**
- * Static helper functions for rewriting plans using indexes. 
+ * Static helper functions for rewriting plans using indexes.
  */
 public class AccessMethodUtils {
-    public static void appendPrimaryIndexTypes(AqlCompiledDatasetDecl datasetDecl, IAType itemType, List<Object> target) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(datasetDecl);
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> t : partitioningFunctions) {
-            target.add(t.third);
+    public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, List<Object> target) {
+        ARecordType recordType = (ARecordType) itemType;
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            target.add(recordType.getFieldType(partitioningKey));
         }
         target.add(itemType);
     }
-	
-	public static ConstantExpression createStringConstant(String str) {
+
+    public static ConstantExpression createStringConstant(String str) {
         return new ConstantExpression(new AsterixConstantValue(new AString(str)));
     }
-	
-	public static ConstantExpression createInt32Constant(int i) {
+
+    public static ConstantExpression createInt32Constant(int i) {
         return new ConstantExpression(new AsterixConstantValue(new AInt32(i)));
     }
-	
-	public static ConstantExpression createBooleanConstant(boolean b) {
+
+    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 String getStringConstant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
-                .getValue()).getObject();
-        return ((AString)obj).getStringValue();
+
+    public static String getStringConstant(Mutable<ILogicalExpression> expr) {
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((AString) obj).getStringValue();
     }
-	
+
     public static int getInt32Constant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
-                .getValue()).getObject();
-        return ((AInt32)obj).getIntegerValue();
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((AInt32) obj).getIntegerValue();
     }
-    
+
     public static boolean getBooleanConstant(Mutable<ILogicalExpression> expr) {
-        IAObject obj = ((AsterixConstantValue)((ConstantExpression) expr.getValue())
-                .getValue()).getObject();
-        return ((ABoolean)obj).getBoolean();
+        IAObject obj = ((AsterixConstantValue) ((ConstantExpression) expr.getValue()).getValue()).getObject();
+        return ((ABoolean) obj).getBoolean();
     }
-	
+
     public static boolean analyzeFuncExprArgsForOneConstAndVar(AbstractFunctionCallExpression funcExpr,
             AccessMethodAnalysisContext analysisCtx) {
         IAlgebricksConstantValue constFilterVal = null;
@@ -114,48 +109,49 @@
         analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
         return true;
     }
-    
-    public static int getNumSecondaryKeys(AqlCompiledDatasetDecl datasetDecl, AqlCompiledIndexDecl indexDecl,
-            ARecordType recordType) throws AlgebricksException {
-        switch (indexDecl.getKind()) {
+
+    public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
+        switch (index.getIndexType()) {
             case BTREE:
             case WORD_INVIX:
             case NGRAM_INVIX: {
-                return indexDecl.getFieldExprs().size();
+                return index.getKeyFieldNames().size();
             }
             case RTREE: {
-            	Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(indexDecl.getFieldExprs().get(0), recordType);
+                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: " + indexDecl.getKind());
+                throw new AlgebricksException("Unknown index kind: " + index.getIndexType());
             }
         }
     }
-    
+
     /**
      * Appends the types of the fields produced by the given secondary index to dest.
      */
-    public static void appendSecondaryIndexTypes(AqlCompiledDatasetDecl datasetDecl, ARecordType recordType,
-            AqlCompiledIndexDecl indexDecl, boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
+    public static void appendSecondaryIndexTypes(Dataset dataset, ARecordType recordType, Index index,
+            boolean primaryKeysOnly, List<Object> dest) throws AlgebricksException {
         if (!primaryKeysOnly) {
-            switch (indexDecl.getKind()) {
+            switch (index.getIndexType()) {
                 case BTREE:
                 case WORD_INVIX:
                 case NGRAM_INVIX: {
-                    for (String sk : indexDecl.getFieldExprs()) {
-                    	Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(sk, recordType);
+                    for (String sk : index.getKeyFieldNames()) {
+                        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(sk, recordType);
                         dest.add(keyPairType.first);
                     }
                     break;
                 }
                 case RTREE: {
-                	Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(indexDecl.getFieldExprs().get(0), recordType);
+                    Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(index.getKeyFieldNames()
+                            .get(0), recordType);
                     IAType keyType = keyPairType.first;
                     IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
-                    int numKeys = getNumSecondaryKeys(datasetDecl, indexDecl, recordType);
+                    int numKeys = getNumSecondaryKeys(index, recordType);
                     for (int i = 0; i < numKeys; i++) {
                         dest.add(nestedKeyType);
                     }
@@ -164,26 +160,25 @@
             }
         }
         // Primary keys.
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> t : DatasetUtils
-                .getPartitioningFunctions(datasetDecl)) {
-            dest.add(t.third);
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            dest.add(recordType.getFieldType(partitioningKey));
         }
     }
-    
-    public static void appendSecondaryIndexOutputVars(AqlCompiledDatasetDecl datasetDecl, ARecordType recordType,
-            AqlCompiledIndexDecl indexDecl, boolean primaryKeysOnly, IOptimizationContext context,
-            List<LogicalVariable> dest) throws AlgebricksException {
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
-        int numSecondaryKeys = getNumSecondaryKeys(datasetDecl, indexDecl, recordType);
+
+    public static void appendSecondaryIndexOutputVars(Dataset dataset, ARecordType recordType, Index index,
+            boolean primaryKeysOnly, IOptimizationContext context, List<LogicalVariable> dest)
+            throws AlgebricksException {
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+        int numSecondaryKeys = getNumSecondaryKeys(index, recordType);
         int numVars = (primaryKeysOnly) ? numPrimaryKeys : numPrimaryKeys + numSecondaryKeys;
         for (int i = 0; i < numVars; i++) {
             dest.add(context.newVar());
         }
     }
-    
-    public static List<LogicalVariable> getPrimaryKeyVarsFromUnnestMap(AqlCompiledDatasetDecl datasetDecl,
-            ILogicalOperator unnestMapOp) {
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+
+    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.
@@ -194,11 +189,10 @@
         }
         return primaryKeyVars;
     }
-    
-    public static UnnestMapOperator createSecondaryIndexUnnestMap(AqlCompiledDatasetDecl datasetDecl,
-            ARecordType recordType, AqlCompiledIndexDecl indexDecl, ILogicalOperator inputOp,
-            AccessMethodJobGenParams jobGenParams, IOptimizationContext context, boolean outputPrimaryKeysOnly,
-            boolean retainInput) throws AlgebricksException {
+
+    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);
@@ -206,32 +200,35 @@
         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(datasetDecl, recordType, indexDecl, outputPrimaryKeysOnly, context, secondaryIndexUnnestVars);
-        appendSecondaryIndexTypes(datasetDecl, recordType, indexDecl, outputPrimaryKeysOnly, secondaryIndexOutputTypes);        
+        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);
+        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));        
+        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, AqlCompiledDatasetDecl datasetDecl, 
-            ARecordType recordType, ILogicalOperator inputOp,
-            IOptimizationContext context, boolean sortPrimaryKeys, boolean retainInput, boolean requiresBroadcast) throws AlgebricksException {        
-        List<LogicalVariable> primaryKeyVars = AccessMethodUtils.getPrimaryKeyVarsFromUnnestMap(datasetDecl, inputOp);
+
+    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));
+                Mutable<ILogicalExpression> vRef = new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(pkVar));
                 order.getOrderExpressions().add(
                         new Pair<IOrder, Mutable<ILogicalExpression>>(OrderOperator.ASC_ORDER, vRef));
             }
@@ -242,7 +239,8 @@
         }
         // 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(datasetDecl.getName(), IndexKind.BTREE, datasetDecl.getName(), retainInput, requiresBroadcast);
+        BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
+                dataset.getDatasetName(), retainInput, requiresBroadcast);
         // Set low/high inclusive to true for a point lookup.
         jobGenParams.setLowKeyInclusive(true);
         jobGenParams.setHighKeyInclusive(true);
@@ -254,14 +252,15 @@
         List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
         // Append output variables/types generated by the primary-index search (not forwarded from input).
         primaryIndexUnnestVars.addAll(dataSourceScan.getVariables());
-        appendPrimaryIndexTypes(datasetDecl, recordType, primaryIndexOutputTypes);
+        appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
         // 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);
+        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);
+        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));
@@ -272,5 +271,5 @@
         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 25902bc..414dca3 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
@@ -9,10 +9,9 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+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;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -37,15 +36,19 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
 /**
- * Class for helping rewrite rules to choose and apply BTree indexes.  
+ * Class for helping rewrite rules to choose and apply BTree indexes.
  */
 public class BTreeAccessMethod implements IAccessMethod {
 
     // Describes whether a search predicate is an open/closed interval.
     private enum LimitType {
-        LOW_INCLUSIVE, LOW_EXCLUSIVE, HIGH_INCLUSIVE, HIGH_EXCLUSIVE, EQUAL
+        LOW_INCLUSIVE,
+        LOW_EXCLUSIVE,
+        HIGH_INCLUSIVE,
+        HIGH_EXCLUSIVE,
+        EQUAL
     }
-    
+
     // TODO: There is some redundancy here, since these are listed in AlgebricksBuiltinFunctions as well.
     private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
     static {
@@ -56,16 +59,17 @@
         funcIdents.add(AlgebricksBuiltinFunctions.GT);
         funcIdents.add(AlgebricksBuiltinFunctions.NEQ);
     }
-    
+
     public static BTreeAccessMethod INSTANCE = new BTreeAccessMethod();
-    
+
     @Override
     public List<FunctionIdentifier> getOptimizableFunctions() {
         return funcIdents;
     }
-    
+
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
     }
 
@@ -81,10 +85,10 @@
     }
 
     @Override
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) 
-                    throws AlgebricksException {
-        AqlCompiledDatasetDecl datasetDecl = subTree.datasetDecl;
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException {
+        Dataset dataset = subTree.dataset;
         ARecordType recordType = subTree.recordType;
         SelectOperator select = (SelectOperator) selectRef.getValue();
         DataSourceScanOperator dataSourceScan = subTree.dataSourceScan;
@@ -92,9 +96,9 @@
         AssignOperator assign = null;
         if (assignRef != null) {
             assign = (AssignOperator) assignRef.getValue();
-        }        
-        int numSecondaryKeys = chosenIndex.getFieldExprs().size();        
-        
+        }
+        int numSecondaryKeys = chosenIndex.getKeyFieldNames().size();
+
         // Info on high and low keys for the BTree search predicate.
         IAlgebricksConstantValue[] lowKeyConstants = new IAlgebricksConstantValue[numSecondaryKeys];
         IAlgebricksConstantValue[] highKeyConstants = new IAlgebricksConstantValue[numSecondaryKeys];
@@ -102,7 +106,7 @@
         LimitType[] highKeyLimits = new LimitType[numSecondaryKeys];
         boolean[] lowKeyInclusive = new boolean[numSecondaryKeys];
         boolean[] highKeyInclusive = new boolean[numSecondaryKeys];
-        
+
         List<Integer> exprList = analysisCtx.indexExprs.get(chosenIndex);
         List<IOptimizableFuncExpr> matchedFuncExprs = analysisCtx.matchedFuncExprs;
         // List of function expressions that will be replaced by the secondary-index search.
@@ -117,7 +121,7 @@
         for (Integer exprIndex : exprList) {
             // Position of the field of matchedFuncExprs.get(exprIndex) in the chosen index's indexed exprs.
             IOptimizableFuncExpr optFuncExpr = matchedFuncExprs.get(exprIndex);
-            int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getFieldExprs());
+            int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getKeyFieldNames());
             if (keyPos < 0) {
                 throw new InternalError();
             }
@@ -201,10 +205,12 @@
             if (lowKeyInclusive[i] != lowKeyInclusive[0] || highKeyInclusive[i] != highKeyInclusive[0]) {
                 return false;
             }
-            if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null && lowKeyLimits[i] == null) {
+            if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null
+                    && lowKeyLimits[i] == null) {
                 return false;
             }
-            if (highKeyLimits[0] == null && highKeyLimits[i] != null || highKeyLimits[0] != null && highKeyLimits[i] == null) {
+            if (highKeyLimits[0] == null && highKeyLimits[i] != null || highKeyLimits[0] != null
+                    && highKeyLimits[i] == null) {
                 return false;
             }
         }
@@ -214,7 +220,7 @@
         if (highKeyLimits[0] == null) {
             highKeyInclusive[0] = true;
         }
-        
+
         // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index search.
         // List of variables for the assign.
         ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
@@ -222,30 +228,32 @@
         ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
         int numLowKeys = createKeyVarsAndExprs(lowKeyLimits, lowKeyConstants, keyExprList, keyVarList, context);
         int numHighKeys = createKeyVarsAndExprs(highKeyLimits, highKeyConstants, keyExprList, keyVarList, context);
-        
-        BTreeJobGenParams jobGenParams = new BTreeJobGenParams(chosenIndex.getIndexName(), IndexKind.BTREE, datasetDecl.getName(), false, false);
+
+        BTreeJobGenParams jobGenParams = new BTreeJobGenParams(chosenIndex.getIndexName(), IndexType.BTREE,
+                dataset.getDatasetName(), false, false);
         jobGenParams.setLowKeyInclusive(lowKeyInclusive[0]);
         jobGenParams.setHighKeyInclusive(highKeyInclusive[0]);
         jobGenParams.setLowKeyVarList(keyVarList, 0, numLowKeys);
         jobGenParams.setHighKeyVarList(keyVarList, numLowKeys, numHighKeys);
-        
+
         // Assign operator that sets the secondary-index search-key fields.
         AssignOperator assignSearchKeys = new AssignOperator(keyVarList, keyExprList);
         // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
         assignSearchKeys.getInputs().add(dataSourceScan.getInputs().get(0));
         assignSearchKeys.setExecutionMode(dataSourceScan.getExecutionMode());
-        
-        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(datasetDecl,
-                recordType, chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
-        
+
+        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+                chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
+
         // Generate the rest of the upstream plan which feeds the search results into the primary index.        
         UnnestMapOperator primaryIndexUnnestOp;
-        boolean isPrimaryIndex = chosenIndex == DatasetUtils.getPrimaryIndex(datasetDecl);
+        boolean isPrimaryIndex = chosenIndex.getIndexName().equals(dataset.getDatasetName());
         if (!isPrimaryIndex) {
-            primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, datasetDecl, recordType, secondaryIndexUnnestOp, context, true, false, false);
+            primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
+                    secondaryIndexUnnestOp, context, true, false, false);
         } else {
             List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
-            AccessMethodUtils.appendPrimaryIndexTypes(datasetDecl, recordType, primaryIndexOutputTypes);
+            AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
             primaryIndexUnnestOp = new UnnestMapOperator(dataSourceScan.getVariables(),
                     secondaryIndexUnnestOp.getExpressionRef(), primaryIndexOutputTypes, false);
             primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(assignSearchKeys));
@@ -276,19 +284,18 @@
         }
         return true;
     }
-    
+
     @Override
     public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
         // TODO: Implement this.
         return false;
     }
 
     private int createKeyVarsAndExprs(LimitType[] keyLimits, IAlgebricksConstantValue[] keyConstants,
             ArrayList<Mutable<ILogicalExpression>> keyExprList, ArrayList<LogicalVariable> keyVarList,
-            IOptimizationContext context) {        
+            IOptimizationContext context) {
         if (keyLimits[0] == null) {
             return 0;
         }
@@ -300,8 +307,9 @@
         }
         return numKeys;
     }
-    
-    private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs) {
+
+    private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs,
+            List<Mutable<ILogicalExpression>> remainingFuncExprs) {
         remainingFuncExprs.clear();
         if (replacedFuncExprs.isEmpty()) {
             return;
@@ -334,7 +342,7 @@
             }
         }
     }
-    
+
     private <T> int indexOf(T value, List<T> coll) {
         int i = 0;
         for (T member : coll) {
@@ -345,9 +353,10 @@
         }
         return -1;
     }
-    
+
     private LimitType getLimitType(IOptimizableFuncExpr optFuncExpr) {
-        ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr().getFunctionIdentifier());
+        ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(optFuncExpr.getFuncExpr()
+                .getFunctionIdentifier());
         LimitType limit = null;
         switch (ck) {
             case EQ: {
@@ -380,7 +389,7 @@
         }
         return limit;
     }
-    
+
     // Returns true if there is a constant value on the left-hand side  if the given optimizable function (assuming a binary function).
     public boolean constantIsOnLhs(IOptimizableFuncExpr optFuncExpr) {
         return optFuncExpr.getFuncExpr().getArguments().get(0) == optFuncExpr.getConstantVal(0);
@@ -395,7 +404,7 @@
     }
 
     @Override
-    public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr) {
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
         // No additional analysis required for BTrees.
         return true;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
index 5dea91f..c377a34 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/BTreeJobGenParams.java
@@ -6,7 +6,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
@@ -19,43 +19,43 @@
 
     protected List<LogicalVariable> lowKeyVarList;
     protected List<LogicalVariable> highKeyVarList;
-    
+
     protected boolean lowKeyInclusive;
     protected boolean highKeyInclusive;
-    
+
     public BTreeJobGenParams() {
         super();
     }
-    
-    public BTreeJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput,
+
+    public BTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
             boolean requiresBroadcast) {
-        super(indexName, indexKind, datasetName, retainInput, requiresBroadcast);
+        super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
     }
 
     public void setLowKeyVarList(List<LogicalVariable> keyVarList, int startIndex, int numKeys) {
         lowKeyVarList = new ArrayList<LogicalVariable>(numKeys);
         setKeyVarList(keyVarList, lowKeyVarList, startIndex, numKeys);
     }
-    
+
     public void setHighKeyVarList(List<LogicalVariable> keyVarList, int startIndex, int numKeys) {
         highKeyVarList = new ArrayList<LogicalVariable>(numKeys);
         setKeyVarList(keyVarList, highKeyVarList, startIndex, numKeys);
     }
-    
+
     private void setKeyVarList(List<LogicalVariable> src, List<LogicalVariable> dest, int startIndex, int numKeys) {
         for (int i = 0; i < numKeys; i++) {
             dest.add(src.get(startIndex + i));
         }
     }
-    
+
     public void setLowKeyInclusive(boolean lowKeyInclusive) {
         this.lowKeyInclusive = lowKeyInclusive;
     }
-    
+
     public void setHighKeyInclusive(boolean highKeyInclusive) {
         this.highKeyInclusive = highKeyInclusive;
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.writeToFuncArgs(funcArgs);
         writeVarList(lowKeyVarList, funcArgs);
@@ -63,7 +63,7 @@
         writeKeyInclusive(lowKeyInclusive, funcArgs);
         writeKeyInclusive(highKeyInclusive, funcArgs);
     }
-    
+
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.readFromFuncArgs(funcArgs);
         int index = super.getNumParams();
@@ -71,19 +71,19 @@
         highKeyVarList = new ArrayList<LogicalVariable>();
         int nextIndex = readVarList(funcArgs, index, lowKeyVarList);
         nextIndex = readVarList(funcArgs, nextIndex, highKeyVarList);
-        readKeyInclusives(funcArgs, nextIndex);        
+        readKeyInclusives(funcArgs, nextIndex);
     }
-    
+
     private void readKeyInclusives(List<Mutable<ILogicalExpression>> funcArgs, int index) {
         lowKeyInclusive = ((ConstantExpression) funcArgs.get(index).getValue()).getValue().isTrue();
         highKeyInclusive = ((ConstantExpression) funcArgs.get(index + 1).getValue()).getValue().isTrue();
     }
-    
+
     private void writeKeyInclusive(boolean keyInclusive, List<Mutable<ILogicalExpression>> funcArgs) {
         ILogicalExpression keyExpr = keyInclusive ? ConstantExpression.TRUE : ConstantExpression.FALSE;
         funcArgs.add(new MutableObject<ILogicalExpression>(keyExpr));
     }
-    
+
     public List<LogicalVariable> getLowKeyVarList() {
         return lowKeyVarList;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
index 8dc8b8e..eadbc69 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IAccessMethod.java
@@ -4,7 +4,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
@@ -19,13 +19,13 @@
  * with a given index.
  */
 public interface IAccessMethod {
-    
+
     /**
      * @return A list of function identifiers that are optimizable by this
      *         access method.
      */
     public List<FunctionIdentifier> getOptimizableFunctions();
-    
+
     /**
      * Analyzes the arguments of a given optimizable funcExpr to see if this
      * access method is applicable (e.g., one arg is a constant and one is a
@@ -37,8 +37,9 @@
      * @return true if funcExpr is optimizable by this access method, false
      *         otherwise
      */
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx);
-    
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx);
+
     /**
      * Indicates whether all index expressions must be matched in order for this
      * index to be applicable.
@@ -54,23 +55,23 @@
      * @return boolean
      */
     public boolean matchPrefixIndexExprs();
-    
+
     /**
      * Applies the plan transformation to use chosenIndex to optimize a selection query.
      */
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,            
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException;
-    
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException;
+
     /**
      * Applies the plan transformation to use chosenIndex to optimize a join query.
      */
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException;
-    
+    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException;
+
     /**
      * Analyzes expr to see whether it is optimizable by the given concrete index.
      */
-    public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr);
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr);
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
index 72acc1d..e4555aa 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceJoinAccessMethodRule.java
@@ -6,8 +6,8 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Index;
 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.ILogicalExpression;
@@ -47,26 +47,28 @@
  * to accept any subtree on one side, as long as the other side has a datasource scan.
  */
 public class IntroduceJoinAccessMethodRule extends AbstractIntroduceAccessMethodRule {
-    
+
     protected Mutable<ILogicalOperator> joinRef = null;
     protected InnerJoinOperator join = null;
-	protected AbstractFunctionCallExpression joinCond = null;
-	protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();	
-	protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
-	
-	// Register access methods.
-	protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
-	static {
-	    registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
-	}
-	
+    protected AbstractFunctionCallExpression joinCond = null;
+    protected final OptimizableOperatorSubTree leftSubTree = new OptimizableOperatorSubTree();
+    protected final OptimizableOperatorSubTree rightSubTree = new OptimizableOperatorSubTree();
+
+    // Register access methods.
+    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+    static {
+        registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+    }
+
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-    	// Match operator pattern and initialize optimizable sub trees.
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        setMetadataDeclarations(context);
+
+        // Match operator pattern and initialize optimizable sub trees.
         if (!matchesOperatorPattern(opRef, context)) {
             return false;
         }
-        
         // Analyze condition on those optimizable subtrees that have a datasource scan.
         Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new HashMap<IAccessMethod, AccessMethodAnalysisContext>();
         boolean matchInLeftSubTree = false;
@@ -101,14 +103,14 @@
             fillSubTreeIndexExprs(rightSubTree, analyzedAMs);
         }
         pruneIndexCandidates(analyzedAMs);
-        
+
         // Choose index to be applied.
-        Pair<IAccessMethod, AqlCompiledIndexDecl> chosenIndex = chooseIndex(analyzedAMs);
+        Pair<IAccessMethod, Index> chosenIndex = chooseIndex(analyzedAMs);
         if (chosenIndex == null) {
             context.addToDontApplySet(this, join);
             return false;
         }
-        
+
         // Apply plan transformation using chosen index.
         AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
         boolean res = chosenIndex.first.applyJoinPlanTransformation(joinRef, leftSubTree, rightSubTree,
@@ -119,7 +121,7 @@
         context.addToDontApplySet(this, join);
         return res;
     }
-    
+
     protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         // First check that the operator is a join and its condition is a function call.
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
index 1d2cc0e..59b11fc 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IntroduceSelectAccessMethodRule.java
@@ -6,8 +6,8 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Index;
 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.ILogicalExpression;
@@ -51,29 +51,32 @@
  * 
  */
 public class IntroduceSelectAccessMethodRule extends AbstractIntroduceAccessMethodRule {
-    
-	// Operators representing the patterns to be matched:
+
+    // Operators representing the patterns to be matched:
     // These ops are set in matchesPattern()
     protected Mutable<ILogicalOperator> selectRef = null;
     protected SelectOperator select = null;
-	protected AbstractFunctionCallExpression selectCond = null;
-	protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
+    protected AbstractFunctionCallExpression selectCond = null;
+    protected final OptimizableOperatorSubTree subTree = new OptimizableOperatorSubTree();
 
-	// Register access methods.
-	protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
-	static {
-	    registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
-	    registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
-	    registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
-	}
-	
+    // Register access methods.
+    protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
+    static {
+        registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
+        registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
+        registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+    }
+
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-    	// Match operator pattern and initialize operator members.
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        setMetadataDeclarations(context);
+
+        // Match operator pattern and initialize operator members.
         if (!matchesOperatorPattern(opRef, context)) {
             return false;
         }
-        
+
         // Analyze select condition.
         Map<IAccessMethod, AccessMethodAnalysisContext> analyzedAMs = new HashMap<IAccessMethod, AccessMethodAnalysisContext>();
         if (!analyzeCondition(selectCond, subTree.assigns, analyzedAMs)) {
@@ -84,28 +87,28 @@
         if (!subTree.setDatasetAndTypeMetadata((AqlMetadataProvider) context.getMetadataProvider())) {
             return false;
         }
-        
+
         fillSubTreeIndexExprs(subTree, analyzedAMs);
         pruneIndexCandidates(analyzedAMs);
-        
+
         // Choose index to be applied.
-        Pair<IAccessMethod, AqlCompiledIndexDecl> chosenIndex = chooseIndex(analyzedAMs);
+        Pair<IAccessMethod, Index> chosenIndex = chooseIndex(analyzedAMs);
         if (chosenIndex == null) {
             context.addToDontApplySet(this, select);
             return false;
         }
-        
+
         // Apply plan transformation using chosen index.
         AccessMethodAnalysisContext analysisCtx = analyzedAMs.get(chosenIndex.first);
-        boolean res = chosenIndex.first.applySelectPlanTransformation(selectRef, subTree, 
-                chosenIndex.second, analysisCtx, context);
+        boolean res = chosenIndex.first.applySelectPlanTransformation(selectRef, subTree, chosenIndex.second,
+                analysisCtx, context);
         if (res) {
             OperatorPropertiesUtil.typeOpRec(opRef, context);
         }
         context.addToDontApplySet(this, select);
         return res;
     }
-    
+
     protected boolean matchesOperatorPattern(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         // First check that the operator is a select and its condition is a function call.
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
index fe3a539..dbd92c0 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexAccessMethod.java
@@ -14,9 +14,8 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryTokenizerFactoryProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.base.AFloat;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.ANull;
@@ -66,7 +65,7 @@
 import edu.uci.ics.hyracks.storage.am.invertedindex.tokenizers.IBinaryTokenizerFactory;
 
 /**
- * Class for helping rewrite rules to choose and apply inverted indexes.  
+ * Class for helping rewrite rules to choose and apply inverted indexes.
  */
 public class InvertedIndexAccessMethod implements IAccessMethod {
 
@@ -77,15 +76,15 @@
         EDIT_DISTANCE,
         INVALID
     }
-    
+
     private static List<FunctionIdentifier> funcIdents = new ArrayList<FunctionIdentifier>();
     static {
-        funcIdents.add(AsterixBuiltinFunctions.CONTAINS);        
+        funcIdents.add(AsterixBuiltinFunctions.CONTAINS);
         // For matching similarity-check functions. For example, similarity-jaccard-check returns a list of two items,
         // and the select condition will get the first list-item and check whether it evaluates to true. 
         funcIdents.add(AsterixBuiltinFunctions.GET_ITEM);
     }
-    
+
     // These function identifiers are matched in this AM's analyzeFuncExprArgs(), 
     // and are not visible to the outside driver.
     private static HashSet<FunctionIdentifier> secondLevelFuncIdents = new HashSet<FunctionIdentifier>();
@@ -93,23 +92,25 @@
         secondLevelFuncIdents.add(AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK);
         secondLevelFuncIdents.add(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK);
     }
-    
+
     public static InvertedIndexAccessMethod INSTANCE = new InvertedIndexAccessMethod();
-    
+
     @Override
     public List<FunctionIdentifier> getOptimizableFunctions() {
-       return funcIdents;
+        return funcIdents;
     }
 
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS) {
             return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
         }
         return analyzeGetItemFuncExpr(funcExpr, assigns, analysisCtx);
     }
-    
-    public boolean analyzeGetItemFuncExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+
+    public boolean analyzeGetItemFuncExpr(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         if (funcExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GET_ITEM) {
             return false;
         }
@@ -121,8 +122,8 @@
         }
         // The first arg must be a variable or a function expr.
         // If it is a variable we must track its origin in the assigns to get the original function expr.
-        if (arg1.getExpressionTag() != LogicalExpressionTag.VARIABLE &&
-                arg1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+        if (arg1.getExpressionTag() != LogicalExpressionTag.VARIABLE
+                && arg1.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
         }
         AbstractFunctionCallExpression matchedFuncExpr = null;
@@ -132,7 +133,7 @@
         }
         // The get-item arg is a variable. Search the assigns for its origination function.
         int matchedAssignIndex = -1;
-        if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {             
+        if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
             VariableReferenceExpression varRefExpr = (VariableReferenceExpression) arg1;
             // Try to find variable ref expr in all assigns.
             for (int i = 0; i < assigns.size(); i++) {
@@ -163,8 +164,10 @@
         if (!secondLevelFuncIdents.contains(matchedFuncExpr.getFunctionIdentifier())) {
             return false;
         }
-        boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex, analysisCtx);
-        boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex, analysisCtx);
+        boolean selectMatchFound = analyzeSelectSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns,
+                matchedAssignIndex, analysisCtx);
+        boolean joinMatchFound = analyzeJoinSimilarityCheckFuncExprArgs(matchedFuncExpr, assigns, matchedAssignIndex,
+                analysisCtx);
         if (selectMatchFound || joinMatchFound) {
             return true;
         }
@@ -184,7 +187,7 @@
         ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
         ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
         // We expect arg1 and arg2 to be non-constants for a join.
-        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT 
+        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
                 || arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
             return false;
         }
@@ -196,10 +199,11 @@
         if (fieldVar2 == null) {
             return false;
         }
-        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar1, fieldVar2 }, new IAlgebricksConstantValue[] { constThreshVal }));
+        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
+                new LogicalVariable[] { fieldVar1, fieldVar2 }, new IAlgebricksConstantValue[] { constThreshVal }));
         return true;
     }
-    
+
     private boolean analyzeSelectSimilarityCheckFuncExprArgs(AbstractFunctionCallExpression funcExpr,
             List<AssignOperator> assigns, int matchedAssignIndex, AccessMethodAnalysisContext analysisCtx) {
         // There should be exactly three arguments.
@@ -215,11 +219,11 @@
         // Determine whether one arg is constant, and the other is non-constant.
         ILogicalExpression constArg = null;
         ILogicalExpression nonConstArg = null;
-        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT 
-                && arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) { 
+        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT
+                && arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
             constArg = arg1;
             nonConstArg = arg2;
-        } else if(arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
+        } else if (arg2.getExpressionTag() == LogicalExpressionTag.CONSTANT
                 && arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
             constArg = arg2;
             nonConstArg = arg1;
@@ -232,21 +236,22 @@
         if (fieldVar == null) {
             return false;
         }
-        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar }, new IAlgebricksConstantValue[] { constFilterVal, constThreshVal }));
+        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, new LogicalVariable[] { fieldVar },
+                new IAlgebricksConstantValue[] { constFilterVal, constThreshVal }));
         return true;
     }
-    
-    private LogicalVariable getNonConstArgFieldVar(ILogicalExpression nonConstArg, AbstractFunctionCallExpression funcExpr,
-            List<AssignOperator> assigns, int matchedAssignIndex) {
+
+    private LogicalVariable getNonConstArgFieldVar(ILogicalExpression nonConstArg,
+            AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, int matchedAssignIndex) {
         LogicalVariable fieldVar = null;
         // Analyze nonConstArg depending on similarity function.
-        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {            
+        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK) {
             AbstractFunctionCallExpression nonConstFuncExpr = funcExpr;
             if (nonConstArg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 nonConstFuncExpr = (AbstractFunctionCallExpression) nonConstArg;
                 // TODO: Currently, we're only looking for word and gram tokens (non hashed).
-                if (nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS &&
-                        nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
+                if (nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS
+                        && nonConstFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
                     return null;
                 }
                 // Find the variable that is being tokenized.
@@ -269,8 +274,8 @@
                         }
                         AbstractFunctionCallExpression childFuncExpr = (AbstractFunctionCallExpression) childExpr;
                         // If fieldVar references the result of a tokenization, then we should remember the variable being tokenized.
-                        if (childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS &&
-                                childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
+                        if (childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.WORD_TOKENS
+                                && childFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.GRAM_TOKENS) {
                             break;
                         }
                         // We expect the tokenizer's argument to be a variable, otherwise we cannot apply an index.
@@ -292,12 +297,12 @@
         }
         if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
             if (nonConstArg.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                fieldVar = ((VariableReferenceExpression) nonConstArg).getVariableReference();                
+                fieldVar = ((VariableReferenceExpression) nonConstArg).getVariableReference();
             }
         }
         return fieldVar;
     }
-    
+
     @Override
     public boolean matchAllIndexExprs() {
         return true;
@@ -308,18 +313,20 @@
         return false;
     }
 
-    private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, 
-            AqlCompiledIndexDecl chosenIndex, IOptimizableFuncExpr optFuncExpr, boolean retainInput, boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
-        AqlCompiledDatasetDecl datasetDecl = indexSubTree.datasetDecl;
+    private ILogicalOperator createSecondaryToPrimaryPlan(OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, IOptimizableFuncExpr optFuncExpr,
+            boolean retainInput, boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
+        Dataset dataset = indexSubTree.dataset;
         ARecordType recordType = indexSubTree.recordType;
         DataSourceScanOperator dataSourceScan = indexSubTree.dataSourceScan;
-        
-        InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams(chosenIndex.getIndexName(), chosenIndex.getKind(), datasetDecl.getName(), retainInput, requiresBroadcast);
+
+        InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams(chosenIndex.getIndexName(),
+                chosenIndex.getIndexType(), dataset.getDatasetName(), retainInput, requiresBroadcast);
         // Add function-specific args such as search modifier, and possibly a similarity threshold.
         addFunctionSpecificArgs(optFuncExpr, jobGenParams);
         // Add the type of search key from the optFuncExpr.
         addSearchKeyType(optFuncExpr, indexSubTree, context, jobGenParams);
-        
+
         // Operator that feeds the secondary-index search.
         AbstractLogicalOperator inputOp = null;
         // Here we generate vars and funcs for assigning the secondary-index keys to be fed into the secondary-index search.
@@ -332,7 +339,7 @@
             // Add key vars and exprs to argument list.
             addKeyVarsAndExprs(optFuncExpr, keyVarList, keyExprList, context);
             // Assign operator that sets the secondary-index search-key fields.
-            inputOp = new AssignOperator(keyVarList, keyExprList);            
+            inputOp = new AssignOperator(keyVarList, keyExprList);
             // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
             inputOp.getInputs().add(dataSourceScan.getInputs().get(0));
             inputOp.setExecutionMode(dataSourceScan.getExecutionMode());
@@ -343,14 +350,14 @@
             inputOp = (AbstractLogicalOperator) probeSubTree.root;
         }
         jobGenParams.setKeyVarList(keyVarList);
-        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(datasetDecl,
-                recordType, chosenIndex, inputOp, jobGenParams, context, true, retainInput);
+        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+                chosenIndex, inputOp, jobGenParams, context, true, retainInput);
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
-        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan,
-                datasetDecl, recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
+        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+                recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
         return primaryIndexUnnestOp;
     }
-    
+
     /**
      * Returns the variable which acts as the input search key to a secondary
      * index that optimizes optFuncExpr by replacing rewriting indexSubTree
@@ -365,96 +372,99 @@
             return optFuncExpr.getLogicalVar(0);
         }
     }
-    
+
     @Override
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException {
         IOptimizableFuncExpr optFuncExpr = chooseOptFuncExpr(chosenIndex, analysisCtx);
-        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr, false, false, context);
+        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr, false,
+                false, context);
         // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
         subTree.dataSourceScanRef.setValue(indexPlanRootOp);
         return true;
     }
-    
+
     @Override
     public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
         // Figure out if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
-        AqlCompiledDatasetDecl dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
+        Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
         // Determine probe and index subtrees based on chosen index.
         OptimizableOperatorSubTree indexSubTree = null;
         OptimizableOperatorSubTree probeSubTree = null;
-        if (dataset.getName().equals(leftSubTree.datasetDecl.getName())) {
+        if (dataset.getDatasetName().equals(leftSubTree.dataset.getDatasetName())) {
             indexSubTree = leftSubTree;
             probeSubTree = rightSubTree;
-        } else if (dataset.getName().equals(rightSubTree.datasetDecl.getName())) {
+        } else if (dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
             indexSubTree = rightSubTree;
             probeSubTree = leftSubTree;
         }
         IOptimizableFuncExpr optFuncExpr = chooseOptFuncExpr(chosenIndex, analysisCtx);
-        
+
         // Clone the original join condition because we may have to modify it (and we also need the original).
         InnerJoinOperator join = (InnerJoinOperator) joinRef.getValue();
         ILogicalExpression joinCond = join.getCondition().getValue().cloneExpression();
-        
+
         // Remember original live variables to make sure our new index-based plan returns exactly those vars as well.
         List<LogicalVariable> originalLiveVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getLiveVariables(join, originalLiveVars);
-        
+
         // Create "panic" (non indexed) nested-loop join path if necessary.
         Mutable<ILogicalOperator> panicJoinRef = null;
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
             panicJoinRef = new MutableObject<ILogicalOperator>(joinRef.getValue());
-            Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree, probeSubTree, optFuncExpr, chosenIndex, context);
+            Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree,
+                    probeSubTree, optFuncExpr, chosenIndex, context);
             probeSubTree.rootRef.setValue(newProbeRootRef.getValue());
             probeSubTree.root = newProbeRootRef.getValue();
         }
         // Create regular indexed-nested loop join path.
-        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex, optFuncExpr, true, true, context);        
+        ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex,
+                optFuncExpr, true, true, context);
         indexSubTree.dataSourceScanRef.setValue(indexPlanRootOp);
-        
+
         // Change join into a select with the same condition.
         SelectOperator topSelect = new SelectOperator(new MutableObject<ILogicalExpression>(joinCond));
         topSelect.getInputs().add(indexSubTree.rootRef);
         topSelect.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(topSelect);
-        
+
         // Add a project operator on top to guarantee that our new index-based plan returns exactly the same variables as the original plan.
         ProjectOperator projectOp = new ProjectOperator(originalLiveVars);
         projectOp.getInputs().add(new MutableObject<ILogicalOperator>(topSelect));
         projectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(projectOp);
         joinRef.setValue(projectOp);
-        
+
         // Hook up the indexed-nested loop join path with the "panic" (non indexed) nested-loop join path by putting a union all on top.
         if (panicJoinRef != null) {
             // Gather live variables from the index plan and the panic plan.
             List<LogicalVariable> indexPlanLiveVars = new ArrayList<LogicalVariable>();
-        	VariableUtilities.getLiveVariables(joinRef.getValue(), indexPlanLiveVars);
-        	List<LogicalVariable> panicPlanLiveVars = new ArrayList<LogicalVariable>();
-        	VariableUtilities.getLiveVariables(panicJoinRef.getValue(), panicPlanLiveVars);
-        	if (indexPlanLiveVars.size() != panicPlanLiveVars.size()) {
-        	    throw new AlgebricksException("Unequal number of variables returned from index plan and panic plan.");
-        	}
-        	// Create variable mapping for union all operator.
-        	List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
-        	for (int i = 0; i < indexPlanLiveVars.size(); i++) {
-        	    varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(indexPlanLiveVars.get(i), panicPlanLiveVars.get(i), indexPlanLiveVars.get(i)));
-        	}
-        	UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
-        	unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(joinRef.getValue()));
-        	unionAllOp.getInputs().add(panicJoinRef);
-        	unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
-        	context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
-        	joinRef.setValue(unionAllOp);
+            VariableUtilities.getLiveVariables(joinRef.getValue(), indexPlanLiveVars);
+            List<LogicalVariable> panicPlanLiveVars = new ArrayList<LogicalVariable>();
+            VariableUtilities.getLiveVariables(panicJoinRef.getValue(), panicPlanLiveVars);
+            if (indexPlanLiveVars.size() != panicPlanLiveVars.size()) {
+                throw new AlgebricksException("Unequal number of variables returned from index plan and panic plan.");
+            }
+            // Create variable mapping for union all operator.
+            List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>();
+            for (int i = 0; i < indexPlanLiveVars.size(); i++) {
+                varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(indexPlanLiveVars.get(i),
+                        panicPlanLiveVars.get(i), indexPlanLiveVars.get(i)));
+            }
+            UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
+            unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(joinRef.getValue()));
+            unionAllOp.getInputs().add(panicJoinRef);
+            unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
+            context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
+            joinRef.setValue(unionAllOp);
         }
         return true;
     }
-    
-    private IOptimizableFuncExpr chooseOptFuncExpr(AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx) {
+
+    private IOptimizableFuncExpr chooseOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
         // TODO: We can probably do something smarter here.
         // Pick the first expr optimizable by this index.
         List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
@@ -462,15 +472,18 @@
         return analysisCtx.matchedFuncExprs.get(firstExprIndex);
     }
 
-    private Mutable<ILogicalOperator> createPanicNestedLoopJoinPlan(Mutable<ILogicalOperator> joinRef, OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree, IOptimizableFuncExpr optFuncExpr, AqlCompiledIndexDecl chosenIndex, IOptimizationContext context) throws AlgebricksException {
+    private Mutable<ILogicalOperator> createPanicNestedLoopJoinPlan(Mutable<ILogicalOperator> joinRef,
+            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
+            IOptimizableFuncExpr optFuncExpr, Index chosenIndex, IOptimizationContext context)
+            throws AlgebricksException {
         LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
-        
+
         // We split the plan into two "branches", and add selections on each side.
         AbstractLogicalOperator replicateOp = new ReplicateOperator(2);
         replicateOp.getInputs().add(new MutableObject<ILogicalOperator>(probeSubTree.root));
         replicateOp.setExecutionMode(ExecutionMode.PARTITIONED);
-        context.computeAndSetTypeEnvironmentForOperator(replicateOp);        
-        
+        context.computeAndSetTypeEnvironmentForOperator(replicateOp);
+
         // Create select ops for removing tuples that are filterable and not filterable, respectively.
         IVariableTypeEnvironment topTypeEnv = context.getOutputTypeEnvironment(joinRef.getValue());
         IAType inputSearchVarType = (IAType) topTypeEnv.getVarType(inputSearchVar);
@@ -478,23 +491,22 @@
         Mutable<ILogicalOperator> isNotFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         createIsFilterableSelectOps(replicateOp, inputSearchVar, inputSearchVarType, optFuncExpr, chosenIndex, context,
                 isFilterableSelectOpRef, isNotFilterableSelectOpRef);
-        
+
         List<LogicalVariable> originalLiveVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getLiveVariables(indexSubTree.root, originalLiveVars);
-        
+
         // Copy the scan subtree in indexSubTree.
         Counter counter = new Counter(context.getVarCounter());
         LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
-        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);        
+        ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);
         context.setVarCounter(counter.get());
-        //context.computeAndSetTypeEnvironmentForOperator(scanSubTree);
-        
+
         List<LogicalVariable> copyLiveVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getLiveVariables(scanSubTree, copyLiveVars);
-        
-		// Replace the inputs of the given join op, and replace variables in its
-		// condition since we deep-copied one of the scanner subtrees which
-		// changed variables. 
+
+        // Replace the inputs of the given join op, and replace variables in its
+        // condition since we deep-copied one of the scanner subtrees which
+        // changed variables. 
         InnerJoinOperator joinOp = (InnerJoinOperator) joinRef.getValue();
         // Substitute vars in the join condition due to copying of the scanSubTree.
         List<LogicalVariable> joinCondUsedVars = new ArrayList<LogicalVariable>();
@@ -503,80 +515,99 @@
             int ix = originalLiveVars.indexOf(joinCondUsedVars.get(i));
             if (ix >= 0) {
                 joinOp.getCondition().getValue().substituteVar(originalLiveVars.get(ix), copyLiveVars.get(ix));
-            }        
-        }    
+            }
+        }
         joinOp.getInputs().clear();
         joinOp.getInputs().add(new MutableObject<ILogicalOperator>(scanSubTree));
         // Make sure that the build input (which may be materialized causing blocking) comes from 
         // the split+select, otherwise the plan will have a deadlock.
         joinOp.getInputs().add(isNotFilterableSelectOpRef);
         context.computeAndSetTypeEnvironmentForOperator(joinOp);
-        
+
         // Return the new root of the probeSubTree.
         return isFilterableSelectOpRef;
     }
-    
-    private void createIsFilterableSelectOps(ILogicalOperator inputOp, LogicalVariable inputSearchVar, IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, AqlCompiledIndexDecl chosenIndex, IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef, Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {        
+
+    private void createIsFilterableSelectOps(ILogicalOperator inputOp, LogicalVariable inputSearchVar,
+            IAType inputSearchVarType, IOptimizableFuncExpr optFuncExpr, Index chosenIndex,
+            IOptimizationContext context, Mutable<ILogicalOperator> isFilterableSelectOpRef,
+            Mutable<ILogicalOperator> isNotFilterableSelectOpRef) throws AlgebricksException {
         // Create select operator for removing tuples that are not filterable.
         // First determine the proper filter function and args based on the type of the input search var.
         ILogicalExpression isFilterableExpr = null;
         switch (inputSearchVarType.getTypeTag()) {
             case STRING: {
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(4);
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                        inputSearchVar)));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(chosenIndex.getGramLength())));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr
+                        .getConstantVal(0))));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils
+                        .createInt32Constant(chosenIndex.getGramLength())));
                 // TODO: Currently usePrePost is hardcoded to be true.
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(true)));
-                isFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE), isFilterableArgs);        
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils
+                        .createBooleanConstant(true)));
+                isFilterableExpr = new ScalarFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_STRING_IS_FILTERABLE),
+                        isFilterableArgs);
                 break;
             }
             case UNORDEREDLIST:
             case ORDEREDLIST: {
                 List<Mutable<ILogicalExpression>> isFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>(2);
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputSearchVar)));
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                        inputSearchVar)));
                 // Since we are optimizing a join, the similarity threshold should be the only constant in the optimizable function expression.
-                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
-                isFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE), isFilterableArgs);        
+                isFilterableArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr
+                        .getConstantVal(0))));
+                isFilterableExpr = new ScalarFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_LIST_IS_FILTERABLE),
+                        isFilterableArgs);
                 break;
             }
             default: {
             }
         }
-        SelectOperator isFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(isFilterableExpr));
+        SelectOperator isFilterableSelectOp = new SelectOperator(
+                new MutableObject<ILogicalExpression>(isFilterableExpr));
         isFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isFilterableSelectOp);
-        
+
         // Select operator for removing tuples that are filterable.
         List<Mutable<ILogicalExpression>> isNotFilterableArgs = new ArrayList<Mutable<ILogicalExpression>>();
         isNotFilterableArgs.add(new MutableObject<ILogicalExpression>(isFilterableExpr));
-        ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NOT), isNotFilterableArgs);
-        SelectOperator isNotFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(isNotFilterableExpr));
+        ILogicalExpression isNotFilterableExpr = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NOT), isNotFilterableArgs);
+        SelectOperator isNotFilterableSelectOp = new SelectOperator(new MutableObject<ILogicalExpression>(
+                isNotFilterableExpr));
         isNotFilterableSelectOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         isNotFilterableSelectOp.setExecutionMode(ExecutionMode.LOCAL);
         context.computeAndSetTypeEnvironmentForOperator(isNotFilterableSelectOp);
-        
+
         isFilterableSelectOpRef.setValue(isFilterableSelectOp);
         isNotFilterableSelectOpRef.setValue(isNotFilterableSelectOp);
     }
-    
-    private void addSearchKeyType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree indexSubTree, IOptimizationContext context, InvertedIndexJobGenParams jobGenParams) throws AlgebricksException {
+
+    private void addSearchKeyType(IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree indexSubTree,
+            IOptimizationContext context, InvertedIndexJobGenParams jobGenParams) throws AlgebricksException {
         // If we have two variables in the optFunxExpr, then we are optimizing a join.
         IAType type = null;
         ATypeTag typeTag = null;
-        if (optFuncExpr.getNumLogicalVars() == 2) {            
+        if (optFuncExpr.getNumLogicalVars() == 2) {
             // Find the type of the variable that is going to feed into the index search.
             if (optFuncExpr.getOperatorSubTree(0) == indexSubTree) {
                 // If the index is on a dataset in subtree 0, then subtree 1 will feed.
-                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(1).root).getVarType(optFuncExpr.getLogicalVar(1));
+                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(1).root).getVarType(
+                        optFuncExpr.getLogicalVar(1));
             } else {
                 // If the index is on a dataset in subtree 1, then subtree 0 will feed.
-                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(0).root).getVarType(optFuncExpr.getLogicalVar(0));
+                type = (IAType) context.getOutputTypeEnvironment(optFuncExpr.getOperatorSubTree(0).root).getVarType(
+                        optFuncExpr.getLogicalVar(0));
             }
             typeTag = type.getTypeTag();
-        } else { 
+        } else {
             // We are optimizing a selection query. Add the type of the search key constant.
             AsterixConstantValue constVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
             IAObject obj = constVal.getObject();
@@ -588,7 +619,7 @@
         }
         jobGenParams.setSearchKeyType(typeTag);
     }
-    
+
     private void addFunctionSpecificArgs(IOptimizableFuncExpr optFuncExpr, InvertedIndexJobGenParams jobGenParams) {
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS) {
             jobGenParams.setSearchModifierType(SearchModifierType.CONJUNCTIVE);
@@ -606,17 +637,19 @@
         }
     }
 
-    private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList, ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context) throws AlgebricksException {
+    private void addKeyVarsAndExprs(IOptimizableFuncExpr optFuncExpr, ArrayList<LogicalVariable> keyVarList,
+            ArrayList<Mutable<ILogicalExpression>> keyExprList, IOptimizationContext context)
+            throws AlgebricksException {
         // For now we are assuming a single secondary index key.
         // Add a variable and its expr to the lists which will be passed into an assign op.
         LogicalVariable keyVar = context.newVar();
         keyVarList.add(keyVar);
-        keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));        
+        keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
         return;
     }
 
     @Override
-    public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr) {
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
             // Must be for a join query.
             if (optFuncExpr.getNumConstantVals() == 1) {
@@ -631,16 +664,16 @@
             AInt32 edThresh = (AInt32) intObj;
             int mergeThreshold = 0;
             // We can only optimize edit distance on strings using an ngram index.
-            if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING && index.getKind() == IndexKind.NGRAM_INVIX) {
-                AString astr = (AString) listOrStrObj;                    
+            if (listOrStrObj.getType().getTypeTag() == ATypeTag.STRING && index.getIndexType() == IndexType.NGRAM_INVIX) {
+                AString astr = (AString) listOrStrObj;
                 // Compute merge threshold.
                 mergeThreshold = (astr.getStringValue().length() + index.getGramLength() - 1)
                         - edThresh.getIntegerValue() * index.getGramLength();
             }
             // We can only optimize edit distance on lists using a word index.
             if ((listOrStrObj.getType().getTypeTag() == ATypeTag.ORDEREDLIST || listOrStrObj.getType().getTypeTag() == ATypeTag.UNORDEREDLIST)
-                    && index.getKind() == IndexKind.WORD_INVIX) {
-                IACollection alist = (IACollection) listOrStrObj;        
+                    && index.getIndexType() == IndexType.WORD_INVIX) {
+                IACollection alist = (IACollection) listOrStrObj;
                 // Compute merge threshold.
                 mergeThreshold = alist.size() - edThresh.getIntegerValue();
             }
@@ -664,12 +697,12 @@
             if (nonConstArg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                 AbstractFunctionCallExpression nonConstfuncExpr = (AbstractFunctionCallExpression) nonConstArg;
                 // We can use this index if the tokenization function matches the index type.
-                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS &&
-                        index.getKind() == IndexKind.WORD_INVIX) {
+                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.WORD_TOKENS
+                        && index.getIndexType() == IndexType.WORD_INVIX) {
                     return true;
                 }
-                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS &&
-                        index.getKind() == IndexKind.NGRAM_INVIX) {
+                if (nonConstfuncExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.GRAM_TOKENS
+                        && index.getIndexType() == IndexType.NGRAM_INVIX) {
                     return true;
                 }
             }
@@ -682,12 +715,12 @@
         }
         // We can only optimize contains with ngram indexes.
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.CONTAINS
-                && index.getKind() == IndexKind.NGRAM_INVIX) {
+                && index.getIndexType() == IndexType.NGRAM_INVIX) {
             // Check that the constant search string has at least gramLength characters.
             AsterixConstantValue strConstVal = (AsterixConstantValue) optFuncExpr.getConstantVal(0);
             IAObject strObj = strConstVal.getObject();
             if (strObj.getType().getTypeTag() == ATypeTag.STRING) {
-                AString astr = (AString) strObj;                    
+                AString astr = (AString) strObj;
                 if (astr.getStringValue().length() >= index.getGramLength()) {
                     return true;
                 }
@@ -695,10 +728,10 @@
         }
         return false;
     }
-    
+
     public static IBinaryComparatorFactory getTokenBinaryComparatorFactory(IAType keyType) throws AlgebricksException {
         IAType type = keyType;
-        ATypeTag typeTag = keyType.getTypeTag();        
+        ATypeTag typeTag = keyType.getTypeTag();
         // Extract item type from list.
         if (typeTag == ATypeTag.UNORDEREDLIST || typeTag == ATypeTag.ORDEREDLIST) {
             AbstractCollectionType listType = (AbstractCollectionType) keyType;
@@ -708,10 +741,9 @@
             type = listType.getItemType();
         }
         // Ignore case for string types.
-        return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
-                type, true, true);
+        return AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(type, true, true);
     }
-    
+
     public static ITypeTraits getTokenTypeTrait(IAType keyType) throws AlgebricksException {
         IAType type = keyType;
         ATypeTag typeTag = keyType.getTypeTag();
@@ -732,10 +764,10 @@
         }
         return AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
     }
-    
-    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType, ATypeTag searchKeyType, AqlCompiledIndexDecl index)
-            throws AlgebricksException {
-        switch (index.getKind()) {
+
+    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(SearchModifierType searchModifierType,
+            ATypeTag searchKeyType, Index index) throws AlgebricksException {
+        switch (index.getIndexType()) {
             case WORD_INVIX: {
                 return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(searchKeyType, false);
             }
@@ -746,29 +778,30 @@
                         index.getGramLength(), prePost, false);
             }
             default: {
-                throw new AlgebricksException("Tokenizer not applicable to index kind '" + index.getKind() + "'.");
+                throw new AlgebricksException("Tokenizer not applicable to index kind '" + index.getIndexType() + "'.");
             }
         }
     }
-    
-    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType, int gramLength)
-            throws AlgebricksException {
+
+    public static IBinaryTokenizerFactory getBinaryTokenizerFactory(ATypeTag keyType, IndexType indexType,
+            int gramLength) throws AlgebricksException {
         switch (indexType) {
             case WORD_INVIX: {
                 return AqlBinaryTokenizerFactoryProvider.INSTANCE.getWordTokenizerFactory(keyType, false);
             }
             case NGRAM_INVIX: {
-                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType,
-                        gramLength, true, false);
+                return AqlBinaryTokenizerFactoryProvider.INSTANCE.getNGramTokenizerFactory(keyType, gramLength, true,
+                        false);
             }
             default: {
                 throw new AlgebricksException("Tokenizer not applicable to index type '" + indexType + "'.");
             }
         }
     }
-    
-    public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType, IAObject simThresh, AqlCompiledIndexDecl index) throws AlgebricksException {
-        switch(searchModifierType) {
+
+    public static IInvertedIndexSearchModifierFactory getSearchModifierFactory(SearchModifierType searchModifierType,
+            IAObject simThresh, Index index) throws AlgebricksException {
+        switch (searchModifierType) {
             case CONJUNCTIVE: {
                 return new ConjunctiveSearchModifierFactory();
             }
@@ -778,7 +811,7 @@
             }
             case EDIT_DISTANCE: {
                 int edThresh = ((AInt32) simThresh).getIntegerValue();
-                switch (index.getKind()) {
+                switch (index.getIndexType()) {
                     case NGRAM_INVIX: {
                         // Edit distance on strings, filtered with overlapping grams.
                         return new EditDistanceSearchModifierFactory(index.getGramLength(), edThresh);
@@ -788,7 +821,8 @@
                         return new ListEditDistanceSearchModifierFactory(edThresh);
                     }
                     default: {
-                        throw new AlgebricksException("Incompatible search modifier '" + searchModifierType + "' for index type '" + index.getKind() + "'");
+                        throw new AlgebricksException("Incompatible search modifier '" + searchModifierType
+                                + "' for index type '" + index.getIndexType() + "'");
                     }
                 }
             }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
index fa5b5e1..530606e 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/InvertedIndexJobGenParams.java
@@ -6,7 +6,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod.SearchModifierType;
@@ -20,44 +20,47 @@
  * and from a list of function arguments, typically of an unnest-map.
  */
 public class InvertedIndexJobGenParams extends AccessMethodJobGenParams {
-    
+
     protected SearchModifierType searchModifierType;
     protected IAlgebricksConstantValue similarityThreshold;
     protected ATypeTag searchKeyType;
     protected List<LogicalVariable> keyVarList;
     protected List<LogicalVariable> nonKeyVarList;
-    
+
     public InvertedIndexJobGenParams() {
     }
-    
-    public InvertedIndexJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput, boolean requiresBroadcast) {
-        super(indexName, indexKind, datasetName, retainInput, requiresBroadcast);
+
+    public InvertedIndexJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+            boolean requiresBroadcast) {
+        super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
     }
-    
+
     public void setSearchModifierType(SearchModifierType searchModifierType) {
         this.searchModifierType = searchModifierType;
     }
-        
+
     public void setSimilarityThreshold(IAlgebricksConstantValue similarityThreshold) {
         this.similarityThreshold = similarityThreshold;
     }
-    
+
     public void setSearchKeyType(ATypeTag searchKeyType) {
         this.searchKeyType = searchKeyType;
     }
-    
+
     public void setKeyVarList(List<LogicalVariable> keyVarList) {
         this.keyVarList = keyVarList;
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.writeToFuncArgs(funcArgs);
         // Write search modifier type.
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchModifierType.ordinal())));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchModifierType
+                .ordinal())));
         // Write similarity threshold.
         funcArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(similarityThreshold)));
         // Write search key type.
-        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType.ordinal())));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(searchKeyType
+                .ordinal())));
         // Write key var list.
         writeVarList(keyVarList, funcArgs);
         // Write non-key var list.
@@ -65,7 +68,7 @@
             writeVarList(nonKeyVarList, funcArgs);
         }
     }
-    
+
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.readFromFuncArgs(funcArgs);
         int index = super.getNumParams();
@@ -73,7 +76,8 @@
         int searchModifierOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index));
         searchModifierType = SearchModifierType.values()[searchModifierOrdinal];
         // Read similarity threshold. Concrete type depends on search modifier.
-        similarityThreshold = ((AsterixConstantValue) ((ConstantExpression) funcArgs.get(index + 1).getValue()).getValue());
+        similarityThreshold = ((AsterixConstantValue) ((ConstantExpression) funcArgs.get(index + 1).getValue())
+                .getValue());
         // Read type of search key.
         int typeTagOrdinal = AccessMethodUtils.getInt32Constant(funcArgs.get(index + 2));
         searchKeyType = ATypeTag.values()[typeTagOrdinal];
@@ -84,23 +88,23 @@
         // We don't need to read the non-key var list.
         nonKeyVarList = null;
     }
-    
+
     public SearchModifierType getSearchModifierType() {
         return searchModifierType;
     }
-    
+
     public IAlgebricksConstantValue getSimilarityThreshold() {
         return similarityThreshold;
     }
-    
+
     public ATypeTag getSearchKeyType() {
         return searchKeyType;
     }
-    
+
     public List<LogicalVariable> getKeyVarList() {
         return keyVarList;
     }
-    
+
     public List<LogicalVariable> getNonKeyVarList() {
         return nonKeyVarList;
     }
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 dc4f8fc..80f8cc1 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
@@ -6,9 +6,9 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -22,7 +22,7 @@
 
 /**
  * Operator subtree that matches the following patterns, and provides convenient access to its nodes:
- * (select)? <-- (assign)+ <-- (datasource scan) 
+ * (select)? <-- (assign)+ <-- (datasource scan)
  * and
  * (select)? <-- (datasource scan)
  */
@@ -34,9 +34,9 @@
     public Mutable<ILogicalOperator> dataSourceScanRef = null;
     public DataSourceScanOperator dataSourceScan = null;
     // Dataset and type metadata. Set in setDatasetAndTypeMetadata().
-    public AqlCompiledDatasetDecl datasetDecl = null;
-    public ARecordType recordType = null;    
-    
+    public Dataset dataset = null;
+    public ARecordType recordType = null;
+
     public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) {
         rootRef = subTreeOpRef;
         root = subTreeOpRef.getValue();
@@ -62,7 +62,7 @@
             assignRefs.add(subTreeOpRef);
             assigns.add((AssignOperator) subTreeOp);
             subTreeOpRef = subTreeOp.getInputs().get(0);
-            subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();   
+            subTreeOp = (AbstractLogicalOperator) subTreeOpRef.getValue();
         } while (subTreeOp.getOperatorTag() == LogicalOperatorTag.ASSIGN);
         // Set to last valid assigns.
         subTreeOpRef = assignRefs.get(assignRefs.size() - 1);
@@ -77,11 +77,11 @@
         dataSourceScan = (DataSourceScanOperator) op3;
         return true;
     }
-    
+
     /**
      * Find the dataset corresponding to the datasource scan in the metadata.
      * Also sets recordType to be the type of that dataset.
-     */  
+     */
     public boolean setDatasetAndTypeMetadata(AqlMetadataProvider metadataProvider) throws AlgebricksException {
         if (dataSourceScan == null) {
             return false;
@@ -92,22 +92,22 @@
             return false;
         }
         AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
-        datasetDecl = metadata.findDataset(datasetName);
-        if (datasetDecl == null) {
+        dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("No metadata for dataset " + datasetName);
         }
-        if (datasetDecl.getDatasetType() != DatasetType.INTERNAL && datasetDecl.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
             return false;
         }
         // Get the record type for that dataset.
-        IAType itemType = metadata.findType(datasetDecl.getItemTypeName());
+        IAType itemType = metadata.findType(dataset.getItemTypeName());
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             return false;
         }
         recordType = (ARecordType) itemType;
         return true;
     }
-    
+
     public boolean hasDataSourceScan() {
         return dataSourceScan != null;
     }
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 ac22095..dfd3ff7 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
@@ -7,9 +7,9 @@
 import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+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;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -31,7 +31,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
 /**
- * Class for helping rewrite rules to choose and apply RTree indexes.  
+ * Class for helping rewrite rules to choose and apply RTree indexes.
  */
 public class RTreeAccessMethod implements IAccessMethod {
 
@@ -39,19 +39,20 @@
     static {
         funcIdents.add(AsterixBuiltinFunctions.SPATIAL_INTERSECT);
     }
-    
+
     public static RTreeAccessMethod INSTANCE = new RTreeAccessMethod();
-    
+
     @Override
     public List<FunctionIdentifier> getOptimizableFunctions() {
         return funcIdents;
     }
 
     @Override
-    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns, AccessMethodAnalysisContext analysisCtx) {
+    public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
+            AccessMethodAnalysisContext analysisCtx) {
         return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
     }
-    
+
     @Override
     public boolean matchAllIndexExprs() {
         return true;
@@ -63,27 +64,28 @@
     }
 
     @Override
-    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef, OptimizableOperatorSubTree subTree,
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
-        AqlCompiledDatasetDecl datasetDecl = subTree.datasetDecl;
+    public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
+            OptimizableOperatorSubTree subTree, Index index, AccessMethodAnalysisContext analysisCtx,
+            IOptimizationContext context) throws AlgebricksException {
+        Dataset dataset = subTree.dataset;
         ARecordType recordType = subTree.recordType;
         // TODO: We can probably do something smarter here based on selectivity or MBR area.
         // Pick the first expr optimizable by this index.
-        List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+        List<Integer> indexExprs = analysisCtx.getIndexExprs(index);
         int firstExprIndex = indexExprs.get(0);
         IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(firstExprIndex);
-        
+
         // Get the number of dimensions corresponding to the field indexed by
         // chosenIndex.
-        Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(optFuncExpr.getFieldName(0), recordType);
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(optFuncExpr.getFieldName(0), recordType);
         IAType spatialType = keyPairType.first;
         int numDimensions = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         int numSecondaryKeys = numDimensions * 2;
-        
+
         DataSourceScanOperator dataSourceScan = subTree.dataSourceScan;
         // TODO: For now retainInput and requiresBroadcast are always false.
-        RTreeJobGenParams jobGenParams = new RTreeJobGenParams(chosenIndex.getIndexName(), IndexKind.RTREE, datasetDecl.getName(), false, false);
+        RTreeJobGenParams jobGenParams = new RTreeJobGenParams(index.getIndexName(), IndexType.RTREE,
+                dataset.getDatasetName(), false, false);
         // A spatial object is serialized in the constant of the func expr we are optimizing.
         // The R-Tree expects as input an MBR represented with 1 field per dimension. 
         // Here we generate vars and funcs for extracting MBR fields from the constant into fields of a tuple (as the R-Tree expects them).
@@ -96,14 +98,16 @@
             AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CREATE_MBR));
             // Spatial object is the constant from the func expr we are optimizing.
-            createMBR.getArguments().add(new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
+            createMBR.getArguments().add(
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(optFuncExpr.getConstantVal(0))));
             // The number of dimensions.
             createMBR.getArguments().add(
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                            new AInt32(numDimensions)))));
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
+                            numDimensions)))));
             // Which part of the MBR to extract.
             createMBR.getArguments().add(
-                    new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(
+                            new AsterixConstantValue(new AInt32(i)))));
             // Add a variable and its expr to the lists which will be passed into an assign op.
             LogicalVariable keyVar = context.newVar();
             keyVarList.add(keyVar);
@@ -117,27 +121,26 @@
         assignSearchKeys.getInputs().add(dataSourceScan.getInputs().get(0));
         assignSearchKeys.setExecutionMode(dataSourceScan.getExecutionMode());
 
-        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(datasetDecl,
-                recordType, chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
+        UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
+                index, assignSearchKeys, jobGenParams, context, false, false);
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
-        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan,
-                datasetDecl, recordType, secondaryIndexUnnestOp, context, true, false, false);
+        UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
+                recordType, secondaryIndexUnnestOp, context, true, false, false);
         // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
         subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
         return true;
     }
-    
+
     @Override
     public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree,
-            AqlCompiledIndexDecl chosenIndex, AccessMethodAnalysisContext analysisCtx, IOptimizationContext context)
-            throws AlgebricksException {
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
         // TODO Implement this.
         return false;
     }
-    
+
     @Override
-    public boolean exprIsOptimizable(AqlCompiledIndexDecl index, IOptimizableFuncExpr optFuncExpr) {
+    public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
         // No additional analysis required.
         return true;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
index 2d850b2..b3153f9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/RTreeJobGenParams.java
@@ -5,7 +5,7 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 
@@ -14,36 +14,33 @@
  * and from a list of function arguments, typically of an unnest-map.
  */
 public class RTreeJobGenParams extends AccessMethodJobGenParams {
-    
+
     protected List<LogicalVariable> keyVarList;
-    
+
     public RTreeJobGenParams() {
     }
-    
-    public RTreeJobGenParams(String indexName, IndexKind indexKind, String datasetName, boolean retainInput, boolean requiresBroadcast) {
-        this.indexName = indexName;
-        this.indexKind = indexKind;
-        this.datasetName = datasetName;
-        this.retainInput = retainInput;
-        this.requiresBroadcast = requiresBroadcast;
+
+    public RTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+            boolean requiresBroadcast) {
+        super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
     }
-    
+
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.writeToFuncArgs(funcArgs);
         writeVarList(keyVarList, funcArgs);
     }
-    
+
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         super.readFromFuncArgs(funcArgs);
         int index = super.getNumParams();
         keyVarList = new ArrayList<LogicalVariable>();
         readVarList(funcArgs, index, keyVarList);
     }
-    
+
     public void setKeyVarList(List<LogicalVariable> keyVarList) {
         this.keyVarList = keyVarList;
     }
-    
+
     public List<LogicalVariable> getKeyVarList() {
         return keyVarList;
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
index b1970ac..e004a7d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -80,8 +80,6 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledInternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.metadata.declared.AqlLogicalPlanAndMetadataImpl;
@@ -89,6 +87,8 @@
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
 import edu.uci.ics.asterix.metadata.declared.FileSplitSinkId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -218,34 +218,31 @@
             topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
         } else {
             String dataVerseName = compiledDeclarations.getDataverseName();
-            AqlCompiledDatasetDecl adecl = compiledDeclarations.findDataset(outputDatasetName);
-            if (adecl == null) {
+            Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+            if (dataset == null) {
                 throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
             }
 
             AqlSourceId sourceId = new AqlSourceId(dataVerseName, outputDatasetName);
-            String itemTypeName = adecl.getItemTypeName();
+            String itemTypeName = dataset.getItemTypeName();
             IAType itemType = compiledDeclarations.findType(itemTypeName);
-            AqlDataSource dataSource = new AqlDataSource(sourceId, adecl, itemType);
-
-            if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+            AqlDataSource dataSource = new AqlDataSource(sourceId, dataset, itemType);
+            if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
                 throw new AlgebricksException("Cannot write output to an external dataset.");
             }
             ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
             ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
             List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
 
-            AqlCompiledInternalDatasetDetails datasetDetails = (AqlCompiledInternalDatasetDetails) adecl
-                    .getAqlCompiledDatasetDetails();
-            List<String> partitionKeys = datasetDetails.getPartitioningExprs();
+            List<String> partitionKeys = DatasetUtils.getPartitioningKeys(dataset);
             for (String keyFieldName : partitionKeys) {
                 IFunctionInfo finfoAccess = AsterixBuiltinFunctions
                         .getAsterixFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
                 @SuppressWarnings("unchecked")
                 ScalarFunctionCallExpression f = new ScalarFunctionCallExpression(finfoAccess,
                         new MutableObject<ILogicalExpression>(new VariableReferenceExpression(METADATA_DUMMY_VAR)),
-                        new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AString(
-                                keyFieldName)))));
+                        new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                                new AString(keyFieldName)))));
                 f.substituteVar(METADATA_DUMMY_VAR, resVar);
                 exprs.add(new MutableObject<ILogicalExpression>(f));
                 LogicalVariable v = context.newVar();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
index e6aaaad..d8f294d 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AqlPlusExpressionToPlanTranslator.java
@@ -81,12 +81,12 @@
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlLogicalPlanAndMetadataImpl;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.FileSplitDataSink;
 import edu.uci.ics.asterix.metadata.declared.FileSplitSinkId;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AString;
@@ -94,6 +94,7 @@
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.AsterixFunction;
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
+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;
@@ -144,17 +145,13 @@
 import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 
 /**
- * 
  * Each visit returns a pair of an operator and a variable. The variable
  * corresponds to the new column, if any, added to the tuple flow. E.g., for
  * Unnest, the column is the variable bound to the elements in the list, for
  * Subplan it is null.
- * 
  * The first argument of a visit method is the expression which is translated.
- * 
  * The second argument of a visit method is the tuple source for the current
  * subtree.
- * 
  */
 
 public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator implements
@@ -254,19 +251,21 @@
             topOp = new WriteOperator(writeExprList, sink);
             topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
         } else {
-            AqlCompiledDatasetDecl adecl = compiledDeclarations.findDataset(outputDatasetName);
-            if (adecl == null) {
+            Dataset dataset = compiledDeclarations.findDataset(outputDatasetName);
+            if (dataset == null) {
                 throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
             }
-
-            if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+            if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
                 throw new AlgebricksException("Cannot write output to an external dataset.");
             }
+            ARecordType itemType = (ARecordType) compiledDeclarations.findType(dataset.getItemTypeName());
+            List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
             ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
             ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
             List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
-            for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner : DatasetUtils
-                    .getPartitioningFunctions(adecl)) {
+            for (String partitioningKey : partitioningKeys) {
+                Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> partitioner = format
+                        .partitioningEvaluatorFactory(itemType, partitioningKey);
                 AbstractFunctionCallExpression f = partitioner.second.cloneExpression();
                 f.substituteVar(METADATA_DUMMY_VAR, resVar);
                 exprs.add(new MutableObject<ILogicalExpression>(f));
@@ -321,8 +320,8 @@
             returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)));
         } else {
             LogicalVariable pVar = context.newVar(fc.getPosVarExpr());
-            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)), pVar,
-                    BuiltinType.AINT32);
+            returnedOp = new UnnestOperator(v, new MutableObject<ILogicalExpression>(makeUnnestExpression(eo.first)),
+                    pVar, BuiltinType.AINT32);
         }
         returnedOp.getInputs().add(eo.second);
 
@@ -339,8 +338,8 @@
             case VARIABLE_EXPRESSION: {
                 v = context.newVar(lc.getVarExpr());
                 LogicalVariable prev = context.getVar(((VariableExpr) lc.getBindingExpr()).getVar().getId());
-                returnedOp = new AssignOperator(v,
-                        new MutableObject<ILogicalExpression>(new VariableReferenceExpression(prev)));
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(prev)));
                 returnedOp.getInputs().add(tupSource);
                 break;
             }
@@ -429,15 +428,16 @@
             f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM));
             f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
             f.getArguments().add(
-                    new  MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(i)))));
+                    new MutableObject<ILogicalExpression>(new ConstantExpression(
+                            new AsterixConstantValue(new AInt32(i)))));
         }
-        AssignOperator a = new AssignOperator(v, new  MutableObject<ILogicalExpression>(f));
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
         a.getInputs().add(p.second);
         return new Pair<ILogicalOperator, LogicalVariable>(a, v);
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall,  Mutable<ILogicalOperator> tupSource)
+    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
             throws AsterixException {
         LogicalVariable v = context.newVar();
         AsterixFunction fid = fcall.getIdent();
@@ -469,11 +469,11 @@
             }
         }
 
-        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName(), false);
+        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName(),
+                false);
         AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
         FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
 
-
         if (builtinAquafi != null) {
             fi = builtinAquafi;
         } else {
@@ -503,7 +503,8 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+            Mutable<ILogicalOperator> tupSource) {
         // TODO Auto-generated method stub
         throw new NotImplementedException();
     }
@@ -547,9 +548,10 @@
             AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
                     AsterixBuiltinFunctions.LISTIFY, flArgs);
             AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
-                    (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
+                    (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
             agg.getInputs().add(
-                    new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(gOp))));
+                    new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                            new MutableObject<ILogicalOperator>(gOp))));
             ILogicalPlan plan = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(agg));
             gOp.getNestedPlans().add(plan);
             // Hide the variable that was part of the "with", replacing it with
@@ -583,8 +585,8 @@
                 new MutableObject<ILogicalOperator>(sp)));
 
         Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
-        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(
-                new VariableReferenceExpression(varCond)));
+        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                varCond)));
         sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
         Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
@@ -604,9 +606,9 @@
 
         LogicalVariable resV = context.newVar();
         AbstractFunctionCallExpression concatNonNull = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL), new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(pThen.second)), new MutableObject<ILogicalExpression>(
-                        new VariableReferenceExpression(pElse.second)));
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CONCAT_NON_NULL),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pThen.second)),
+                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(pElse.second)));
         AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
         a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
@@ -625,8 +627,8 @@
     }
 
     @Override
-    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-            throws AsterixException {
+    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
         ArrayList<OperatorType> ops = op.getOpList();
         int nOps = ops.size();
 
@@ -683,7 +685,8 @@
                     }
                 }
             } else { // don't forget the last expression...
-                ((AbstractFunctionCallExpression) currExpr).getArguments().add(new MutableObject<ILogicalExpression>(e));
+                ((AbstractFunctionCallExpression) currExpr).getArguments()
+                        .add(new MutableObject<ILogicalExpression>(e));
                 if (i == 1 && op.isBroadcastOperand(i)) {
                     BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
                     bcast.setObject(BroadcastSide.RIGHT);
@@ -712,8 +715,9 @@
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(e, topOp);
             OrderModifier m = modifIter.next();
             OrderOperator.IOrder comp = (m == OrderModifier.ASC) ? OrderOperator.ASC_ORDER : OrderOperator.DESC_ORDER;
-            ord.getOrderExpressions().add(
-                    new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(p.first)));
+            ord.getOrderExpressions()
+                    .add(new Pair<IOrder, Mutable<ILogicalExpression>>(comp, new MutableObject<ILogicalExpression>(
+                            p.first)));
             topOp = p.second;
         }
         ord.getInputs().add(topOp);
@@ -775,7 +779,7 @@
         }
         LogicalVariable qeVar = context.newVar();
         AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
-                (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
         a.getInputs().add(new MutableObject<ILogicalOperator>(s));
         return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
     }
@@ -848,8 +852,8 @@
         // Should we ever get to this method?
         LogicalVariable var = context.newVar();
         LogicalVariable oldV = context.getVar(v.getVar().getId());
-        AssignOperator a = new AssignOperator(var,
-                new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldV)));
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+                new VariableReferenceExpression(oldV)));
         a.getInputs().add(tupSource);
         return new Pair<ILogicalOperator, LogicalVariable>(a, var);
     }
@@ -1122,7 +1126,7 @@
         funAgg.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
         LogicalVariable varListified = context.newVar();
         AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(varListified),
-                (List)mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(funAgg)));
         agg.getInputs().add(opRef);
         ILogicalOperator res;
         if (bProject) {
@@ -1238,7 +1242,6 @@
         return null;
     }
 
-  
     @Override
     public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
             Mutable<ILogicalOperator> arg) throws AsterixException {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
index d0e9626..c8b1079 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
@@ -35,7 +35,6 @@
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
@@ -461,11 +460,11 @@
                 clauseList.add(dieClause);
             }
 
-            AqlCompiledDatasetDecl aqlDataset = compiledDeclarations.findDataset(datasetName);
-            if (aqlDataset == null) {
+            Dataset dataset = compiledDeclarations.findDataset(datasetName);
+            if (dataset == null) {
                 throw new AlgebricksException("Unknown dataset " + datasetName);
             }
-            String itemTypeName = aqlDataset.getItemTypeName();
+            String itemTypeName = dataset.getItemTypeName();
             IAType itemType = compiledDeclarations.findType(itemTypeName);
             ARecordType recType = (ARecordType) itemType;
             String[] fieldNames = recType.getFieldNames();
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
index 8c32522..dc3fb25 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/DdlTranslator.java
@@ -308,7 +308,7 @@
                         List<Dataset> datasets = MetadataManager.INSTANCE.getDataverseDatasets(mdTxnCtx, dvName);
                         for (int j = 0; j < datasets.size(); j++) {
                             String datasetName = datasets.get(j).getDatasetName();
-                            DatasetType dsType = datasets.get(j).getType();
+                            DatasetType dsType = datasets.get(j).getDatasetType();
                             if (dsType == DatasetType.INTERNAL || dsType == DatasetType.FEED) {
                                 List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx, dvName,
                                         datasetName);
@@ -341,7 +341,7 @@
                         if (!stmtDelete.getIfExists())
                             throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
                     } else {
-                        if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
+                        if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
                             List<Index> indexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
                                     compiledDeclarations.getDataverseName(), datasetName);
                             for (int j = 0; j < indexes.size(); j++) {
@@ -362,7 +362,7 @@
                             datasetName);
                     if (ds == null)
                         throw new AlgebricksException("There is no dataset with this name " + datasetName + ".");
-                    if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
+                    if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
                         String indexName = stmtDelete.getIndexName().getValue();
                         Index idx = MetadataManager.INSTANCE.getIndex(mdTxnCtx,
                                 compiledDeclarations.getDataverseName(), datasetName, indexName);
@@ -502,7 +502,7 @@
         CompiledDatasetDropStatement cds = new CompiledDatasetDropStatement(datasetName);
         Dataset ds = MetadataManager.INSTANCE
                 .getDataset(mdTxnCtx, compiledDeclarations.getDataverseName(), datasetName);
-        if (ds.getType() == DatasetType.INTERNAL || ds.getType() == DatasetType.FEED) {
+        if (ds.getDatasetType() == DatasetType.INTERNAL || ds.getDatasetType() == DatasetType.FEED) {
             JobSpecification[] jobs = DatasetOperations.createDropDatasetJobSpec(cds, compiledDeclarations);
             for (JobSpecification job : jobs)
                 runJob(hcc, job);
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 5aa0f6e..b028e06 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
@@ -27,11 +27,11 @@
 import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -95,32 +95,38 @@
         IIndexRegistryProvider<IIndex> indexRegistryProvider = AsterixIndexRegistryProvider.INSTANCE;
         IStorageManagerInterface storageManager = AsterixStorageManagerInterface.INSTANCE;
 
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("DROP DATASET: No metadata for dataset " + datasetName);
         }
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return new JobSpecification[0];
         }
 
-        List<AqlCompiledIndexDecl> secondaryIndexes = DatasetUtils.getSecondaryIndexes(adecl);
-
+        List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        int numSecondaryIndexes = 0;
+        for (Index index : datasetIndexes) {
+            if (index.isSecondaryIndex()) {
+                numSecondaryIndexes++;
+            }
+        }
         JobSpecification[] specs;
-
-        if (secondaryIndexes != null && !secondaryIndexes.isEmpty()) {
-            int n = secondaryIndexes.size();
-            specs = new JobSpecification[n + 1];
+        if (numSecondaryIndexes > 0) {
+            specs = new JobSpecification[numSecondaryIndexes + 1];
             int i = 0;
             // First, drop secondary indexes.
-            for (AqlCompiledIndexDecl acid : secondaryIndexes) {
-                specs[i] = new JobSpecification();
-                Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
-                        .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, acid.getIndexName());
-                TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
-                        storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
-                AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
-                        idxSplitsAndConstraint.second);
-                i++;
+            for (Index index : datasetIndexes) {
+                if (index.isSecondaryIndex()) {
+                    specs[i] = new JobSpecification();
+                    Pair<IFileSplitProvider, AlgebricksPartitionConstraint> idxSplitsAndConstraint = metadata
+                            .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
+                                    index.getIndexName());
+                    TreeIndexDropOperatorDescriptor secondaryBtreeDrop = new TreeIndexDropOperatorDescriptor(specs[i],
+                            storageManager, indexRegistryProvider, idxSplitsAndConstraint.first);
+                    AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specs[i], secondaryBtreeDrop,
+                            idxSplitsAndConstraint.second);
+                    i++;
+                }
             }
         } else {
             specs = new JobSpecification[1];
@@ -140,17 +146,17 @@
         return specs;
     }
 
-    // TODO: Lots of common code in this file. Refactor everything after merging in asterix-fix-issue-9.
     public static JobSpecification createDatasetJobSpec(String datasetName, AqlCompiledMetadataDeclarations metadata)
             throws AsterixException, AlgebricksException {
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AsterixException("Could not find dataset " + datasetName);
         }
+        ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         JobSpecification spec = new JobSpecification();
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, metadata.getFormat().getBinaryComparatorFactoryProvider());
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
         FileSplit[] fs = splitsAndConstraint.first.getFileSplits();
@@ -170,49 +176,47 @@
         return spec;
     }
 
+    @SuppressWarnings("rawtypes")
     public static Job createLoadDatasetJobSpec(CompiledLoadFromFileStatement loadStmt,
             AqlCompiledMetadataDeclarations metadata) throws AsterixException, AlgebricksException {
         String datasetName = loadStmt.getDatasetName();
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AsterixException("Could not find dataset " + datasetName);
         }
-        if (compiledDatasetDecl.getDatasetType() != DatasetType.INTERNAL
-                && compiledDatasetDecl.getDatasetType() != DatasetType.FEED) {
+        if (dataset.getDatasetType() != DatasetType.INTERNAL && dataset.getDatasetType() != DatasetType.FEED) {
             throw new AsterixException("Cannot load data into dataset  (" + datasetName + ")" + "of type "
-                    + compiledDatasetDecl.getDatasetType());
+                    + dataset.getDatasetType());
         }
         JobSpecification spec = new JobSpecification();
 
-        ARecordType itemType = (ARecordType) metadata.findType(compiledDatasetDecl.getItemTypeName());
+        ARecordType itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         IDataFormat format = metadata.getFormat();
         ISerializerDeserializer payloadSerde = format.getSerdeProvider().getSerializerDeserializer(itemType);
 
-        IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(
-                compiledDatasetDecl, metadata.getFormat().getBinaryHashFunctionFactoryProvider());
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, metadata.getFormat().getBinaryComparatorFactoryProvider());
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        IBinaryHashFunctionFactory[] hashFactories = DatasetUtils.computeKeysBinaryHashFunFactories(dataset, itemType,
+                metadata.getFormat().getBinaryHashFunctionFactoryProvider());
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, metadata.getFormat().getBinaryComparatorFactoryProvider());
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
 
-        AqlCompiledExternalDatasetDetails externalDatasetDetails = new AqlCompiledExternalDatasetDetails(
-                loadStmt.getAdapter(), loadStmt.getProperties());
+        ExternalDatasetDetails externalDatasetDetails = new ExternalDatasetDetails(loadStmt.getAdapter(),
+                loadStmt.getProperties());
         Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = AqlMetadataProvider
                 .buildExternalDataScannerRuntime(spec, itemType, externalDatasetDetails, format);
         IOperatorDescriptor scanner = p.first;
         AlgebricksPartitionConstraint scannerPc = p.second;
-        RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(compiledDatasetDecl, payloadSerde,
+        RecordDescriptor recDesc = computePayloadKeyRecordDescriptor(dataset, itemType, payloadSerde,
                 metadata.getFormat());
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, scanner, scannerPc);
 
-        AssignRuntimeFactory assign = makeAssignRuntimeFactory(compiledDatasetDecl);
+        AssignRuntimeFactory assign = makeAssignRuntimeFactory(dataset, itemType, metadata.getFormat());
         AlgebricksMetaOperatorDescriptor asterixOp = new AlgebricksMetaOperatorDescriptor(spec, 1, 1,
                 new IPushRuntimeFactory[] { assign }, new RecordDescriptor[] { recDesc });
 
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, asterixOp, scannerPc);
 
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
+        int numKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         int[] keys = new int[numKeys];
         for (int i = 0; i < numKeys; i++) {
             keys[i] = i + 1;
@@ -270,14 +274,14 @@
         return fs.getNodeName() + ":" + fs.getLocalFile().toString();
     }
 
-    private static AssignRuntimeFactory makeAssignRuntimeFactory(AqlCompiledDatasetDecl compiledDatasetDecl) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
+    private static AssignRuntimeFactory makeAssignRuntimeFactory(Dataset dataset, ARecordType itemType,
+            IDataFormat format) throws AlgebricksException {
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
         ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys];
         for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
+            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = format
+                    .partitioningEvaluatorFactory(itemType, partitioningKeys.get(i));
             evalFactories[i] = evalFactoryAndType.first;
         }
         int[] outColumns = new int[numKeys];
@@ -296,21 +300,18 @@
         return new AssignRuntimeFactory(outColumns, sefs, projectionList);
     }
 
-    private static RecordDescriptor computePayloadKeyRecordDescriptor(AqlCompiledDatasetDecl compiledDatasetDecl,
+    @SuppressWarnings("rawtypes")
+    private static RecordDescriptor computePayloadKeyRecordDescriptor(Dataset dataset, ARecordType itemType,
             ISerializerDeserializer payloadSerde, IDataFormat dataFormat) throws AlgebricksException {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
         ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys];
         recordFields[0] = payloadSerde;
         for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             ISerializerDeserializer keySerde = dataFormat.getSerdeProvider().getSerializerDeserializer(keyType);
             recordFields[i + 1] = keySerde;
         }
         return new RecordDescriptor(recordFields);
     }
-
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
index 4d7cbe2..66a5d56 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/FeedOperations.java
@@ -24,30 +24,18 @@
 import edu.uci.ics.asterix.feed.comm.FeedMessage;
 import edu.uci.ics.asterix.feed.comm.IFeedMessage;
 import edu.uci.ics.asterix.feed.comm.IFeedMessage.MessageType;
-import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledFeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
-import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
-import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
 import edu.uci.ics.asterix.translator.DmlTranslator.CompiledControlFeedStatement;
 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.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.operators.std.AssignRuntimeFactory;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
-import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.RecordDescriptor;
 import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
 import edu.uci.ics.hyracks.dataflow.std.misc.NullSinkOperatorDescriptor;
 
 public class FeedOperations {
@@ -77,12 +65,17 @@
 
         LOGGER.info(" DATASETPATH: " + datasetPath);
 
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset;
+        try {
+            dataset = metadata.findDataset(datasetName);
+        } catch (AlgebricksException e) {
+            throw new AsterixException(e);
+        }
+        if (dataset == null) {
             throw new AsterixException("FEED DATASET: No metadata for dataset " + datasetName);
         }
-        if (adecl.getDatasetType() != DatasetType.FEED) {
-            throw new AsterixException("Operation not support for dataset type  " + adecl.getDatasetType());
+        if (dataset.getDatasetType() != DatasetType.FEED) {
+            throw new AsterixException("Operation not support for dataset type  " + dataset.getDatasetType());
         }
 
         JobSpecification spec = new JobSpecification();
@@ -107,8 +100,8 @@
 
         try {
             Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> p = AqlMetadataProvider.buildFeedMessengerRuntime(
-                    spec, metadata, (AqlCompiledFeedDatasetDetails) adecl.getAqlCompiledDatasetDetails(),
-                    metadata.getDataverseName(), datasetName, feedMessages);
+                    spec, metadata, (FeedDatasetDetails) dataset.getDatasetDetails(), metadata.getDataverseName(),
+                    datasetName, feedMessages);
             feedMessenger = p.first;
             messengerPc = p.second;
         } catch (AlgebricksException e) {
@@ -126,54 +119,4 @@
         return spec;
 
     }
-
-    private static AssignRuntimeFactory makeAssignRuntimeFactory(AqlCompiledDatasetDecl compiledDatasetDecl) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
-        ICopyEvaluatorFactory[] evalFactories = new ICopyEvaluatorFactory[numKeys];
-
-        int index = 0;
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
-            evalFactories[index++] = evalFactoryAndType.first;
-        }
-
-        int[] outColumns = new int[numKeys];
-        int[] projectionList = new int[numKeys + 1];
-        projectionList[0] = 0;
-
-        for (int i = 0; i < numKeys; i++) {
-            outColumns[i] = i + 1;
-            projectionList[i + 1] = i + 1;
-        }
-        IScalarEvaluatorFactory[] sefs = new IScalarEvaluatorFactory[evalFactories.length];
-        for (int i = 0; i < evalFactories.length; ++i) {
-            sefs[i] = new LogicalExpressionJobGenToExpressionRuntimeProviderAdapter.ScalarEvaluatorFactoryAdapter(
-                    evalFactories[i]);
-        }
-        return new AssignRuntimeFactory(outColumns, sefs, projectionList);
-    }
-
-    @SuppressWarnings("unchecked")
-    private static RecordDescriptor computePayloadKeyRecordDescriptor(AqlCompiledDatasetDecl compiledDatasetDecl,
-            ISerializerDeserializer payloadSerde, IDataFormat dataFormat) throws AlgebricksException {
-
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
-        ISerializerDeserializer[] recordFields = new ISerializerDeserializer[1 + numKeys];
-        recordFields[0] = payloadSerde;
-        int index = 0;
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
-            IAType keyType = evalFactoryAndType.third;
-            ISerializerDeserializer keySerde = dataFormat.getSerdeProvider().getSerializerDeserializer(keyType);
-            recordFields[index + 1] = keySerde;
-            index++;
-        }
-        return new RecordDescriptor(recordFields);
-    }
-
-    private static String stringOf(FileSplit fs) {
-        return fs.getNodeName() + ":" + fs.getLocalFile().toString();
-    }
 }
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 c1be9f9..65f6f5e 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
@@ -12,9 +12,9 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
 import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+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.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -26,9 +26,7 @@
 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.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.LogicalExpressionJobGenToExpressionRuntimeProviderAdapter;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.ISerializerDeserializerProvider;
@@ -68,7 +66,7 @@
     protected int numSecondaryKeys;
     protected AqlCompiledMetadataDeclarations metadata;
     protected String datasetName;
-    protected AqlCompiledDatasetDecl datasetDecl;
+    protected Dataset dataset;
     protected ARecordType itemType;
     protected ISerializerDeserializer payloadSerde;
     protected IFileSplitProvider primaryFileSplitProvider;
@@ -124,16 +122,16 @@
         this.metadata = metadata;
         datasetName = createIndexStmt.getDatasetName();
         secondaryIndexName = createIndexStmt.getIndexName();
-        datasetDecl = metadata.findDataset(datasetName);
-        if (datasetDecl == null) {
+        dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AsterixException("Unknown dataset " + datasetName);
         }
-        if (datasetDecl.getDatasetType() == DatasetType.EXTERNAL) {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AsterixException("Cannot index an external dataset (" + datasetName + ").");
         }
-        itemType = (ARecordType) metadata.findType(datasetDecl.getItemTypeName());
+        itemType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         payloadSerde = AqlSerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(itemType);
-        numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+        numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         numSecondaryKeys = createIndexStmt.getKeyFields().size();
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> primarySplitsAndConstraint = metadata
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, datasetName);
@@ -149,22 +147,18 @@
     }
 
     protected void setPrimaryRecDescAndComparators() throws AlgebricksException {
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numPrimaryKeys = partitioningKeys.size();
         ISerializerDeserializer[] primaryRecFields = new ISerializerDeserializer[numPrimaryKeys + 1];
         ITypeTraits[] primaryTypeTraits = new ITypeTraits[numPrimaryKeys + 1];
         primaryComparatorFactories = new IBinaryComparatorFactory[numPrimaryKeys];
         ISerializerDeserializerProvider serdeProvider = metadata.getFormat().getSerdeProvider();
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(datasetDecl);
-        int i = 0;
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : partitioningFunctions) {
-            IAType keyType = evalFactoryAndType.third;
-            ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
-            primaryRecFields[i] = keySerde;
+        for (int i = 0; i < numPrimaryKeys; i++) {
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
+            primaryRecFields[i] = serdeProvider.getSerializerDeserializer(keyType);
             primaryComparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
                     keyType, true);
             primaryTypeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
-            ++i;
         }
         primaryRecFields[numPrimaryKeys] = payloadSerde;
         primaryTypeTraits[numPrimaryKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
@@ -185,8 +179,7 @@
         for (int i = 0; i < numSecondaryKeys; i++) {
             secondaryFieldAccessEvalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType,
                     secondaryKeyFields.get(i), numPrimaryKeys);
-            Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                    secondaryKeyFields.get(i), itemType);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
             IAType keyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(keyType);
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 224401f..d5e8222 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
@@ -5,7 +5,7 @@
 import edu.uci.ics.asterix.common.context.AsterixIndexRegistryProvider;
 import edu.uci.ics.asterix.common.context.AsterixStorageManagerInterface;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.optimizer.rules.am.InvertedIndexAccessMethod;
 import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
@@ -73,8 +73,7 @@
         for (int i = 0; i < numSecondaryKeys; i++) {
             secondaryFieldAccessEvalFactories[i] = metadata.getFormat().getFieldAccessEvaluatorFactory(itemType,
                     secondaryKeyFields.get(i), numPrimaryKeys);
-            Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                    secondaryKeyFields.get(i), itemType);
+            Pair<IAType, Boolean> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(i), itemType);
             secondaryKeyType = keyTypePair.first;
             anySecondaryKeyIsNullable = anySecondaryKeyIsNullable || keyTypePair.second;
             ISerializerDeserializer keySerde = serdeProvider.getSerializerDeserializer(secondaryKeyType);
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 6399ae3..3127573 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
@@ -9,7 +9,7 @@
 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.AqlCompiledIndexDecl;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
 import edu.uci.ics.asterix.translator.DmlTranslator.CompiledCreateIndexStatement;
@@ -69,8 +69,7 @@
                             + 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 = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                secondaryKeyFields.get(0), itemType);
+        Pair<IAType, Boolean> spatialTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), itemType);
         IAType spatialType = spatialTypePair.first;
         anySecondaryKeyIsNullable = spatialTypePair.second;
         if (spatialType == null) {
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index ede1063..03fd04e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -15,9 +15,8 @@
                             -- STREAM_PROJECT  |PARTITIONED|
                               -- ASSIGN  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
-                                  -- ASSIGN  |PARTITIONED|
-                                    -- STREAM_SELECT  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
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 6ac37c3..591154a 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
@@ -91,7 +91,7 @@
     private MetadataNode() {
         super();
     }
-    
+
     public void initialize(AsterixAppRuntimeContext runtimeContext) {
         this.transactionProvider = runtimeContext.getTransactionProvider();
         this.indexRegistry = runtimeContext.getIndexRegistry();
@@ -152,7 +152,7 @@
             DatasetTupleTranslator tupleReaderWriter = new DatasetTupleTranslator(true);
             ITupleReference datasetTuple = tupleReaderWriter.getTupleFromMetadataEntity(dataset);
             insertTupleIntoIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
-            if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
                 // Add the primary index for the dataset.
                 InternalDatasetDetails id = (InternalDatasetDetails) dataset.getDatasetDetails();
                 Index primaryIndex = new Index(dataset.getDataverseName(), dataset.getDatasetName(),
@@ -163,7 +163,7 @@
                 insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, nodeGroupTuple);
             }
             // Add entry in datatype secondary index.
-            ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getDatatypeName(),
+            ITupleReference dataTypeTuple = createTuple(dataset.getDataverseName(), dataset.getItemTypeName(),
                     dataset.getDatasetName());
             insertTupleIntoIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
         } catch (BTreeDuplicateKeyException e) {
@@ -320,7 +320,7 @@
             ITupleReference datasetTuple = getTupleToBeDeleted(txnId, MetadataPrimaryIndexes.DATASET_DATASET, searchKey);
             deleteTupleFromIndex(txnId, MetadataPrimaryIndexes.DATASET_DATASET, datasetTuple);
             // Delete entry from secondary index 'group'.
-            if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+            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
@@ -330,14 +330,14 @@
                 deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.GROUPNAME_ON_DATASET_INDEX, groupNameTuple);
             }
             // Delete entry from secondary index 'type'.
-            ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getDatatypeName(), datasetName);
+            ITupleReference dataTypeSearchKey = createTuple(dataverseName, dataset.getItemTypeName(), datasetName);
             // Searches the index for the tuple to be deleted. Acquires an S
             // lock on the DATATYPENAME_ON_DATASET_INDEX index.
             ITupleReference dataTypeTuple = getTupleToBeDeleted(txnId,
                     MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeSearchKey);
             deleteTupleFromIndex(txnId, MetadataSecondaryIndexes.DATATYPENAME_ON_DATASET_INDEX, dataTypeTuple);
             // Delete entry(s) from the 'indexes' dataset.
-            if (dataset.getType() == DatasetType.INTERNAL || dataset.getType() == DatasetType.FEED) {
+            if (dataset.getDatasetType() == DatasetType.INTERNAL || dataset.getDatasetType() == DatasetType.FEED) {
                 List<Index> datasetIndexes = getDatasetIndexes(txnId, dataverseName, datasetName);
                 for (Index index : datasetIndexes) {
                     dropIndex(txnId, dataverseName, datasetName, index.getIndexName());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java
deleted file mode 100644
index 1d806b1..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledDatasetDecl.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public class AqlCompiledDatasetDecl {
-
-    private String name;
-    private String itemTypeName;
-    private DatasetType datasetType;
-    private IAqlCompiledDatasetDetails acdd;
-
-    public AqlCompiledDatasetDecl(String name, String itemTypeName, DatasetType datasetType,
-            IAqlCompiledDatasetDetails acdd) {
-        this.name = name;
-        this.itemTypeName = itemTypeName;
-        this.datasetType = datasetType;
-        this.acdd = acdd;
-    }
-
-    public String getName() {
-        return name;
-    }
-
-    public String getItemTypeName() {
-        return itemTypeName;
-    }
-
-    public DatasetType getDatasetType() {
-        return datasetType;
-    }
-
-    public IAqlCompiledDatasetDetails getAqlCompiledDatasetDetails() {
-        return acdd;
-    }
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java
deleted file mode 100644
index 2e5eb3a..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledExternalDatasetDetails.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public class AqlCompiledExternalDatasetDetails implements IAqlCompiledDatasetDetails {
-
-    private final String adapter;
-    private final Map<String, String> properties;
-
-    public AqlCompiledExternalDatasetDetails(String adapter, Map<String, String> properties) {
-        this.adapter = adapter;
-        this.properties = properties;
-    }
-
-    public String getAdapter() {
-        return adapter;
-    }
-
-    public Map<String, String> getProperties() {
-        return properties;
-    }
-
-    @Override
-    public DatasetType getDatasetType() {
-        return DatasetType.EXTERNAL;
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java
deleted file mode 100644
index 5cfc8be..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledFeedDatasetDetails.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class AqlCompiledFeedDatasetDetails extends
-		AqlCompiledInternalDatasetDetails {
-	private final String adapter;
-	private final Map<String, String> properties;
-	private final String functionIdentifier;
-	private final String feedState;
-
-	public AqlCompiledFeedDatasetDetails(
-			List<String> partitioningExprs,
-			List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns,
-			String nodegroupName, AqlCompiledIndexDecl primaryIndex,
-			List<AqlCompiledIndexDecl> secondaryIndexes, String adapter,
-			Map<String, String> properties, String functionIdentifier,
-			String feedState) {
-		super(partitioningExprs, partitionFuns, nodegroupName, primaryIndex,
-				secondaryIndexes);
-		this.adapter = adapter;
-		this.properties = properties;
-		this.functionIdentifier = functionIdentifier;
-		this.feedState = feedState;
-	}
-
-	public String getAdapter() {
-		return adapter;
-	}
-
-	public Map<String, String> getProperties() {
-		return properties;
-	}
-
-	public String getFunctionIdentifier() {
-		return functionIdentifier;
-	}
-
-	public DatasetType getDatasetType() {
-		return DatasetType.FEED;
-	}
-
-	public String getFeedState() {
-		return feedState;
-	}
-
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java
deleted file mode 100644
index 5755331..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledIndexDecl.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.AUnionType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-
-public class AqlCompiledIndexDecl {
-
-    public enum IndexKind {
-        BTREE,
-        RTREE,
-        WORD_INVIX,
-        NGRAM_INVIX
-    }
-
-    private String indexName;
-    private IndexKind kind;
-    private List<String> fieldExprs = new ArrayList<String>();
-    // Only for NGRAM indexes.
-    private int gramLength;
-
-    public AqlCompiledIndexDecl(String indexName, IndexKind kind, List<String> fieldExprs, int gramLength) {
-        this.indexName = indexName;
-        this.kind = kind;
-        this.fieldExprs = fieldExprs;
-        this.gramLength = gramLength;
-    }
-
-    public AqlCompiledIndexDecl(String indexName, IndexKind kind, List<String> fieldExprs) {
-        this.indexName = indexName;
-        this.kind = kind;
-        this.fieldExprs = fieldExprs;
-        this.gramLength = -1;
-    }
-
-    @Override
-    public String toString() {
-        return "INDEX " + indexName + " (" + kind + ") " + fieldExprs;
-    }
-
-    public IndexKind getKind() {
-        return kind;
-    }
-
-    public String getIndexName() {
-        return indexName;
-    }
-
-    public List<String> getFieldExprs() {
-        return fieldExprs;
-    }
-
-    public int getGramLength() {
-        return gramLength;
-    }
-
-    public static Pair<IAType, Boolean> getNonNullableKeyFieldType(String expr, ARecordType recType)
-            throws AlgebricksException {
-        IAType keyType = AqlCompiledIndexDecl.keyFieldType(expr, recType);
-        boolean nullable = false;
-        if (keyType.getTypeTag() == ATypeTag.UNION) {
-            AUnionType unionType = (AUnionType) keyType;
-            if (unionType.isNullableType()) {
-                // The non-null type is always at index 1.
-                keyType = unionType.getUnionList().get(1);
-                nullable = true;
-            }
-        }
-        return new Pair<IAType, Boolean>(keyType, nullable);
-    }
-
-    private static IAType keyFieldType(String expr, ARecordType recType) throws AlgebricksException {
-        String[] names = recType.getFieldNames();
-        int n = names.length;
-        for (int i = 0; i < n; i++) {
-            if (names[i].equals(expr)) {
-                return recType.getFieldTypes()[i];
-            }
-        }
-        throw new AlgebricksException("Could not find field " + expr + " in the schema.");
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java
deleted file mode 100644
index 912b766..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledInternalDatasetDetails.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
-
-public class AqlCompiledInternalDatasetDetails implements IAqlCompiledDatasetDetails {
-    private final List<String> partitioningExprs;
-    private final List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns;
-    private final String nodegroupName;
-    private final List<AqlCompiledIndexDecl> secondaryIndexes;
-    private final AqlCompiledIndexDecl primaryIndex;
-    private HashMap<String, List<AqlCompiledIndexDecl>> secondaryIndexInvertedList;
-
-    public AqlCompiledInternalDatasetDetails(List<String> partitioningExprs,
-            List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitionFuns, String nodegroupName,
-            AqlCompiledIndexDecl primaryIndex, List<AqlCompiledIndexDecl> secondaryIndexes) {
-        this.partitioningExprs = partitioningExprs;
-        this.partitionFuns = partitionFuns;
-        this.nodegroupName = nodegroupName;
-        this.primaryIndex = primaryIndex;
-        this.secondaryIndexes = secondaryIndexes;
-        invertSecondaryIndexExprs();
-    }
-
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("dataset partitioned-by " + partitionFuns + " on " + nodegroupName);
-        if (secondaryIndexes != null && !secondaryIndexes.isEmpty()) {
-            sb.append(System.getProperty("line.separator") + " with indexes: " + secondaryIndexes);
-        }
-        return sb.toString();
-    }
-
-    public List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> getPartitioningFunctions() {
-        return partitionFuns;
-    }
-
-    public List<String> getPartitioningExprs() {
-        return partitioningExprs;
-    }
-
-    public int getPositionOfPartitioningKeyField(String fieldName) {
-        int pos = 0;
-        for (String pe : partitioningExprs) {
-            if (pe.equals(fieldName)) {
-                return pos;
-            }
-            ++pos;
-        }
-        return -1;
-    }
-
-    public String getNodegroupName() {
-        return nodegroupName;
-    }
-
-    public List<AqlCompiledIndexDecl> getSecondaryIndexes() {
-        return secondaryIndexes;
-    }
-
-    public AqlCompiledIndexDecl getPrimaryIndex() {
-        return primaryIndex;
-    }
-
-    public List<AqlCompiledIndexDecl> findSecondaryIndexesByOneOfTheKeys(String fieldExpr) {
-        return secondaryIndexInvertedList.get(fieldExpr);
-    }
-
-    public AqlCompiledIndexDecl findSecondaryIndexByExactKeyList(List<String> fieldExprs) {
-        if (secondaryIndexes == null) {
-            return null;
-        }
-        for (AqlCompiledIndexDecl acid : secondaryIndexes) {
-            if (acid.getFieldExprs().equals(fieldExprs)) {
-                return acid;
-            }
-        }
-        return null;
-    }
-
-    public AqlCompiledIndexDecl findSecondaryIndexByName(String idxName) {
-        if (secondaryIndexes == null) {
-            return null;
-        }
-        for (AqlCompiledIndexDecl acid : secondaryIndexes) {
-            if (acid.getIndexName().equals(idxName)) {
-                return acid;
-            }
-        }
-        return null;
-    }
-
-    private void invertSecondaryIndexExprs() {
-        secondaryIndexInvertedList = new HashMap<String, List<AqlCompiledIndexDecl>>();
-        if (secondaryIndexes == null) {
-            return;
-        }
-        for (AqlCompiledIndexDecl idx : secondaryIndexes) {
-            for (String s : idx.getFieldExprs()) {
-                List<AqlCompiledIndexDecl> idxList = secondaryIndexInvertedList.get(s);
-                if (idxList == null) {
-                    idxList = new ArrayList<AqlCompiledIndexDecl>();
-                    secondaryIndexInvertedList.put(s, idxList);
-                }
-                idxList.add(idx);
-            }
-        }
-    }
-
-    @Override
-    public DatasetType getDatasetType() {
-        return DatasetType.INTERNAL;
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index 756fab4..ba7c797 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -23,7 +23,6 @@
 
 import edu.uci.ics.asterix.common.annotations.TypeDataGen;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataException;
@@ -31,25 +30,18 @@
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
 import edu.uci.ics.asterix.metadata.api.IMetadataManager;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl.IndexKind;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.metadata.entities.Datatype;
 import edu.uci.ics.asterix.metadata.entities.Dataverse;
-import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.entities.NodeGroup;
-import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.api.io.FileReference;
 import edu.uci.ics.hyracks.dataflow.std.file.ConstantFileSplitProvider;
 import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
@@ -81,10 +73,11 @@
         this.dataverseName = dataverseName;
         this.outputFile = outputFile;
         this.config = config;
-        if (stores == null && online)
+        if (stores == null && online) {
             this.stores = AsterixProperties.INSTANCE.getStores();
-        else
+        } else {
             this.stores = stores;
+        }
         this.types = types;
         this.typeDataGenMap = typeDataGenMap;
         this.writerFactory = writerFactory;
@@ -113,17 +106,16 @@
     }
 
     public void disconnectFromDataverse() throws AlgebricksException {
-        if (!isConnected)
+        if (!isConnected) {
             throw new AlgebricksException("You are not connected to any dataverse");
-        else {
-            dataverseName = null;
-            format = null;
-            isConnected = false;
         }
+        dataverseName = null;
+        format = null;
+        isConnected = false;
     }
 
     public boolean isConnectedToDataverse() {
-        return this.isConnected;
+        return isConnected;
     }
 
     public String getDataverseName() {
@@ -135,8 +127,9 @@
     }
 
     public IDataFormat getFormat() throws AlgebricksException {
-        if (!isConnected)
+        if (!isConnected) {
             throw new AlgebricksException("You need first to connect to a dataverse.");
+        }
         return format;
     }
 
@@ -157,8 +150,13 @@
         return type.getDatatype();
     }
 
-    public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException, MetadataException {
-        NodeGroup ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+    public List<String> findNodeGroupNodeNames(String nodeGroupName) throws AlgebricksException {
+        NodeGroup ng;
+        try {
+            ng = metadataManager.getNodegroup(mdTxnCtx, nodeGroupName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
         if (ng == null) {
             throw new AlgebricksException("No node group with this name " + nodeGroupName);
         }
@@ -173,103 +171,44 @@
         return stores;
     }
 
-    public AqlCompiledDatasetDecl findDataset(String datasetName) {
+    public Dataset findDataset(String datasetName) throws AlgebricksException {
         try {
-            Dataset datasetRecord = this.metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
-            if (datasetRecord == null) {
-                return null;
-            }
-
-            IAqlCompiledDatasetDetails acdd = null;
-            switch (datasetRecord.getType()) {
-                case FEED:
-                case INTERNAL: {
-                    String typeName = datasetRecord.getDatatypeName();
-                    InternalDatasetDetails id = (InternalDatasetDetails) datasetRecord.getDatasetDetails();
-                    ARecordType recType = (ARecordType) findType(typeName);
-                    List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningEvalFactories = computePartitioningEvaluatorFactories(
-                            id.getPartitioningKey(), recType);
-                    List<Index> indexRecord = this.metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName,
-                            datasetName);
-                    AqlCompiledIndexDecl primaryIndex = null;
-                    List<AqlCompiledIndexDecl> secondaryIndexes = new ArrayList<AqlCompiledIndexDecl>();
-                    for (int i = 0; i < indexRecord.size(); i++) {
-                        Index rec = indexRecord.get(i);
-                        if (rec.isPrimaryIndex()) {
-                            primaryIndex = new AqlCompiledIndexDecl(rec.getIndexName(), IndexKind.BTREE,
-                                    rec.getKeyFieldNames());
-                        } else {
-                            secondaryIndexes.add(new AqlCompiledIndexDecl(rec.getIndexName(), getIndexKindFromType(rec
-                                    .getIndexType()), rec.getKeyFieldNames(), rec.getGramLength()));
-                        }
-                    }
-
-                    if (datasetRecord.getType() == DatasetType.INTERNAL) {
-                        acdd = new AqlCompiledInternalDatasetDetails(id.getPartitioningKey(),
-                                partitioningEvalFactories, id.getNodeGroupName(), primaryIndex, secondaryIndexes);
-                    } else {
-                        acdd = new AqlCompiledFeedDatasetDetails(id.getPartitioningKey(), partitioningEvalFactories,
-                                id.getNodeGroupName(), primaryIndex, secondaryIndexes,
-                                ((FeedDatasetDetails) id).getAdapter(), ((FeedDatasetDetails) id).getProperties(),
-                                ((FeedDatasetDetails) id).getFunctionIdentifier(), ((FeedDatasetDetails) id)
-                                        .getFeedState().toString());
-                    }
-                    break;
-                }
-
-                case EXTERNAL: {
-                    acdd = new AqlCompiledExternalDatasetDetails(
-                            ((ExternalDatasetDetails) datasetRecord.getDatasetDetails()).getAdapter(),
-                            ((ExternalDatasetDetails) datasetRecord.getDatasetDetails()).getProperties());
-                    break;
-                }
-
-            }
-            AqlCompiledDatasetDecl dataset = new AqlCompiledDatasetDecl(datasetRecord.getDatasetName(),
-                    datasetRecord.getDatatypeName(), datasetRecord.getType(), acdd);
-            return dataset;
-
-        } catch (Exception e) {
-            throw new IllegalStateException(e);
+            return metadataManager.getDataset(mdTxnCtx, dataverseName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
         }
     }
 
-    private IndexKind getIndexKindFromType(IndexType type) {
-        switch (type) {
-            case BTREE:
-                return IndexKind.BTREE;
-            case RTREE:
-                return IndexKind.RTREE;
-            case WORD_INVIX:
-                return IndexKind.WORD_INVIX;
-            case NGRAM_INVIX:
-                return IndexKind.NGRAM_INVIX;
+    public List<Index> getDatasetIndexes(String dataverseName, String datasetName) throws AlgebricksException {
+    	try {
+            return metadataManager.getDatasetIndexes(mdTxnCtx, dataverseName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
         }
-        return null;
+    }
+    
+    public Index getDatasetPrimaryIndex(String dataverseName, String datasetName) throws AlgebricksException {
+        try {
+            return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, datasetName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
     }
 
+    public Index getIndex(String dataverseName, String datasetName, String indexName) throws AlgebricksException {
+        try {
+            return metadataManager.getIndex(mdTxnCtx, dataverseName, datasetName, indexName);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+    
     public void setOutputFile(FileSplit outputFile) {
         this.outputFile = outputFile;
     }
 
-    public List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> computePartitioningEvaluatorFactories(
-            List<String> partitioningExprs, ARecordType recType) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> evalFactories = new ArrayList<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>>(
-                partitioningExprs.size());
-        for (String expr : partitioningExprs) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFact = null;
-            try {
-                evalFact = format.partitioningEvaluatorFactory(recType, expr);
-            } catch (AlgebricksException e) {
-                throw new IllegalStateException(e);
-            }
-            evalFactories.add(evalFact);
-        }
-        return evalFactories;
-    }
-
     public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
-            String datasetName, String targetIdxName) throws AlgebricksException, MetadataException {
+            String datasetName, String targetIdxName) throws AlgebricksException {
         FileSplit[] splits = splitsForInternalOrFeedDataset(datasetName, targetIdxName);
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
         String[] loc = new String[splits.length];
@@ -296,23 +235,20 @@
     }
 
     private FileSplit[] splitsForInternalOrFeedDataset(String datasetName, String targetIdxName)
-            throws AlgebricksException, MetadataException {
+            throws AlgebricksException {
 
         File relPathFile = new File(getRelativePath(datasetName + "_idx_" + targetIdxName));
-        AqlCompiledDatasetDecl adecl = findDataset(datasetName);
-        if (adecl.getDatasetType() != DatasetType.INTERNAL & adecl.getDatasetType() != DatasetType.FEED) {
+        Dataset dataset = findDataset(datasetName);
+        if (dataset.getDatasetType() != DatasetType.INTERNAL & dataset.getDatasetType() != DatasetType.FEED) {
             throw new AlgebricksException("Not an internal or feed dataset");
         }
-        AqlCompiledInternalDatasetDetails compiledDatasetDetails = (AqlCompiledInternalDatasetDetails) adecl
-                .getAqlCompiledDatasetDetails();
-        List<String> nodeGroup = findNodeGroupNodeNames(compiledDatasetDetails.getNodegroupName());
-
+        InternalDatasetDetails datasetDetails = (InternalDatasetDetails) dataset.getDatasetDetails();
+        List<String> nodeGroup = findNodeGroupNodeNames(datasetDetails.getNodeGroupName());
         if (nodeGroup == null) {
-            throw new AlgebricksException("Couldn't find node group " + compiledDatasetDetails.getNodegroupName());
+            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) {
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index 4ae38eb..b7ba1a1 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -21,12 +21,13 @@
 import java.util.Set;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
@@ -41,12 +42,11 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 
 public class AqlDataSource implements IDataSource<AqlSourceId> {
 
     private AqlSourceId id;
-    private AqlCompiledDatasetDecl adecl;
+    private Dataset dataset;
     private IAType[] schemaTypes;
     private INodeDomain domain;
     private AqlDataSourceType datasourceType;
@@ -58,14 +58,14 @@
         EXTERNAL_FEED
     }
 
-    public AqlDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType, AqlDataSourceType datasourceType)
+    public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType datasourceType)
             throws AlgebricksException {
         this.id = id;
-        this.adecl = adecl;
+        this.dataset = dataset;
         this.datasourceType = datasourceType;
         switch (datasourceType) {
             case FEED:
-                initFeedDataset(itemType, adecl);
+                initFeedDataset(itemType, dataset);
             case INTERNAL: {
                 initInternalDataset(itemType);
                 break;
@@ -81,12 +81,12 @@
         }
     }
 
-    public AqlDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType) throws AlgebricksException {
+    public AqlDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
         this.id = id;
-        this.adecl = adecl;
-        switch (adecl.getDatasetType()) {
+        this.dataset = dataset;
+        switch (dataset.getDatasetType()) {
             case FEED:
-                initFeedDataset(itemType, adecl);
+                initFeedDataset(itemType, dataset);
                 break;
             case INTERNAL:
                 initInternalDataset(itemType);
@@ -101,32 +101,32 @@
         }
     }
 
+    // TODO: Seems like initFeedDataset() could simply call this method.
     private void initInternalDataset(IAType itemType) {
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                .getPartitioningFunctions(adecl);
-        int n = partitioningFunctions.size();
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        ARecordType recordType = (ARecordType) itemType;
+        int n = partitioningKeys.size();
         schemaTypes = new IAType[n + 1];
         for (int i = 0; i < n; i++) {
-            schemaTypes[i] = partitioningFunctions.get(i).third;
+            schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
         }
         schemaTypes[n] = itemType;
-        domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(adecl));
+        domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
     }
 
-    private void initFeedDataset(IAType itemType, AqlCompiledDatasetDecl decl) {
-
-        if (decl.getAqlCompiledDatasetDetails() instanceof AqlCompiledExternalDatasetDetails) {
+    private void initFeedDataset(IAType itemType, Dataset dataset) {
+        if (dataset.getDatasetDetails() instanceof ExternalDatasetDetails) {
             initExternalDataset(itemType);
         } else {
-            List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions = DatasetUtils
-                    .getPartitioningFunctions(adecl);
-            int n = partitioningFunctions.size();
+            List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+            int n = partitioningKeys.size();
             schemaTypes = new IAType[n + 1];
+            ARecordType recordType = (ARecordType) itemType;
             for (int i = 0; i < n; i++) {
-                schemaTypes[i] = partitioningFunctions.get(i).third;
+                schemaTypes[i] = recordType.getFieldType(partitioningKeys.get(i));
             }
             schemaTypes[n] = itemType;
-            domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(adecl));
+            domain = new AsterixNodeGroupDomain(DatasetUtils.getNodegroupName(dataset));
         }
     }
 
@@ -152,8 +152,8 @@
         return id;
     }
 
-    public AqlCompiledDatasetDecl getCompiledDatasetDecl() {
-        return adecl;
+    public Dataset getDataset() {
+        return dataset;
     }
 
     @Override
@@ -170,7 +170,7 @@
 
     @Override
     public IDataSourcePropertiesProvider getPropertiesProvider() {
-        return new AqlDataSourcePartitioningProvider(adecl.getDatasetType(), domain);
+        return new AqlDataSourcePartitioningProvider(dataset.getDatasetType(), domain);
     }
 
     @Override
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
index 4a6d6e4..bac7733 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlIndex.java
@@ -15,24 +15,26 @@
 
 package edu.uci.ics.asterix.metadata.declared;
 
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 
 public class AqlIndex implements IDataSourceIndex<String, AqlSourceId> {
 
-    private final AqlCompiledIndexDecl acid;
+    private final Index index;
     private final AqlCompiledMetadataDeclarations acmd;
     private final String datasetName;
 
     // Every transactions needs to work with its own instance of an
     // AqlMetadataProvider.
-    public AqlIndex(AqlCompiledIndexDecl acid, AqlCompiledMetadataDeclarations acmd, String datasetName) {
-        this.acid = acid;
+    public AqlIndex(Index index, AqlCompiledMetadataDeclarations acmd, String datasetName) {
+        this.index = index;
         this.acmd = acmd;
         this.datasetName = datasetName;
     }
 
+    // TODO: Maybe Index can directly implement IDataSourceIndex<String, AqlSourceId>
     @Override
     public IDataSource<AqlSourceId> getDataSource() {
         try {
@@ -45,7 +47,7 @@
 
     @Override
     public String getId() {
-        return acid.getIndexName();
+        return index.getIndexName();
     }
 
 }
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 57b2e72..a3d3472 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
@@ -34,7 +34,10 @@
 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.AqlCompiledIndexDecl.IndexKind;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.ExternalDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
+import edu.uci.ics.asterix.metadata.entities.Index;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -47,12 +50,10 @@
 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.common.utils.Triple;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionRuntimeProvider;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
@@ -63,7 +64,6 @@
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IPushRuntimeFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import edu.uci.ics.hyracks.algebricks.runtime.operators.std.SinkWriterRuntimeFactory;
@@ -121,23 +121,23 @@
             List<LogicalVariable> projectVariables, boolean projectPushed, IOperatorSchema opSchema,
             IVariableTypeEnvironment typeEnv, JobGenContext context, JobSpecification jobSpec)
             throws AlgebricksException {
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(dataSource.getId().getDatasetName());
-        if (adecl == null) {
+        Dataset dataset = metadata.findDataset(dataSource.getId().getDatasetName());
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + dataSource.getId().getDatasetName());
         }
-        switch (adecl.getDatasetType()) {
+        switch (dataset.getDatasetType()) {
             case FEED:
                 if (dataSource instanceof ExternalFeedDataSource) {
-                    return buildExternalDatasetScan(jobSpec, adecl, dataSource);
+                    return buildExternalDatasetScan(jobSpec, dataset, dataSource);
                 } else {
-                    return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, adecl, dataSource,
+                    return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource,
                             context);
                 }
             case INTERNAL: {
-                return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, adecl, dataSource, context);
+                return buildInternalDatasetScan(jobSpec, scanVariables, opSchema, typeEnv, dataset, dataSource, context);
             }
             case EXTERNAL: {
-                return buildExternalDatasetScan(jobSpec, adecl, dataSource);
+                return buildExternalDatasetScan(jobSpec, dataset, dataSource);
             }
             default: {
                 throw new IllegalArgumentException();
@@ -147,40 +147,32 @@
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInternalDatasetScan(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
-            AqlCompiledDatasetDecl acedl, IDataSource<AqlSourceId> dataSource, JobGenContext context)
-            throws AlgebricksException {
+            Dataset dataset, IDataSource<AqlSourceId> dataSource, JobGenContext context) throws AlgebricksException {
         AqlSourceId asid = dataSource.getId();
+        String dataverseName = asid.getDataverseName();
         String datasetName = asid.getDatasetName();
-        String indexName = DatasetUtils.getPrimaryIndex(acedl).getIndexName();
-        return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, metadata, context, false, datasetName, acedl,
-                indexName, null, null, true, true);
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataverseName, datasetName);
+        return buildBtreeRuntime(jobSpec, outputVars, opSchema, typeEnv, metadata, context, false, datasetName,
+                dataset, primaryIndex.getIndexName(), null, null, true, true);
     }
 
     private Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDatasetScan(JobSpecification jobSpec,
-            AqlCompiledDatasetDecl acedl, IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
-        String itemTypeName = acedl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+            Dataset dataset, IDataSource<AqlSourceId> dataSource) throws AlgebricksException {
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(itemTypeName);
         if (dataSource instanceof ExternalFeedDataSource) {
-            AqlCompiledFeedDatasetDetails acfdd = (AqlCompiledFeedDatasetDetails) ((ExternalFeedDataSource) dataSource)
-                    .getCompiledDatasetDecl().getAqlCompiledDatasetDetails();
-
-            return buildFeedIntakeRuntime(jobSpec, metadata.getDataverseName(), acedl.getName(), itemType, acfdd,
-                    metadata.getFormat());
+            FeedDatasetDetails datasetDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
+            return buildFeedIntakeRuntime(jobSpec, metadata.getDataverseName(), dataset.getDatasetName(), itemType,
+                    datasetDetails, metadata.getFormat());
         } else {
             return buildExternalDataScannerRuntime(jobSpec, itemType,
-                    (AqlCompiledExternalDatasetDetails) acedl.getAqlCompiledDatasetDetails(), metadata.getFormat());
+                    (ExternalDatasetDetails) dataset.getDatasetDetails(), metadata.getFormat());
         }
     }
 
     @SuppressWarnings("rawtypes")
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildExternalDataScannerRuntime(
-            JobSpecification jobSpec, IAType itemType, AqlCompiledExternalDatasetDetails decl, IDataFormat format)
+            JobSpecification jobSpec, IAType itemType, ExternalDatasetDetails datasetDetails, IDataFormat format)
             throws AlgebricksException {
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Can only scan datasets of records.");
@@ -188,7 +180,7 @@
 
         IDatasourceReadAdapter adapter;
         try {
-            adapter = (IDatasourceReadAdapter) Class.forName(decl.getAdapter()).newInstance();
+            adapter = (IDatasourceReadAdapter) Class.forName(datasetDetails.getAdapter()).newInstance();
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to load the adapter class " + e);
@@ -201,7 +193,7 @@
         ARecordType rt = (ARecordType) itemType;
 
         try {
-            adapter.configure(decl.getProperties(), itemType);
+            adapter.configure(datasetDetails.getProperties(), itemType);
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to configure the datasource adapter " + e);
@@ -211,7 +203,7 @@
         RecordDescriptor scannerDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
         ExternalDataScanOperatorDescriptor dataScanner = new ExternalDataScanOperatorDescriptor(jobSpec,
-                decl.getAdapter(), decl.getProperties(), rt, scannerDesc);
+                datasetDetails.getAdapter(), datasetDetails.getProperties(), rt, scannerDesc);
         dataScanner.setDatasourceAdapter(adapter);
         AlgebricksPartitionConstraint constraint = adapter.getPartitionConstraint();
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(dataScanner, constraint);
@@ -243,13 +235,13 @@
     @SuppressWarnings("rawtypes")
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedIntakeRuntime(
             JobSpecification jobSpec, String dataverse, String dataset, IAType itemType,
-            AqlCompiledFeedDatasetDetails decl, IDataFormat format) throws AlgebricksException {
+            FeedDatasetDetails datasetDetails, IDataFormat format) throws AlgebricksException {
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Can only consume records.");
         }
         IDatasourceAdapter adapter;
         try {
-            adapter = (IDatasourceAdapter) Class.forName(decl.getAdapter()).newInstance();
+            adapter = (IDatasourceAdapter) Class.forName(datasetDetails.getAdapter()).newInstance();
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to load the adapter class " + e);
@@ -257,7 +249,7 @@
 
         ARecordType rt = (ARecordType) itemType;
         try {
-            adapter.configure(decl.getProperties(), itemType);
+            adapter.configure(datasetDetails.getProperties(), itemType);
         } catch (Exception e) {
             e.printStackTrace();
             throw new AlgebricksException("unable to configure the datasource adapter " + e);
@@ -267,46 +259,39 @@
         RecordDescriptor feedDesc = new RecordDescriptor(new ISerializerDeserializer[] { payloadSerde });
 
         FeedIntakeOperatorDescriptor feedIngestor = new FeedIntakeOperatorDescriptor(jobSpec, new FeedId(dataverse,
-                dataset), decl.getAdapter(), decl.getProperties(), rt, feedDesc);
+                dataset), datasetDetails.getAdapter(), datasetDetails.getProperties(), rt, feedDesc);
 
         AlgebricksPartitionConstraint constraint = adapter.getPartitionConstraint();
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedIngestor, constraint);
     }
 
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildFeedMessengerRuntime(
-            JobSpecification jobSpec, AqlCompiledMetadataDeclarations metadata, AqlCompiledFeedDatasetDetails decl,
+            JobSpecification jobSpec, AqlCompiledMetadataDeclarations metadata, FeedDatasetDetails datasetDetails,
             String dataverse, String dataset, List<IFeedMessage> feedMessages) throws AlgebricksException {
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-        try {
-            spPc = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset, dataset);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset, dataset);
         FeedMessageOperatorDescriptor feedMessenger = new FeedMessageOperatorDescriptor(jobSpec, dataverse, dataset,
                 feedMessages);
-
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(feedMessenger, spPc.second);
     }
 
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildBtreeRuntime(JobSpecification jobSpec,
             List<LogicalVariable> outputVars, IOperatorSchema opSchema, IVariableTypeEnvironment typeEnv,
             AqlCompiledMetadataDeclarations metadata, JobGenContext context, boolean retainInput, String datasetName,
-            AqlCompiledDatasetDecl datasetDecl, String indexName, int[] lowKeyFields, int[] highKeyFields,
-            boolean lowKeyInclusive, boolean highKeyInclusive) throws AlgebricksException {
+            Dataset dataset, String indexName, int[] lowKeyFields, int[] highKeyFields, boolean lowKeyInclusive,
+            boolean highKeyInclusive) throws AlgebricksException {
         boolean isSecondary = true;
-        AqlCompiledIndexDecl primaryIndexDecl = DatasetUtils.getPrimaryIndex(datasetDecl);
-        if (primaryIndexDecl != null) {
-            isSecondary = !indexName.equals(primaryIndexDecl.getIndexName());
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        if (primaryIndex != null) {
+            isSecondary = !indexName.equals(primaryIndex.getIndexName());
         }
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(datasetDecl).size();
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         RecordDescriptor outputRecDesc = JobGenHelper.mkRecordDescriptor(typeEnv, opSchema, context);
         int numKeys = numPrimaryKeys;
         int keysStartIndex = outputRecDesc.getFieldCount() - numKeys - 1;
         if (isSecondary) {
-            AqlCompiledIndexDecl secondaryIndexDecl = DatasetUtils.findSecondaryIndexByName(datasetDecl, indexName);
-            int numSecondaryKeys = secondaryIndexDecl.getFieldExprs().size();
+            Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+            int numSecondaryKeys = secondaryIndex.getKeyFieldNames().size();
             numKeys += numSecondaryKeys;
             keysStartIndex = outputRecDesc.getFieldCount() - numKeys;
         }
@@ -332,23 +317,15 @@
     @SuppressWarnings("rawtypes")
     public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildRtreeRuntime(
             AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
-            String datasetName, AqlCompiledDatasetDecl ddecl, String indexName, int[] keyFields)
-            throws AlgebricksException {
-        String itemTypeName = ddecl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+            String datasetName, Dataset dataset, String indexName, int[] keyFields) throws AlgebricksException {
+        ARecordType recType = (ARecordType) metadata.findType(dataset.getItemTypeName());
         boolean isSecondary = true;
-        AqlCompiledIndexDecl primIdxDecl = DatasetUtils.getPrimaryIndex(ddecl);
-        if (primIdxDecl != null) {
-            isSecondary = !indexName.equals(primIdxDecl.getIndexName());
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        if (primaryIndex != null) {
+            isSecondary = !indexName.equals(primaryIndex.getIndexName());
         }
 
-        int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(ddecl).size();
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
         ISerializerDeserializer[] recordFields;
         IBinaryComparatorFactory[] comparatorFactories;
         ITypeTraits[] typeTraits;
@@ -356,77 +333,60 @@
         int numSecondaryKeys = 0;
         int numNestedSecondaryKeyFields = 0;
         int i = 0;
-        if (isSecondary) {
-            AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(ddecl, indexName);
-            if (cid == null) {
-                throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
-                        + datasetName);
-            }
-            List<String> secondaryKeyFields = cid.getFieldExprs();
-            numSecondaryKeys = secondaryKeyFields.size();
-
-            if (numSecondaryKeys != 1) {
-                throw new AlgebricksException(
-                        "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.");
-            }
-
-            if (itemType.getTypeTag() != ATypeTag.RECORD) {
-                throw new AlgebricksException("Only record types can be indexed.");
-            }
-            ARecordType recType = (ARecordType) itemType;
-
-            Pair<IAType, Boolean> keyTypePair = AqlCompiledIndexDecl.getNonNullableKeyFieldType(
-                    secondaryKeyFields.get(0), recType);
-            IAType keyType = keyTypePair.first;
-            if (keyType == null) {
-                throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
-            }
-
-            int dimension = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
-            numNestedSecondaryKeyFields = dimension * 2;
-
-            int numFields = numNestedSecondaryKeyFields + numPrimaryKeys;
-            recordFields = new ISerializerDeserializer[numFields];
-            typeTraits = new ITypeTraits[numFields];
-            comparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
-            valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
-
-            IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
-            for (i = 0; i < numNestedSecondaryKeyFields; i++) {
-                ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
-                        .getSerializerDeserializer(nestedKeyType);
-                recordFields[i] = keySerde;
-                comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
-                        nestedKeyType, true);
-                typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
-                valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
-            }
-        } else {
+        if (!isSecondary) {
             throw new AlgebricksException("R-tree can only be used as a secondary index");
         }
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        if (secondaryIndex == null) {
+            throw new AlgebricksException("Code generation error: no index " + indexName + " for dataset "
+                    + datasetName);
+        }
+        List<String> secondaryKeyFields = secondaryIndex.getKeyFieldNames();
+        numSecondaryKeys = secondaryKeyFields.size();
+        if (numSecondaryKeys != 1) {
+            throw new AlgebricksException(
+                    "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> keyTypePair = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
+        IAType keyType = keyTypePair.first;
+        if (keyType == null) {
+            throw new AlgebricksException("Could not find field " + secondaryKeyFields.get(0) + " in the schema.");
+        }
+        int dimension = NonTaggedFormatUtil.getNumDimensions(keyType.getTypeTag());
+        numNestedSecondaryKeyFields = dimension * 2;
 
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(ddecl)) {
-            IAType keyType = evalFactoryAndType.third;
+        int numFields = numNestedSecondaryKeyFields + numPrimaryKeys;
+        recordFields = new ISerializerDeserializer[numFields];
+        typeTraits = new ITypeTraits[numFields];
+        comparatorFactories = new IBinaryComparatorFactory[numNestedSecondaryKeyFields];
+        valueProviderFactories = new IPrimitiveValueProviderFactory[numNestedSecondaryKeyFields];
+
+        IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(keyType.getTypeTag());
+        for (i = 0; i < numNestedSecondaryKeyFields; i++) {
             ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(keyType);
+                    .getSerializerDeserializer(nestedKeyType);
             recordFields[i] = keySerde;
-            typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
+            comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
+                    nestedKeyType, true);
+            typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
+            valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
+        }
+
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            IAType type = recType.getFieldType(partitioningKey);
+            ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
+                    .getSerializerDeserializer(type);
+            recordFields[i] = keySerde;
+            typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(type);
             ++i;
         }
-
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
         RecordDescriptor recDesc = new RecordDescriptor(recordFields);
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc;
-        try {
-            spPc = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> spPc = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         RTreeSearchOperatorDescriptor rtreeSearchOp = new RTreeSearchOperatorDescriptor(jobSpec, recDesc,
                 appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(), spPc.first, typeTraits,
                 comparatorFactories, keyFields, new RTreeDataflowHelperFactory(valueProviderFactories), false,
@@ -453,20 +413,19 @@
     public IDataSourceIndex<String, AqlSourceId> findDataSourceIndex(String indexId, AqlSourceId dataSourceId)
             throws AlgebricksException {
         AqlDataSource ads = findDataSource(dataSourceId);
-        AqlCompiledDatasetDecl adecl = ads.getCompiledDatasetDecl();
-        if (adecl.getDatasetType() == DatasetType.EXTERNAL) {
+        Dataset dataset = ads.getDataset();
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("No index for external dataset " + dataSourceId);
         }
 
-        String idxName = (String) indexId;
-        AqlCompiledIndexDecl acid = DatasetUtils.findSecondaryIndexByName(adecl, idxName);
-        AqlSourceId asid = (AqlSourceId) dataSourceId;
-        if (acid != null) {
-            return new AqlIndex(acid, metadata, asid.getDatasetName());
+        String indexName = (String) indexId;
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        if (secondaryIndex != null) {
+            return new AqlIndex(secondaryIndex, metadata, dataset.getDatasetName());
         } else {
-            AqlCompiledIndexDecl primIdx = DatasetUtils.getPrimaryIndex(adecl);
-            if (primIdx.getIndexName().equals(indexId)) {
-                return new AqlIndex(primIdx, metadata, asid.getDatasetName());
+            Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+            if (primaryIndex.getIndexName().equals(indexId)) {
+                return new AqlIndex(primaryIndex, metadata, dataset.getDatasetName());
             } else {
                 return null;
             }
@@ -478,29 +437,29 @@
         if (!aqlId.getDataverseName().equals(metadata.getDataverseName())) {
             return null;
         }
-        AqlCompiledDatasetDecl acdd = metadata.findDataset(aqlId.getDatasetName());
-        if (acdd == null) {
+        Dataset dataset = metadata.findDataset(aqlId.getDatasetName());
+        if (dataset == null) {
             throw new AlgebricksException("Datasource with id " + aqlId + " was not found.");
         }
-        String tName = acdd.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(tName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-        return new AqlDataSource(aqlId, acdd, itemType);
+        String tName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(tName);
+        return new AqlDataSource(aqlId, dataset, itemType);
     }
 
     @Override
     public boolean scannerOperatorIsLeaf(IDataSource<AqlSourceId> dataSource) {
         AqlSourceId asid = dataSource.getId();
         String datasetName = asid.getDatasetName();
-        AqlCompiledDatasetDecl adecl = metadata.findDataset(datasetName);
-        if (adecl == null) {
+        Dataset dataset = null;
+        try {
+            dataset = metadata.findDataset(datasetName);
+        } catch (AlgebricksException e) {
+            throw new IllegalStateException(e);
+        }
+        if (dataset == null) {
             throw new IllegalArgumentException("Unknown dataset " + datasetName);
         }
-        return adecl.getDatasetType() == DatasetType.EXTERNAL;
+        return dataset.getDatasetType() == DatasetType.EXTERNAL;
     }
 
     @Override
@@ -520,27 +479,23 @@
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
 
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        String indexName = primaryIndex.getIndexName();
 
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        String itemTypeName = dataset.getItemTypeName();
+        ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
 
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, context.getBinaryComparatorFactoryProvider());
+
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
                 splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation,
@@ -549,16 +504,14 @@
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
-    @Override
-    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertOrDeleteRuntime(IndexOp indexOp,
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
             throws AlgebricksException {
         String datasetName = dataSource.getId().getDatasetName();
         int numKeys = keys.size();
-        // move key fields to front
+        // Move key fields to front.
         int[] fieldPermutation = new int[numKeys + 1];
-        // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
         int i = 0;
         for (LogicalVariable varKey : keys) {
             int idx = propagatedSchema.findVariable(varKey);
@@ -567,78 +520,75 @@
         }
         fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
 
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
+        Index primaryIndex = metadata.getDatasetPrimaryIndex(dataset.getDataverseName(), dataset.getDatasetName());
+        String indexName = primaryIndex.getIndexName();
 
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
+        String itemTypeName = dataset.getItemTypeName();
+        ARecordType itemType = (ARecordType) metadata.findType(itemTypeName);
+
+        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(dataset, itemType);
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
-
+        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(dataset,
+                itemType, context.getBinaryComparatorFactoryProvider());
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, IndexOp.INSERT,
+                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
                 new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
         return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
     @Override
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getInsertRuntime(
+            IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
+            LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
+            throws AlgebricksException {
+        return getInsertOrDeleteRuntime(IndexOp.INSERT, dataSource, propagatedSchema, keys, payload, recordDesc,
+                context, spec);
+    }
+
+    @Override
     public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getDeleteRuntime(
             IDataSource<AqlSourceId> dataSource, IOperatorSchema propagatedSchema, List<LogicalVariable> keys,
             LogicalVariable payload, RecordDescriptor recordDesc, JobGenContext context, JobSpecification spec)
             throws AlgebricksException {
-        String datasetName = dataSource.getId().getDatasetName();
-        int numKeys = keys.size();
-        // move key fields to front
-        int[] fieldPermutation = new int[numKeys + 1];
-        // System.arraycopy(keys, 0, fieldPermutation, 0, numKeys);
-        int i = 0;
-        for (LogicalVariable varKey : keys) {
-            int idx = propagatedSchema.findVariable(varKey);
-            fieldPermutation[i] = idx;
-            i++;
-        }
-        fieldPermutation[numKeys] = propagatedSchema.findVariable(payload);
+        return getInsertOrDeleteRuntime(IndexOp.DELETE, dataSource, propagatedSchema, keys, payload, recordDesc,
+                context, spec);
+    }
 
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> getIndexInsertOrDeleteRuntime(IndexOp indexOp,
+            IDataSourceIndex<String, AqlSourceId> dataSourceIndex, IOperatorSchema propagatedSchema,
+            IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
+            List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
+            JobGenContext context, JobSpecification spec) throws AlgebricksException {
+        String indexName = dataSourceIndex.getId();
+        String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String indexName = DatasetUtils.getPrimaryIndex(compiledDatasetDecl).getIndexName();
-
-        ITypeTraits[] typeTraits = DatasetUtils.computeTupleTypeTraits(compiledDatasetDecl, metadata);
-
-        IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-
-        IBinaryComparatorFactory[] comparatorFactories = DatasetUtils.computeKeysBinaryComparatorFactories(
-                compiledDatasetDecl, context.getBinaryComparatorFactoryProvider());
-
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
+        switch (secondaryIndex.getIndexType()) {
+            case BTREE: {
+                return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
+                        filterFactory, recordDesc, context, spec, indexOp);
+            }
+            case RTREE: {
+                return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
+                        filterFactory, recordDesc, context, spec, indexOp);
+            }
+            default: {
+                throw new AlgebricksException("Insert and delete not implemented for index type: "
+                        + secondaryIndex.getIndexType());
+            }
         }
-
-        TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
-                spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
-                splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, IndexOp.DELETE,
-                new BTreeDataflowHelperFactory(), null, NoOpOperationCallbackProvider.INSTANCE, txnId);
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad, splitsAndConstraint.second);
     }
 
     @Override
@@ -647,21 +597,8 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec) throws AlgebricksException {
-        String indexName = dataSourceIndex.getId();
-        String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName);
-        }
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
-        if (cid.getKind() == IndexKind.BTREE) {
-            return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.INSERT);
-        } else {
-            return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.INSERT);
-        }
+        return getIndexInsertOrDeleteRuntime(IndexOp.INSERT, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+                primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
     @Override
@@ -670,21 +607,8 @@
             IOperatorSchema[] inputSchemas, IVariableTypeEnvironment typeEnv, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, ILogicalExpression filterExpr, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec) throws AlgebricksException {
-        String indexName = dataSourceIndex.getId();
-        String datasetName = dataSourceIndex.getDataSource().getId().getDatasetName();
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName);
-        }
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        AsterixTupleFilterFactory filterFactory = createTupleFilterFactory(inputSchemas, typeEnv, filterExpr, context);
-        if (cid.getKind() == IndexKind.BTREE) {
-            return getBTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.DELETE);
-        } else {
-            return getRTreeDmlRuntime(datasetName, indexName, propagatedSchema, primaryKeys, secondaryKeys,
-                    filterFactory, recordDesc, context, spec, IndexOp.DELETE);
-        }
+        return getIndexInsertOrDeleteRuntime(IndexOp.DELETE, dataSourceIndex, propagatedSchema, inputSchemas, typeEnv,
+                primaryKeys, secondaryKeys, filterExpr, recordDesc, context, spec);
     }
 
     private AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
@@ -719,39 +643,33 @@
             i++;
         }
 
-        // dataset
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        if (compiledDatasetDecl == null) {
+        Dataset dataset = metadata.findDataset(datasetName);
+        if (dataset == null) {
             throw new AlgebricksException("Unknown dataset " + datasetName);
         }
-        String itemTypeName = compiledDatasetDecl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
 
-        // index parameters
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        List<String> secondaryKeyExprs = cid.getFieldExprs();
+        // Index parameters.
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
         ITypeTraits[] typeTraits = new ITypeTraits[numKeys];
         IBinaryComparatorFactory[] comparatorFactories = new IBinaryComparatorFactory[numKeys];
         for (i = 0; i < secondaryKeys.size(); ++i) {
-            Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(secondaryKeyExprs
-                    .get(i).toString(), recType);
+            Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(i).toString(),
+                    recType);
             IAType keyType = keyPairType.first;
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
                     true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
         }
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl)) {
-            IAType keyType = evalFactoryAndType.third;
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            IAType keyType = recType.getFieldType(partitioningKey);
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
                     true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
@@ -759,13 +677,8 @@
         }
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor btreeBulkLoad = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
                 splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
@@ -777,22 +690,16 @@
             String indexName, IOperatorSchema propagatedSchema, List<LogicalVariable> primaryKeys,
             List<LogicalVariable> secondaryKeys, AsterixTupleFilterFactory filterFactory, RecordDescriptor recordDesc,
             JobGenContext context, JobSpecification spec, IndexOp indexOp) throws AlgebricksException {
-        AqlCompiledDatasetDecl compiledDatasetDecl = metadata.findDataset(datasetName);
-        String itemTypeName = compiledDatasetDecl.getItemTypeName();
-        IAType itemType;
-        try {
-            itemType = metadata.findType(itemTypeName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        Dataset dataset = metadata.findDataset(datasetName);
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadata.findType(itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
-        AqlCompiledIndexDecl cid = DatasetUtils.findSecondaryIndexByName(compiledDatasetDecl, indexName);
-        List<String> secondaryKeyExprs = cid.getFieldExprs();
-        Pair<IAType, Boolean> keyPairType = AqlCompiledIndexDecl.getNonNullableKeyFieldType(secondaryKeyExprs.get(0),
-                recType);
+        Index secondaryIndex = metadata.getIndex(dataset.getDataverseName(), dataset.getDatasetName(), indexName);
+        List<String> secondaryKeyExprs = secondaryIndex.getKeyFieldNames();
+        Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyExprs.get(0), recType);
         IAType spatialType = keyPairType.first;
         int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
         int numSecondaryKeys = dimension * 2;
@@ -816,16 +723,14 @@
         IAType nestedKeyType = NonTaggedFormatUtil.getNestedSpatialType(spatialType.getTypeTag());
         IPrimitiveValueProviderFactory[] valueProviderFactories = new IPrimitiveValueProviderFactory[numSecondaryKeys];
         for (i = 0; i < numSecondaryKeys; i++) {
-            ISerializerDeserializer keySerde = AqlSerializerDeserializerProvider.INSTANCE
-                    .getSerializerDeserializer(nestedKeyType);
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(
                     nestedKeyType, true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(nestedKeyType);
             valueProviderFactories[i] = AqlPrimitiveValueProviderFactory.INSTANCE;
         }
-        for (Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType : DatasetUtils
-                .getPartitioningFunctions(compiledDatasetDecl)) {
-            IAType keyType = evalFactoryAndType.third;
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (String partitioningKey : partitioningKeys) {
+            IAType keyType = recType.getFieldType(partitioningKey);
             comparatorFactories[i] = AqlBinaryComparatorFactoryProvider.INSTANCE.getBinaryComparatorFactory(keyType,
                     true);
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
@@ -833,13 +738,8 @@
         }
 
         IAsterixApplicationContextInfo appContext = (IAsterixApplicationContextInfo) context.getAppContext();
-        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint;
-        try {
-            splitsAndConstraint = metadata.splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName,
-                    indexName);
-        } catch (Exception e) {
-            throw new AlgebricksException(e);
-        }
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(datasetName, indexName);
         TreeIndexInsertUpdateDeleteOperatorDescriptor rtreeUpdate = new TreeIndexInsertUpdateDeleteOperatorDescriptor(
                 spec, recordDesc, appContext.getStorageManagerInterface(), appContext.getIndexRegistryProvider(),
                 splitsAndConstraint.first, typeTraits, comparatorFactories, fieldPermutation, indexOp,
@@ -860,5 +760,4 @@
     public IFunctionInfo lookupFunction(FunctionIdentifier fid) {
         return AsterixBuiltinFunctions.lookupFunction(fid);
     }
-
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
index 16e8bae..566265f 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ExternalFeedDataSource.java
@@ -1,17 +1,17 @@
 package edu.uci.ics.asterix.metadata.declared;
 
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 
 public class ExternalFeedDataSource extends AqlDataSource {
 
-    public ExternalFeedDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType) throws AlgebricksException {
-        super(id,adecl, itemType);
+    public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType) throws AlgebricksException {
+        super(id, dataset, itemType);
     }
 
-    public ExternalFeedDataSource(AqlSourceId id, AqlCompiledDatasetDecl adecl, IAType itemType, AqlDataSourceType dataSourceType) throws AlgebricksException {
-        super(id,adecl, itemType, dataSourceType);
+    public ExternalFeedDataSource(AqlSourceId id, Dataset dataset, IAType itemType, AqlDataSourceType dataSourceType)
+            throws AlgebricksException {
+        super(id, dataset, itemType, dataSourceType);
     }
-    
-  
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java
deleted file mode 100644
index 2253db9..0000000
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/IAqlCompiledDatasetDetails.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.metadata.declared;
-
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-
-public interface IAqlCompiledDatasetDetails {
-
-    public DatasetType getDatasetType();
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
index e810bc9..d383955 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Dataset.java
@@ -30,16 +30,16 @@
     private final String dataverseName;
     // Enforced to be unique within a dataverse.
     private final String datasetName;
-    // Type of values stored in this dataset.
-    private final String datatypeName;
+    // Type of items stored in this dataset.
+    private final String itemTypeName;
     private final DatasetType datasetType;
     private IDatasetDetails datasetDetails;
 
-    public Dataset(String dataverseName, String datasetName, String datatypeName, IDatasetDetails datasetDetails,
+    public Dataset(String dataverseName, String datasetName, String itemTypeName, IDatasetDetails datasetDetails,
             DatasetType datasetType) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
-        this.datatypeName = datatypeName;
+        this.itemTypeName = itemTypeName;
         this.datasetType = datasetType;
         this.datasetDetails = datasetDetails;
     }
@@ -52,11 +52,11 @@
         return datasetName;
     }
 
-    public String getDatatypeName() {
-        return datatypeName;
+    public String getItemTypeName() {
+        return itemTypeName;
     }
 
-    public DatasetType getType() {
+    public DatasetType getDatasetType() {
         return datasetType;
     }
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
index 36db14a..9761c47 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/FeedDatasetDetails.java
@@ -92,7 +92,7 @@
 
         // write field 2
         listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[2]);
-        for (String field : partitioningKey) {
+        for (String field : partitioningKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -104,7 +104,7 @@
 
         // write field 3
         listBuilder.reset((AOrderedListType) MetadataRecordTypes.FEED_DETAILS_RECORDTYPE.getFieldTypes()[3]);
-        for (String field : primaryKey) {
+        for (String field : primaryKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
index bd3f8a7..7c6d9ed 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/Index.java
@@ -19,6 +19,12 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
 
 /**
  * Metadata describing an index.
@@ -91,4 +97,30 @@
     public boolean isSecondaryIndex() {
         return !isPrimaryIndex();
     }
+
+    public static Pair<IAType, Boolean> getNonNullableKeyFieldType(String expr, ARecordType recType)
+            throws AlgebricksException {
+        IAType keyType = Index.keyFieldType(expr, recType);
+        boolean nullable = false;
+        if (keyType.getTypeTag() == ATypeTag.UNION) {
+            AUnionType unionType = (AUnionType) keyType;
+            if (unionType.isNullableType()) {
+                // The non-null type is always at index 1.
+                keyType = unionType.getUnionList().get(1);
+                nullable = true;
+            }
+        }
+        return new Pair<IAType, Boolean>(keyType, nullable);
+    }
+
+    private static IAType keyFieldType(String expr, ARecordType recType) throws AlgebricksException {
+        String[] names = recType.getFieldNames();
+        int n = names.length;
+        for (int i = 0; i < n; i++) {
+            if (names[i].equals(expr)) {
+                return recType.getFieldTypes()[i];
+            }
+        }
+        throw new AlgebricksException("Could not find field " + expr + " in the schema.");
+    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
index de51dc8..53d10e4 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/InternalDatasetDetails.java
@@ -48,29 +48,29 @@
 
     protected final FileStructure fileStructure;
     protected final PartitioningStrategy partitioningStrategy;
-    protected final List<String> partitioningKey;
-    protected final List<String> primaryKey;
-    protected final String groupName;
+    protected final List<String> partitioningKeys;
+    protected final List<String> primaryKeys;
+    protected final String nodeGroupName;
 
     public InternalDatasetDetails(FileStructure fileStructure, PartitioningStrategy partitioningStrategy,
             List<String> partitioningKey, List<String> primaryKey, String groupName) {
         this.fileStructure = fileStructure;
         this.partitioningStrategy = partitioningStrategy;
-        this.partitioningKey = partitioningKey;
-        this.primaryKey = primaryKey;
-        this.groupName = groupName;
+        this.partitioningKeys = partitioningKey;
+        this.primaryKeys = primaryKey;
+        this.nodeGroupName = groupName;
     }
 
     public String getNodeGroupName() {
-        return this.groupName;
+        return nodeGroupName;
     }
 
     public List<String> getPartitioningKey() {
-        return this.partitioningKey;
+        return partitioningKeys;
     }
 
     public List<String> getPrimaryKey() {
-        return primaryKey;
+        return primaryKeys;
     }
 
     public FileStructure getFileStructure() {
@@ -115,7 +115,7 @@
         // write field 2
         listBuilder
                 .reset((AOrderedListType) MetadataRecordTypes.INTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PARTITIONKEY_FIELD_INDEX]);
-        for (String field : partitioningKey) {
+        for (String field : partitioningKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
@@ -129,7 +129,7 @@
         // write field 3
         listBuilder
                 .reset((AOrderedListType) MetadataRecordTypes.INTERNAL_DETAILS_RECORDTYPE.getFieldTypes()[MetadataRecordTypes.INTERNAL_DETAILS_ARECORD_PRIMARYKEY_FIELD_INDEX]);
-        for (String field : primaryKey) {
+        for (String field : primaryKeys) {
             itemValue.reset();
             aString.setValue(field);
             stringSerde.serialize(aString, itemValue.getDataOutput());
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 78703f3..2eda4a8 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
@@ -147,7 +147,7 @@
                             .getValueByPos(MetadataRecordTypes.FEED_DETAILS_ARECORD_STATE_FIELD_INDEX))
                             .getStringValue();
 
-                     datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
+                    datasetDetails = new FeedDatasetDetails(fileStructure, partitioningStrategy, partitioningKey,
                             partitioningKey, groupName, adapter, properties, functionIdentifier, feedState);
                 }
                 break;
@@ -209,13 +209,13 @@
 
         // write field 2
         fieldValue.reset();
-        aString.setValue(dataset.getDatatypeName());
+        aString.setValue(dataset.getItemTypeName());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATATYPENAME_FIELD_INDEX, fieldValue);
 
         // write field 3
         fieldValue.reset();
-        aString.setValue(dataset.getType().toString());
+        aString.setValue(dataset.getDatasetType().toString());
         stringSerde.serialize(aString, fieldValue.getDataOutput());
         recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_DATASETTYPE_FIELD_INDEX, fieldValue);
 
@@ -241,7 +241,7 @@
             throws HyracksDataException {
 
         dataset.getDatasetDetails().writeDatasetDetailsRecordType(fieldValue.getDataOutput());
-        switch (dataset.getType()) {
+        switch (dataset.getDatasetType()) {
             case INTERNAL:
                 recordBuilder.addField(MetadataRecordTypes.DATASET_ARECORD_INTERNALDETAILS_FIELD_INDEX, fieldValue);
                 break;
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
index dd05b00..edb3808 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/functions/MetadataBuiltinFunctions.java
@@ -1,8 +1,8 @@
 package edu.uci.ics.asterix.metadata.functions;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
 import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
@@ -53,11 +53,11 @@
                 AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
                 String datasetName = ((AString) acv.getObject()).getStringValue();
                 AqlCompiledMetadataDeclarations metadata = ((AqlMetadataProvider) mp).getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
-                if (acdd == null) {
+                Dataset dataset = metadata.findDataset(datasetName);
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
-                String tn = acdd.getItemTypeName();
+                String tn = dataset.getItemTypeName();
                 IAType t2 = metadata.findType(tn);
                 if (t2 == null) {
                     throw new AlgebricksException("No type for dataset " + datasetName);
@@ -89,11 +89,11 @@
                 AsterixConstantValue acv = (AsterixConstantValue) ((ConstantExpression) a1).getValue();
                 String datasetName = ((AString) acv.getObject()).getStringValue();
                 AqlCompiledMetadataDeclarations metadata = ((AqlMetadataProvider) mp).getMetadataDeclarations();
-                AqlCompiledDatasetDecl acdd = metadata.findDataset(datasetName);
-                if (acdd == null) {
+                Dataset dataset = metadata.findDataset(datasetName);
+                if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
-                String tn = acdd.getItemTypeName();
+                String tn = dataset.getItemTypeName();
                 IAType t2 = metadata.findType(tn);
                 if (t2 == null) {
                     throw new AlgebricksException("No type for dataset " + datasetName);
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 a34f14b..cead5f2 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
@@ -4,115 +4,78 @@
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.formats.nontagged.AqlTypeTraitProvider;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledDatasetDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledIndexDecl;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledInternalDatasetDetails;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.entities.Dataset;
+import edu.uci.ics.asterix.metadata.entities.InternalDatasetDetails;
+import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
-import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.api.dataflow.value.ITypeTraits;
 
 public class DatasetUtils {
-    public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(
-            AqlCompiledDatasetDecl compiledDatasetDecl, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
+    public static IBinaryComparatorFactory[] computeKeysBinaryComparatorFactories(Dataset dataset,
+            ARecordType itemType, IBinaryComparatorFactoryProvider comparatorFactoryProvider)
             throws AlgebricksException {
-        if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("not implemented");
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
-        partitioningFunctions = getPartitioningFunctions(compiledDatasetDecl);
-        int numKeys = partitioningFunctions.size();
-        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[numKeys];
-        for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+        }
+        List<String> partitioningKeys = getPartitioningKeys(dataset);
+        IBinaryComparatorFactory[] bcfs = new IBinaryComparatorFactory[partitioningKeys.size()];
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             bcfs[i] = comparatorFactoryProvider.getBinaryComparatorFactory(keyType, true);
         }
         return bcfs;
     }
 
-    public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(
-            AqlCompiledDatasetDecl compiledDatasetDecl, IBinaryHashFunctionFactoryProvider hashFunProvider)
-            throws AlgebricksException {
-        if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+    public static IBinaryHashFunctionFactory[] computeKeysBinaryHashFunFactories(Dataset dataset, ARecordType itemType,
+            IBinaryHashFunctionFactoryProvider hashFunProvider) throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("not implemented");
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
-        partitioningFunctions = ((AqlCompiledInternalDatasetDetails) compiledDatasetDecl.getAqlCompiledDatasetDetails())
-                .getPartitioningFunctions();
-        int numKeys = partitioningFunctions.size();
-        IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[numKeys];
-        for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+        }
+        List<String> partitioningKeys = getPartitioningKeys(dataset);
+        IBinaryHashFunctionFactory[] bhffs = new IBinaryHashFunctionFactory[partitioningKeys.size()];
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             bhffs[i] = hashFunProvider.getBinaryHashFunctionFactory(keyType);
         }
         return bhffs;
     }
 
-    public static ITypeTraits[] computeTupleTypeTraits(AqlCompiledDatasetDecl compiledDatasetDecl,
-            AqlCompiledMetadataDeclarations datasetDecls) throws AlgebricksException {
-        if (compiledDatasetDecl.getDatasetType() == DatasetType.EXTERNAL)
+    public static ITypeTraits[] computeTupleTypeTraits(Dataset dataset, ARecordType itemType)
+            throws AlgebricksException {
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             throw new AlgebricksException("not implemented");
-        List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> partitioningFunctions;
-        partitioningFunctions = ((AqlCompiledInternalDatasetDetails) compiledDatasetDecl.getAqlCompiledDatasetDetails())
-                .getPartitioningFunctions();
-        int numKeys = partitioningFunctions.size();
+        }
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        int numKeys = partitioningKeys.size();
         ITypeTraits[] typeTraits = new ITypeTraits[numKeys + 1];
         for (int i = 0; i < numKeys; i++) {
-            Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType> evalFactoryAndType = partitioningFunctions
-                    .get(i);
-            IAType keyType = evalFactoryAndType.third;
+            IAType keyType = itemType.getFieldType(partitioningKeys.get(i));
             typeTraits[i] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(keyType);
         }
-        IAType payloadType = datasetDecls.findType(compiledDatasetDecl.getItemTypeName());
-        typeTraits[numKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(payloadType);
+        typeTraits[numKeys] = AqlTypeTraitProvider.INSTANCE.getTypeTrait(itemType);
         return typeTraits;
     }
 
-    public static List<Triple<ICopyEvaluatorFactory, ScalarFunctionCallExpression, IAType>> getPartitioningFunctions(
-            AqlCompiledDatasetDecl decl) {
-        return ((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()).getPartitioningFunctions();
+    public static List<String> getPartitioningKeys(Dataset dataset) {
+        return ((InternalDatasetDetails) dataset.getDatasetDetails()).getPartitioningKey();
     }
 
-    public static String getNodegroupName(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getNodegroupName();
+    public static String getNodegroupName(Dataset dataset) {
+        return (((InternalDatasetDetails) dataset.getDatasetDetails())).getNodeGroupName();
     }
 
-    public static AqlCompiledIndexDecl getPrimaryIndex(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getPrimaryIndex();
-
+    public static int getPositionOfPartitioningKeyField(Dataset dataset, String fieldExpr) {
+        List<String> partitioningKeys = DatasetUtils.getPartitioningKeys(dataset);
+        for (int i = 0; i < partitioningKeys.size(); i++) {
+            if (partitioningKeys.get(i).equals(fieldExpr)) {
+                return i;
+            }
+        }
+        return -1;
     }
-
-    public static AqlCompiledIndexDecl findSecondaryIndexByName(AqlCompiledDatasetDecl decl, String indexName) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())
-                .findSecondaryIndexByName(indexName));
-    }
-
-    public static List<AqlCompiledIndexDecl> findSecondaryIndexesByOneOfTheKeys(AqlCompiledDatasetDecl decl,
-            String fieldExpr) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()))
-                .findSecondaryIndexesByOneOfTheKeys(fieldExpr);
-    }
-
-    public static int getPositionOfPartitioningKeyField(AqlCompiledDatasetDecl decl, String fieldExpr) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails()))
-                .getPositionOfPartitioningKeyField(fieldExpr);
-    }
-
-    public static List<String> getPartitioningExpressions(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getPartitioningExprs();
-    }
-
-    public static List<AqlCompiledIndexDecl> getSecondaryIndexes(AqlCompiledDatasetDecl decl) {
-        return (((AqlCompiledInternalDatasetDetails) decl.getAqlCompiledDatasetDetails())).getSecondaryIndexes();
-    }
-
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
index 2953966..0e2f825 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
@@ -2,7 +2,9 @@
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import edu.uci.ics.asterix.common.annotations.IRecordTypeAnnotation;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -15,13 +17,17 @@
     private String[] fieldNames;
     private IAType[] fieldTypes;
     private boolean isOpen;
-    private transient List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
+    private transient final List<IRecordTypeAnnotation> annotations = new ArrayList<IRecordTypeAnnotation>();
+    private transient final Map<String, Integer> typeMap = new HashMap<String, Integer>();
 
     public ARecordType(String typeName, String[] fieldNames, IAType[] fieldTypes, boolean isOpen) {
         super(typeName);
         this.fieldNames = fieldNames;
         this.fieldTypes = fieldTypes;
         this.isOpen = isOpen;
+        for (int i = 0; i < fieldNames.length; i++) {
+            typeMap.put(fieldNames[i], i);
+        }
     }
 
     public final String[] getFieldNames() {
@@ -76,6 +82,10 @@
         return -1;
     }
 
+    public IAType getFieldType(String fieldName) {
+        return fieldTypes[typeMap.get(fieldName)];
+    }
+
     @Override
     public String getDisplayName() {
         return "ARecord";