merged asterix_stabilization r620:1109

git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_stabilization_temporal_functionality@1113 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalExpressionDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalExpressionDeepCopyVisitor.java
index 054ca96..95e71ff 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalExpressionDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalExpressionDeepCopyVisitor.java
@@ -1,7 +1,6 @@
 package edu.uci.ics.asterix.algebra.base;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -24,11 +23,13 @@
 
 public class LogicalExpressionDeepCopyVisitor implements ILogicalExpressionVisitor<ILogicalExpression, Void> {
     private final Counter counter;
-    private final HashMap<LogicalVariable, LogicalVariable> variableMapping;
+    private final Map<LogicalVariable, LogicalVariable> inVarMapping;
+    private final Map<LogicalVariable, LogicalVariable> outVarMapping;
 
-    public LogicalExpressionDeepCopyVisitor(Counter counter, HashMap<LogicalVariable, LogicalVariable> variableMapping) {
+    public LogicalExpressionDeepCopyVisitor(Counter counter, Map<LogicalVariable, LogicalVariable> inVarMapping, Map<LogicalVariable, LogicalVariable> variableMapping) {
         this.counter = counter;
-        this.variableMapping = variableMapping;
+        this.inVarMapping = inVarMapping;
+        this.outVarMapping = variableMapping;
     }
 
     public ILogicalExpression deepCopy(ILogicalExpression expr) throws AlgebricksException {
@@ -102,11 +103,16 @@
     public ILogicalExpression visitVariableReferenceExpression(VariableReferenceExpression expr, Void arg)
             throws AlgebricksException {
         LogicalVariable var = expr.getVariableReference();
-        LogicalVariable varCopy = variableMapping.get(var);
+        LogicalVariable givenVarReplacement = inVarMapping.get(var);
+        if (givenVarReplacement != null) {
+            outVarMapping.put(var, givenVarReplacement);
+            return new VariableReferenceExpression(givenVarReplacement);
+        }
+        LogicalVariable varCopy = outVarMapping.get(var);
         if (varCopy == null) {
             counter.inc();
             varCopy = new LogicalVariable(counter.get());
-            variableMapping.put(var, varCopy);
+            outVarMapping.put(var, varCopy);
         }
         return new VariableReferenceExpression(varCopy);
     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
index a573df3..4af1424 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/base/LogicalOperatorDeepCopyVisitor.java
@@ -1,6 +1,7 @@
 package edu.uci.ics.asterix.algebra.base;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -54,11 +55,29 @@
     private final Counter counter;
     private final LogicalExpressionDeepCopyVisitor exprDeepCopyVisitor;
 
-    private final HashMap<LogicalVariable, LogicalVariable> variableMapping = new HashMap<LogicalVariable, LogicalVariable>();
+    // Key: Variable in the original plan. Value: New variable replacing the original one in the copied plan.
+    private final Map<LogicalVariable, LogicalVariable> outVarMapping = new HashMap<LogicalVariable, LogicalVariable>();
 
+    // Key: Variable in the original plan. Value: Variable with which to replace original variable in the plan copy.
+    private final Map<LogicalVariable, LogicalVariable> inVarMapping;
+    
     public LogicalOperatorDeepCopyVisitor(Counter counter) {
         this.counter = counter;
-        exprDeepCopyVisitor = new LogicalExpressionDeepCopyVisitor(counter, variableMapping);
+        this.inVarMapping = Collections.emptyMap();
+        exprDeepCopyVisitor = new LogicalExpressionDeepCopyVisitor(counter, inVarMapping, outVarMapping);
+    }
+
+    /**
+     * @param counter
+     *            Starting variable counter.
+     * @param inVarMapping
+     *            Variable mapping keyed by variables in the original plan.
+     *            Those variables are replaced by their corresponding value in the map in the copied plan.
+     */
+    public LogicalOperatorDeepCopyVisitor(Counter counter, Map<LogicalVariable, LogicalVariable> inVarMapping) {
+        this.counter = counter;
+        this.inVarMapping = inVarMapping;
+        exprDeepCopyVisitor = new LogicalExpressionDeepCopyVisitor(counter, inVarMapping, outVarMapping);
     }
 
     private void copyAnnotations(ILogicalOperator src, ILogicalOperator dest) {
@@ -132,11 +151,16 @@
         if (var == null) {
             return null;
         }
-        LogicalVariable varCopy = variableMapping.get(var);
+        LogicalVariable givenVarReplacement = inVarMapping.get(var);
+        if (givenVarReplacement != null) {
+            outVarMapping.put(var, givenVarReplacement);
+            return givenVarReplacement;
+        }
+        LogicalVariable varCopy = outVarMapping.get(var);
         if (varCopy == null) {
             counter.inc();
             varCopy = new LogicalVariable(counter.get());
-            variableMapping.put(var, varCopy);
+            outVarMapping.put(var, varCopy);
         }
         return varCopy;
     }
@@ -162,16 +186,16 @@
     }
 
     public void reset() {
-        variableMapping.clear();
+        outVarMapping.clear();
     }
 
     public void updatePrimaryKeys(IOptimizationContext context) {
-        for (Map.Entry<LogicalVariable, LogicalVariable> entry : variableMapping.entrySet()) {
+        for (Map.Entry<LogicalVariable, LogicalVariable> entry : outVarMapping.entrySet()) {
             List<LogicalVariable> primaryKey = context.findPrimaryKey(entry.getKey());
             if (primaryKey != null) {
                 List<LogicalVariable> head = new ArrayList<LogicalVariable>();
                 for (LogicalVariable variable : primaryKey) {
-                    head.add(variableMapping.get(variable));
+                    head.add(outVarMapping.get(variable));
                 }
                 List<LogicalVariable> tail = new ArrayList<LogicalVariable>(1);
                 tail.add(entry.getValue());
@@ -181,7 +205,7 @@
     }
 
     public LogicalVariable varCopy(LogicalVariable var) throws AlgebricksException {
-        return variableMapping.get(var);
+        return outVarMapping.get(var);
     }
 
     @Override
@@ -191,6 +215,7 @@
                 exprDeepCopyVisitor.deepCopyExpressionReferenceList(op.getExpressions()));
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -200,6 +225,7 @@
                 exprDeepCopyVisitor.deepCopyExpressionReferenceList(op.getExpressions()));
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -210,6 +236,7 @@
                 op.getDataSource());
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -220,7 +247,9 @@
 
     @Override
     public ILogicalOperator visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, ILogicalOperator arg) {
-        return new EmptyTupleSourceOperator();
+        EmptyTupleSourceOperator opCopy = new EmptyTupleSourceOperator();
+        opCopy.setExecutionMode(op.getExecutionMode());
+        return opCopy;
     }
 
     @Override
@@ -240,6 +269,7 @@
         deepCopyPlanList(op.getNestedPlans(), nestedPlansCopy, opCopy);
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -250,6 +280,7 @@
                 .getCondition()), deepCopyOperatorReference(op.getInputs().get(0), null), deepCopyOperatorReference(op
                 .getInputs().get(1), null));
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -274,6 +305,7 @@
         NestedTupleSourceOperator opCopy = new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(arg));
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -282,6 +314,7 @@
         OrderOperator opCopy = new OrderOperator(deepCopyOrderExpressionReferencePairList(op.getOrderExpressions()));
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -295,6 +328,7 @@
         ProjectOperator opCopy = new ProjectOperator(deepCopyVariableList(op.getVariables()));
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -319,6 +353,7 @@
         SelectOperator opCopy = new SelectOperator(exprDeepCopyVisitor.deepCopyExpressionReference(op.getCondition()));
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -330,6 +365,7 @@
         deepCopyPlanList(op.getNestedPlans(), nestedPlansCopy, opCopy);
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -350,6 +386,7 @@
                 deepCopyVariable(op.getPositionalVariable()), op.getPositionalVariableType());
         deepCopyInputs(op, opCopy, arg);
         copyAnnotations(op, opCopy);
+        opCopy.setExecutionMode(op.getExecutionMode());
         return opCopy;
     }
 
@@ -383,4 +420,8 @@
             throws AlgebricksException {
         throw new UnsupportedOperationException();
     }
+
+    public Map<LogicalVariable, LogicalVariable> getVariableMapping() {
+        return outVarMapping;
+    }
 }
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 dad8d0a..09a4c6b 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,7 +3,6 @@
 import java.util.ArrayList;
 import java.util.List;
 
-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;
@@ -11,6 +10,7 @@
 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.utils.ListSet;
 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;
@@ -23,8 +23,18 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+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.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 
@@ -33,8 +43,16 @@
  */
 public class BTreeSearchPOperator extends IndexSearchPOperator {
 
-    public BTreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
+    private final List<LogicalVariable> lowKeyVarList;
+    private final List<LogicalVariable> highKeyVarList;
+    private boolean isPrimaryIndex;
+
+    public BTreeSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast,
+            boolean isPrimaryIndex, List<LogicalVariable> lowKeyVarList, List<LogicalVariable> highKeyVarList) {
         super(idx, requiresBroadcast);
+        this.isPrimaryIndex = isPrimaryIndex;
+        this.lowKeyVarList = lowKeyVarList;
+        this.highKeyVarList = highKeyVarList;
     }
 
     @Override
@@ -61,22 +79,48 @@
         int[] lowKeyIndexes = getKeyIndexes(jobGenParams.getLowKeyVarList(), inputSchemas);
         int[] highKeyIndexes = getKeyIndexes(jobGenParams.getHighKeyVarList(), inputSchemas);
         AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
-        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
+        Dataset dataset = metadataProvider.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
         IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(op);
         List<LogicalVariable> outputVars = unnestMap.getVariables();
         if (jobGenParams.getRetainInput()) {
             outputVars = new ArrayList<LogicalVariable>();
             VariableUtilities.getLiveVariables(unnestMap, outputVars);
         }
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = AqlMetadataProvider.buildBtreeRuntime(
-                builder.getJobSpec(), outputVars, opSchema, typeEnv, metadata, context, jobGenParams.getRetainInput(),
-                jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes,
-                jobGenParams.isLowKeyInclusive(), jobGenParams.isHighKeyInclusive());
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> btreeSearch = metadataProvider.buildBtreeRuntime(
+                builder.getJobSpec(), outputVars, opSchema, typeEnv,  context, jobGenParams.getRetainInput(),
+                dataset, jobGenParams.getIndexName(), lowKeyIndexes, highKeyIndexes, jobGenParams.isLowKeyInclusive(),
+                jobGenParams.isHighKeyInclusive());
         builder.contributeHyracksOperator(unnestMap, btreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(btreeSearch.first, btreeSearch.second);
 
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
+    
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent) {
+        if (requiresBroadcast) {
+            if (isPrimaryIndex) {
+                // For primary indexes, we require re-partitioning on the primary key, and not a broadcast.
+                // Also, add a local sorting property to enforce a sort before the primary-index operator.
+                StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+                ListSet<LogicalVariable> searchKeyVars = new ListSet<LogicalVariable>();
+                searchKeyVars.addAll(lowKeyVarList);
+                searchKeyVars.addAll(highKeyVarList);
+                List<ILocalStructuralProperty> propsLocal = new ArrayList<ILocalStructuralProperty>();
+                for (LogicalVariable orderVar : searchKeyVars) {
+                    propsLocal.add(new LocalOrderProperty(new OrderColumn(orderVar, OrderKind.ASC)));
+                }
+                pv[0] = new StructuralPropertiesVector(new UnorderedPartitionedProperty(searchKeyVars, null),
+                        propsLocal);
+                return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+            } else {
+                StructuralPropertiesVector[] pv = new StructuralPropertiesVector[1];
+                pv[0] = new StructuralPropertiesVector(new BroadcastPartitioningProperty(null), null);
+                return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+            }
+        } else {
+            return super.getRequiredPropertiesForChildren(op, reqdByParent);
+        }
+    }
 }
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/IndexSearchPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/IndexSearchPOperator.java
index 65225ca..f6c926b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/IndexSearchPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/IndexSearchPOperator.java
@@ -23,8 +23,8 @@
  */
 public abstract class IndexSearchPOperator extends AbstractScanPOperator {
 
-    private final IDataSourceIndex<String, AqlSourceId> idx;
-    private final boolean requiresBroadcast;
+    protected final IDataSourceIndex<String, AqlSourceId> idx;
+    protected final boolean requiresBroadcast;
 
     public IndexSearchPOperator(IDataSourceIndex<String, AqlSourceId> idx, boolean requiresBroadcast) {
         this.idx = idx;
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 ae2559b..510aa4c 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
@@ -4,7 +4,8 @@
 import java.util.List;
 
 import edu.uci.ics.asterix.common.dataflow.IAsterixApplicationContextInfo;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.MetadataException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
@@ -50,7 +51,8 @@
 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) {
@@ -79,13 +81,18 @@
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
 
         AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
-        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
+        Dataset dataset;
+        try {
+            dataset = MetadataManager.INSTANCE.getDataset(metadataProvider.getMetadataTxnContext(),
+                    jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
+        }
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
 
         // Build runtime.
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch = buildInvertedIndexRuntime(metadata,
-                context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> invIndexSearch = buildInvertedIndexRuntime(
+                metadataProvider, context, builder.getJobSpec(), unnestMapOp, opSchema, jobGenParams.getRetainInput(),
                 jobGenParams.getDatasetName(), dataset, jobGenParams.getIndexName(), jobGenParams.getSearchKeyType(),
                 keyIndexes, jobGenParams.getSearchModifierType(), jobGenParams.getSimilarityThreshold());
         // Contribute operator in hyracks job.
@@ -95,82 +102,93 @@
         builder.contributeGraphEdge(srcExchange, 0, unnestMapOp, 0);
     }
 
-    public static Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
-            AqlCompiledMetadataDeclarations metadata, JobGenContext context, JobSpecification jobSpec,
+    public Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> buildInvertedIndexRuntime(
+            AqlMetadataProvider metadataProvider, 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.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 = secondaryIndex.getKeyFieldNames();
-        int numSecondaryKeys = secondaryKeyFields.size();
-        if (numSecondaryKeys != 1) {
-            throw new AlgebricksException(
-                    "Cannot use "
-                            + numSecondaryKeys
-                            + " fields as a key for an inverted index. There can be only one field as a key for the inverted index index.");
-        }
-        if (itemType.getTypeTag() != ATypeTag.RECORD) {
-            throw new AlgebricksException("Only record types can be indexed.");
-        }
-        ARecordType recordType = (ARecordType) itemType;
-        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.");
-        }
+        try {
+            IAObject simThresh = ((AsterixConstantValue) similarityThreshold).getObject();
+            IAType itemType = MetadataManager.INSTANCE.getDatatype(metadataProvider.getMetadataTxnContext(),
+                    dataset.getDataverseName(), dataset.getItemTypeName()).getDatatype();
+            int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+            Index secondaryIndex = MetadataManager.INSTANCE.getIndex(metadataProvider.getMetadataTxnContext(),
+                    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();
+            int numSecondaryKeys = secondaryKeyFields.size();
+            if (numSecondaryKeys != 1) {
+                throw new AlgebricksException(
+                        "Cannot use "
+                                + numSecondaryKeys
+                                + " fields as a key for an inverted index. There can be only one field as a key for the inverted index index.");
+            }
+            if (itemType.getTypeTag() != ATypeTag.RECORD) {
+                throw new AlgebricksException("Only record types can be indexed.");
+            }
+            ARecordType recordType = (ARecordType) itemType;
+            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.");
+            }
 
-        // TODO: For now we assume the type of the generated tokens is the same as the indexed field.
-        // We need a better way of expressing this because tokens may be hashed, or an inverted-index may index a list type, etc.
-        ITypeTraits[] tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
-        IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
-        for (int i = 0; i < numSecondaryKeys; i++) {
-            tokenComparatorFactories[i] = InvertedIndexAccessMethod.getTokenBinaryComparatorFactory(secondaryKeyType);
-            tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
+            // TODO: For now we assume the type of the generated tokens is the
+            // same
+            // as the indexed field.
+            // We need a better way of expressing this because tokens may be
+            // hashed,
+            // or an inverted-index may index a list type, etc.
+            ITypeTraits[] tokenTypeTraits = new ITypeTraits[numSecondaryKeys];
+            IBinaryComparatorFactory[] tokenComparatorFactories = new IBinaryComparatorFactory[numSecondaryKeys];
+            for (int i = 0; i < numSecondaryKeys; i++) {
+                tokenComparatorFactories[i] = InvertedIndexAccessMethod
+                        .getTokenBinaryComparatorFactory(secondaryKeyType);
+                tokenTypeTraits[i] = InvertedIndexAccessMethod.getTokenTypeTrait(secondaryKeyType);
+            }
+
+            IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
+            List<LogicalVariable> outputVars = unnestMap.getVariables();
+            if (retainInput) {
+                outputVars = new ArrayList<LogicalVariable>();
+                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 = metadataProvider
+                    .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(),
+                            datasetName, indexName);
+            Pair<IFileSplitProvider, IFileSplitProvider> fileSplitProviders = metadataProvider
+                    .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, 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);
+        } catch (MetadataException e) {
+            throw new AlgebricksException(e);
         }
-
-        IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
-        List<LogicalVariable> outputVars = unnestMap.getVariables();
-        if (retainInput) {
-            outputVars = new ArrayList<LogicalVariable>();
-            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 = 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, 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 27a477c..ab66457 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,6 +1,8 @@
 package edu.uci.ics.asterix.algebra.operators.physical;
 
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import java.util.ArrayList;
+import java.util.List;
+
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.metadata.entities.Dataset;
@@ -13,17 +15,21 @@
 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.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import edu.uci.ics.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import edu.uci.ics.hyracks.api.dataflow.IOperatorDescriptor;
 
 /**
- * Contributes the runtime operator for an unnest-map representing a RTree search.
+ * Contributes the runtime operator for an unnest-map representing a RTree
+ * search.
  */
 public class RTreeSearchPOperator extends IndexSearchPOperator {
 
@@ -50,23 +56,24 @@
         if (!funcIdent.equals(AsterixBuiltinFunctions.INDEX_SEARCH)) {
             return;
         }
-
         RTreeJobGenParams jobGenParams = new RTreeJobGenParams();
         jobGenParams.readFromFuncArgs(unnestFuncExpr.getArguments());
-
         int[] keyIndexes = getKeyIndexes(jobGenParams.getKeyVarList(), inputSchemas);
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-        Dataset dataset = metadata.findDataset(jobGenParams.getDatasetName());
-        if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + jobGenParams.getDatasetName());
+        Dataset dataset = mp.findDataset(jobGenParams.getDataverseName(), jobGenParams.getDatasetName());
+        IVariableTypeEnvironment typeEnv = context.getTypeEnvironment(unnestMap);
+        List<LogicalVariable> outputVars = unnestMap.getVariables();
+        if (jobGenParams.getRetainInput()) {
+            outputVars = new ArrayList<LogicalVariable>();
+            VariableUtilities.getLiveVariables(unnestMap, outputVars);
         }
-        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = AqlMetadataProvider.buildRtreeRuntime(
-                metadata, context, builder.getJobSpec(), jobGenParams.getDatasetName(), dataset,
+        Pair<IOperatorDescriptor, AlgebricksPartitionConstraint> rtreeSearch = mp.buildRtreeRuntime(
+                builder.getJobSpec(), outputVars, opSchema, typeEnv, context, jobGenParams.getRetainInput(), dataset,
                 jobGenParams.getIndexName(), keyIndexes);
         builder.contributeHyracksOperator(unnestMap, rtreeSearch.first);
         builder.contributeAlgebricksPartitionConstraint(rtreeSearch.first, rtreeSearch.second);
         ILogicalOperator srcExchange = unnestMap.getInputs().get(0).getValue();
         builder.contributeGraphEdge(srcExchange, 0, unnestMap, 0);
     }
+
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
index 9717d6a..dd8791f 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/jobgen/AqlLogicalExpressionJobGen.java
@@ -4,28 +4,21 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
+import edu.uci.ics.asterix.common.functions.FunctionDescriptorTag;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
-import edu.uci.ics.asterix.runtime.base.IAggregateFunctionDynamicDescriptor;
-import edu.uci.ics.asterix.runtime.base.IRunningAggregateFunctionDynamicDescriptor;
-import edu.uci.ics.asterix.runtime.base.IScalarFunctionDynamicDescriptor;
-import edu.uci.ics.asterix.runtime.base.ISerializableAggregateFunctionDynamicDescriptor;
-import edu.uci.ics.asterix.runtime.base.IUnnestingFunctionDynamicDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.comparisons.ComparisonEvalFactory;
 import edu.uci.ics.asterix.runtime.formats.FormatUtils;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 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.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ILogicalExpressionJobGen;
 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.expressions.StatefulFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
@@ -53,79 +46,25 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations compiledDecls = mp.getMetadataDeclarations();
-        try {
-            fd = compiledDecls.getFormat().resolveFunction(expr, env);
-        } catch (AlgebricksException e) {
-            throw new AlgebricksException(e);
-        }
+        IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
         switch (fd.getFunctionDescriptorTag()) {
-            case SCALAR: {
-                throw new AlgebricksException(
-                        "Trying to create an aggregate from a scalar evaluator function descriptor. (fi="
-                                + expr.getFunctionIdentifier() + ")");
-            }
-            case AGGREGATE: {
-                IAggregateFunctionDynamicDescriptor afdd = (IAggregateFunctionDynamicDescriptor) fd;
-                return afdd.createAggregateFunctionFactory(args);
-            }
-            case SERIALAGGREGATE: {
-                // temporal hack
+            case SERIALAGGREGATE:
                 return null;
-            }
-            case RUNNINGAGGREGATE: {
-                throw new AlgebricksException(
-                        "Trying to create an aggregate from a running aggregate function descriptor.");
-            }
-            case UNNEST: {
-                throw new AlgebricksException(
-                        "Trying to create an aggregate from an unnesting aggregate function descriptor.");
-            }
-
-            default: {
-                throw new IllegalStateException(fd.getFunctionDescriptorTag().toString());
-            }
+            case AGGREGATE:
+                return fd.createAggregateFunctionFactory(args);
+            default:
+                throw new IllegalStateException("Invalid function descriptor " + fd.getFunctionDescriptorTag()
+                        + " expected " + FunctionDescriptorTag.SERIALAGGREGATE + " or "
+                        + FunctionDescriptorTag.AGGREGATE);
         }
-
     }
 
     @Override
-    public ICopyRunningAggregateFunctionFactory createRunningAggregateFunctionFactory(StatefulFunctionCallExpression expr,
-            IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
-            throws AlgebricksException {
+    public ICopyRunningAggregateFunctionFactory createRunningAggregateFunctionFactory(
+            StatefulFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
+            JobGenContext context) throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations compiledDecls = mp.getMetadataDeclarations();
-        try {
-            fd = compiledDecls.getFormat().resolveFunction(expr, env);
-        } catch (AlgebricksException e) {
-            throw new AlgebricksException(e);
-        }
-        switch (fd.getFunctionDescriptorTag()) {
-            case SCALAR: {
-                throw new AlgebricksException(
-                        "Trying to create a running aggregate from a scalar evaluator function descriptor. (fi="
-                                + expr.getFunctionIdentifier() + ")");
-            }
-            case AGGREGATE: {
-                throw new AlgebricksException(
-                        "Trying to create a running aggregate from an aggregate function descriptor.");
-            }
-            case UNNEST: {
-                throw new AlgebricksException(
-                        "Trying to create a running aggregate from an unnesting function descriptor.");
-            }
-            case RUNNINGAGGREGATE: {
-                IRunningAggregateFunctionDynamicDescriptor rafdd = (IRunningAggregateFunctionDynamicDescriptor) fd;
-                return rafdd.createRunningAggregateFunctionFactory(args);
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
+        return getFunctionDescriptor(expr, env, context).createRunningAggregateFunctionFactory(args);
     }
 
     @Override
@@ -133,52 +72,33 @@
             IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas, JobGenContext context)
             throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations compiledDecls = mp.getMetadataDeclarations();
-        try {
-            fd = compiledDecls.getFormat().resolveFunction(expr, env);
-        } catch (AlgebricksException e) {
-            throw new AlgebricksException(e);
-        }
-        switch (fd.getFunctionDescriptorTag()) {
-            case UNNEST: {
-                IUnnestingFunctionDynamicDescriptor ufdd = (IUnnestingFunctionDynamicDescriptor) fd;
-                return ufdd.createUnnestingFunctionFactory(args);
-            }
-            default: {
-                throw new AlgebricksException("Trying to create an unnesting function descriptor from a "
-                        + fd.getFunctionDescriptorTag() + ". (fid=" + expr.getFunctionIdentifier() + ")");
-            }
-        }
+        return getFunctionDescriptor(expr, env, context).createUnnestingFunctionFactory(args);
     }
 
     @Override
     public ICopyEvaluatorFactory createEvaluatorFactory(ILogicalExpression expr, IVariableTypeEnvironment env,
             IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
+        ICopyEvaluatorFactory copyEvaluatorFactory = null;
         switch (expr.getExpressionTag()) {
             case VARIABLE: {
                 VariableReferenceExpression v = (VariableReferenceExpression) expr;
-                return createVariableEvaluatorFactory(v, inputSchemas, context);
+                copyEvaluatorFactory = createVariableEvaluatorFactory(v, inputSchemas, context);
+                return copyEvaluatorFactory;
             }
             case CONSTANT: {
                 ConstantExpression c = (ConstantExpression) expr;
-                return createConstantEvaluatorFactory(c, inputSchemas, context);
+                copyEvaluatorFactory = createConstantEvaluatorFactory(c, inputSchemas, context);
+                return copyEvaluatorFactory;
             }
             case FUNCTION_CALL: {
-                AbstractFunctionCallExpression fun = (AbstractFunctionCallExpression) expr;
-                if (fun.getKind() == FunctionKind.SCALAR) {
-                    ScalarFunctionCallExpression scalar = (ScalarFunctionCallExpression) fun;
-                    return createScalarFunctionEvaluatorFactory(scalar, env, inputSchemas, context);
-                } else {
-                    throw new AlgebricksException("Cannot create evaluator for function " + fun + " of kind "
-                            + fun.getKind());
-                }
+                copyEvaluatorFactory = createScalarFunctionEvaluatorFactory((AbstractFunctionCallExpression) expr, env,
+                        inputSchemas, context);
+                return copyEvaluatorFactory;
             }
-            default: {
+            default:
                 throw new IllegalStateException();
-            }
         }
+
     }
 
     private ICopyEvaluatorFactory createVariableEvaluatorFactory(VariableReferenceExpression expr,
@@ -203,33 +123,17 @@
             return new ComparisonEvalFactory(args[0], args[1], ck);
         }
 
-        IFunctionDescriptor fd;
-
+        IFunctionDescriptor fd = null;
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        IDataFormat format = mp == null ? FormatUtils.getDefaultFormat() : mp.getMetadataDeclarations().getFormat();
-        try {
-            fd = format.resolveFunction(expr, env);
-        } catch (AlgebricksException e) {
-            throw new AlgebricksException(e);
-        }
-
-        switch (fd.getFunctionDescriptorTag()) {
-            case SCALAR: {
-                IScalarFunctionDynamicDescriptor sfdd = (IScalarFunctionDynamicDescriptor) fd;
-                return sfdd.createEvaluatorFactory(args);
-            }
-            default: {
-                throw new AlgebricksException("Trying to create a scalar function descriptor from a "
-                        + fd.getFunctionDescriptorTag() + ". (fid=" + fi + ")");
-            }
-        }
-
+        IDataFormat format = FormatUtils.getDefaultFormat();
+        fd = format.resolveFunction(expr, env);
+        return fd.createEvaluatorFactory(args);
     }
 
-    private ICopyEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr, IOperatorSchema[] inputSchemas,
-            JobGenContext context) throws AlgebricksException {
+    private ICopyEvaluatorFactory createConstantEvaluatorFactory(ConstantExpression expr,
+            IOperatorSchema[] inputSchemas, JobGenContext context) throws AlgebricksException {
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        IDataFormat format = mp == null ? FormatUtils.getDefaultFormat() : mp.getMetadataDeclarations().getFormat();
+        IDataFormat format = FormatUtils.getDefaultFormat();
         return format.getConstantEvalFactory(expr.getValue());
     }
 
@@ -250,27 +154,15 @@
             AggregateFunctionCallExpression expr, IVariableTypeEnvironment env, IOperatorSchema[] inputSchemas,
             JobGenContext context) throws AlgebricksException {
         ICopyEvaluatorFactory[] args = codegenArguments(expr, env, inputSchemas, context);
-        IFunctionDescriptor fd;
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations compiledDecls = mp.getMetadataDeclarations();
-        try {
-            fd = compiledDecls.getFormat().resolveFunction(expr, env);
-        } catch (AlgebricksException e) {
-            throw new AlgebricksException(e);
-        }
+        IFunctionDescriptor fd = getFunctionDescriptor(expr, env, context);
+
         switch (fd.getFunctionDescriptorTag()) {
-            case SCALAR: {
-                throw new AlgebricksException(
-                        "Trying to create an aggregate from a scalar evaluator function descriptor. (fi="
-                                + expr.getFunctionIdentifier() + ")");
-            }
             case AGGREGATE: {
                 if (AsterixBuiltinFunctions.isAggregateFunctionSerializable(fd.getIdentifier())) {
                     AggregateFunctionCallExpression serialAggExpr = AsterixBuiltinFunctions
                             .makeSerializableAggregateFunctionExpression(fd.getIdentifier(), expr.getArguments());
-                    ISerializableAggregateFunctionDynamicDescriptor afdd = (ISerializableAggregateFunctionDynamicDescriptor) compiledDecls
-                            .getFormat().resolveFunction(serialAggExpr, env);
-                    return afdd.createAggregateFunctionFactory(args);
+                    IFunctionDescriptor afdd = getFunctionDescriptor(serialAggExpr, env, context);
+                    return afdd.createSerializableAggregateFunctionFactory(args);
                 } else {
                     throw new AlgebricksException(
                             "Trying to create a serializable aggregate from a non-serializable aggregate function descriptor. (fi="
@@ -278,22 +170,22 @@
                 }
             }
             case SERIALAGGREGATE: {
-                ISerializableAggregateFunctionDynamicDescriptor afdd = (ISerializableAggregateFunctionDynamicDescriptor) fd;
-                return afdd.createAggregateFunctionFactory(args);
-            }
-            case RUNNINGAGGREGATE: {
-                throw new AlgebricksException(
-                        "Trying to create an aggregate from a running aggregate function descriptor.");
-            }
-            case UNNEST: {
-                throw new AlgebricksException(
-                        "Trying to create an aggregate from an unnesting aggregate function descriptor.");
+                return fd.createSerializableAggregateFunctionFactory(args);
             }
 
-            default: {
-                throw new IllegalStateException();
-            }
+            default:
+                throw new IllegalStateException("Invalid function descriptor " + fd.getFunctionDescriptorTag()
+                        + " expected " + FunctionDescriptorTag.SERIALAGGREGATE + " or "
+                        + FunctionDescriptorTag.AGGREGATE);
         }
     }
 
+    private IFunctionDescriptor getFunctionDescriptor(AbstractFunctionCallExpression expr,
+            IVariableTypeEnvironment env, JobGenContext context) throws AlgebricksException {
+        IFunctionDescriptor fd;
+        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
+        fd = FormatUtils.getDefaultFormat().resolveFunction(expr, env);
+        return fd;
+    }
+
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java
index 94a00cb..644996a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/AnalysisUtil.java
@@ -8,6 +8,7 @@
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 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;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
@@ -106,9 +107,9 @@
         return false;
     }
 
-    public static String getDatasetName(DataSourceScanOperator op) throws AlgebricksException {
+    public static Pair<String, String> getDatasetInfo(DataSourceScanOperator op) throws AlgebricksException {
         AqlSourceId srcId = (AqlSourceId) op.getDataSource().getId();
-        return srcId.getDatasetName();
+        return new Pair<String, String>(srcId.getDataverseName(), srcId.getDatasetName());
     }
 
     private static List<FunctionIdentifier> fieldAccessFunctions = new ArrayList<FunctionIdentifier>();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java
index 0c274b9..baf16c2 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/FuzzyUtils.java
@@ -5,7 +5,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.om.base.AFloat;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.IAObject;
@@ -35,13 +35,14 @@
                 return AsterixBuiltinFunctions.COUNTHASHED_WORD_TOKENS;
             case UNORDEREDLIST:
             case ORDEREDLIST:
+            case ANY:
                 return null;
             default:
                 throw new NotImplementedException("No tokenizer for type " + inputTag);
         }
     }
 
-    public static IAObject getSimThreshold(AqlCompiledMetadataDeclarations metadata, String simFuncName) {
+    public static IAObject getSimThreshold(AqlMetadataProvider metadata, String simFuncName) {
         String simThresholValue = metadata.getPropertyValue(SIM_THRESHOLD_PROP_NAME);
         IAObject ret = null;
         if (simFuncName.equals(JACCARD_FUNCTION_NAME)) {
@@ -83,7 +84,7 @@
         return null;
     }
 
-    public static float getSimThreshold(AqlCompiledMetadataDeclarations metadata) {
+    public static float getSimThreshold(AqlMetadataProvider metadata) {
         float simThreshold = JACCARD_DEFAULT_SIM_THRESHOLD;
         String simThresholValue = metadata.getPropertyValue(SIM_THRESHOLD_PROP_NAME);
         if (simThresholValue != null) {
@@ -93,7 +94,7 @@
     }
 
     // TODO: The default function depend on the input types. 
-    public static String getSimFunction(AqlCompiledMetadataDeclarations metadata) {
+    public static String getSimFunction(AqlMetadataProvider metadata) {
         String simFunction = metadata.getPropertyValue(SIM_FUNCTION_PROP_NAME);
         if (simFunction == null) {
             simFunction = DEFAULT_SIM_FUNCTION;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index 76039ab..f63cc0c 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
@@ -28,18 +28,22 @@
 import edu.uci.ics.asterix.optimizer.rules.FuzzyEqRule;
 import edu.uci.ics.asterix.optimizer.rules.FuzzyJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.IfElseToSwitchCaseFunctionRule;
-import edu.uci.ics.asterix.optimizer.rules.InlineAssignIntoAggregateRule;
+import edu.uci.ics.asterix.optimizer.rules.InlineUnnestFunctionRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceDynamicTypeCastRule;
+import edu.uci.ics.asterix.optimizer.rules.IntroduceEnforcedListTypeRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceSecondaryIndexInsertDeleteRule;
 import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
 import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
 import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
 import edu.uci.ics.asterix.optimizer.rules.PullPositionalVariableFromUnnestRule;
+import edu.uci.ics.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import edu.uci.ics.asterix.optimizer.rules.PushAggregateIntoGroupbyRule;
 import edu.uci.ics.asterix.optimizer.rules.PushFieldAccessRule;
 import edu.uci.ics.asterix.optimizer.rules.PushGroupByThroughProduct;
 import edu.uci.ics.asterix.optimizer.rules.PushProperJoinThroughProduct;
+import edu.uci.ics.asterix.optimizer.rules.PushSimilarityFunctionsBelowJoin;
 import edu.uci.ics.asterix.optimizer.rules.RemoveRedundantListifyRule;
+import edu.uci.ics.asterix.optimizer.rules.RemoveUnusedOneToOneEquiJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.SetAsterixPhysicalOperatorsRule;
 import edu.uci.ics.asterix.optimizer.rules.SetClosedRecordConstructorsRule;
 import edu.uci.ics.asterix.optimizer.rules.SimilarityCheckRule;
@@ -55,19 +59,24 @@
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ConsolidateSelectsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EliminateSubplanRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonExpressionsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractCommonOperatorsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ExtractGbyExpressionsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.FactorRedundantGroupAndDecorVarsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.InferTypesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineAssignIntoAggregateRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineSingleReferenceVariablesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.InsertOuterJoinRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.InsertProjectBeforeUnionRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroHashPartitionMergeExchange;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroJoinInsideSubplanRule;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceCombinerRule;
-import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByForStandaloneAggregRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceAggregateCombinerRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByCombinerRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceGroupByForSubplanRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.IntroduceProjectsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.IsolateHyracksOperatorsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PullSelectOutOfEqJoin;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignBelowUnionAllRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushAssignDownThroughProductRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushDieUpRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushLimitDownRule;
@@ -77,6 +86,8 @@
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSelectIntoJoinRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushSubplanWithAggregateDownThroughProductRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.ReinferAllTypesRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantGroupByDecorVars;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveRedundantVariablesRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.RemoveUnusedAssignAndAggregateRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetAlgebricksPhysicalOperatorsRule;
 import edu.uci.ics.hyracks.algebricks.rewriter.rules.SetExecutionModeRule;
@@ -87,6 +98,7 @@
 
     public final static List<IAlgebraicRewriteRule> buildTypeInferenceRuleCollection() {
         List<IAlgebraicRewriteRule> typeInfer = new LinkedList<IAlgebraicRewriteRule>();
+        typeInfer.add(new InlineUnnestFunctionRule());
         typeInfer.add(new InferTypesRule());
         return typeInfer;
     }
@@ -94,17 +106,19 @@
     public final static List<IAlgebraicRewriteRule> buildNormalizationRuleCollection() {
         List<IAlgebraicRewriteRule> normalization = new LinkedList<IAlgebraicRewriteRule>();
         normalization.add(new EliminateSubplanRule());
-        normalization.add(new IntroduceGroupByForStandaloneAggregRule());
+        normalization.add(new PushAggFuncIntoStandaloneAggregateRule());
         normalization.add(new BreakSelectIntoConjunctsRule());
         normalization.add(new ExtractGbyExpressionsRule());
         normalization.add(new ExtractDistinctByExpressionsRule());
         normalization.add(new ExtractOrderExpressionsRule());
+        normalization.add(new ExtractCommonExpressionsRule());
 
         // IntroduceStaticTypeCastRule should go before
         // IntroduceDynamicTypeCastRule to
         // avoid unnecessary dynamic casting
         normalization.add(new IntroduceStaticTypeCastRule());
         normalization.add(new IntroduceDynamicTypeCastRule());
+        normalization.add(new IntroduceEnforcedListTypeRule());
         normalization.add(new ConstantFoldingRule());
         normalization.add(new UnnestToDataScanRule());
         normalization.add(new IfElseToSwitchCaseFunctionRule());
@@ -129,8 +143,11 @@
         condPushDownAndJoinInference.add(new IntroduceGroupByForSubplanRule());
         condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
         condPushDownAndJoinInference.add(new InsertOuterJoinRule());
+
+        condPushDownAndJoinInference.add(new RemoveRedundantVariablesRule());
         condPushDownAndJoinInference.add(new AsterixInlineVariablesRule());
         condPushDownAndJoinInference.add(new RemoveUnusedAssignAndAggregateRule());
+
         condPushDownAndJoinInference.add(new FactorRedundantGroupAndDecorVarsRule());
         condPushDownAndJoinInference.add(new PushAggregateIntoGroupbyRule());
         condPushDownAndJoinInference.add(new EliminateSubplanRule());
@@ -147,8 +164,8 @@
         fieldLoads.add(new PushFieldAccessRule());
         // fieldLoads.add(new ByNameToByHandleFieldAccessRule()); -- disabled
         fieldLoads.add(new ByNameToByIndexFieldAccessRule());
+        fieldLoads.add(new RemoveRedundantVariablesRule());
         fieldLoads.add(new AsterixInlineVariablesRule());
-        // fieldLoads.add(new InlineRecordAccessRule());
         fieldLoads.add(new RemoveUnusedAssignAndAggregateRule());
         fieldLoads.add(new ConstantFoldingRule());
         fieldLoads.add(new FeedScanCollectionToUnnest());
@@ -168,20 +185,34 @@
         consolidation.add(new ConsolidateSelectsRule());
         consolidation.add(new ConsolidateAssignsRule());
         consolidation.add(new InlineAssignIntoAggregateRule());
-        consolidation.add(new IntroduceCombinerRule());
+        consolidation.add(new IntroduceGroupByCombinerRule());
+        consolidation.add(new IntroduceAggregateCombinerRule());
         consolidation.add(new CountVarToCountOneRule());
-        consolidation.add(new IntroduceSelectAccessMethodRule());
-        consolidation.add(new IntroduceJoinAccessMethodRule());
         consolidation.add(new RemoveUnusedAssignAndAggregateRule());
-        consolidation.add(new IntroduceSecondaryIndexInsertDeleteRule());
+        consolidation.add(new RemoveRedundantGroupByDecorVars());
         return consolidation;
     }
 
-    public final static List<IAlgebraicRewriteRule> buildOpPushDownRuleCollection() {
-        List<IAlgebraicRewriteRule> opPushDown = new LinkedList<IAlgebraicRewriteRule>();
-        opPushDown.add(new PushProjectDownRule());
-        opPushDown.add(new PushSelectDownRule());
-        return opPushDown;
+    public final static List<IAlgebraicRewriteRule> buildAccessMethodRuleCollection() {
+        List<IAlgebraicRewriteRule> accessMethod = new LinkedList<IAlgebraicRewriteRule>();
+        accessMethod.add(new IntroduceSelectAccessMethodRule());
+        accessMethod.add(new IntroduceJoinAccessMethodRule());
+        accessMethod.add(new IntroduceSecondaryIndexInsertDeleteRule());
+        accessMethod.add(new RemoveUnusedOneToOneEquiJoinRule());
+        accessMethod.add(new PushSimilarityFunctionsBelowJoin());
+        accessMethod.add(new RemoveUnusedAssignAndAggregateRule());
+        return accessMethod;
+    }
+
+    public final static List<IAlgebraicRewriteRule> buildPlanCleanupRuleCollection() {
+        List<IAlgebraicRewriteRule> planCleanupRules = new LinkedList<IAlgebraicRewriteRule>();
+        planCleanupRules.add(new PushAssignBelowUnionAllRule());
+        planCleanupRules.add(new ExtractCommonExpressionsRule());
+        planCleanupRules.add(new RemoveRedundantVariablesRule());
+        planCleanupRules.add(new PushProjectDownRule());
+        planCleanupRules.add(new PushSelectDownRule());
+        planCleanupRules.add(new RemoveUnusedAssignAndAggregateRule());
+        return planCleanupRules;
     }
 
     public final static List<IAlgebraicRewriteRule> buildDataExchangeRuleCollection() {
@@ -201,6 +232,11 @@
         physicalRewritesAllLevels.add(new PullPositionalVariableFromUnnestRule());
         physicalRewritesAllLevels.add(new PushProjectDownRule());
         physicalRewritesAllLevels.add(new InsertProjectBeforeUnionRule());
+        physicalRewritesAllLevels.add(new InlineSingleReferenceVariablesRule());
+        physicalRewritesAllLevels.add(new RemoveUnusedAssignAndAggregateRule());
+        physicalRewritesAllLevels.add(new ConsolidateAssignsRule());
+        // After adding projects, we may need need to set physical operators again.
+        physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         return physicalRewritesAllLevels;
     }
 
@@ -208,6 +244,9 @@
         List<IAlgebraicRewriteRule> physicalRewritesTopLevel = new LinkedList<IAlgebraicRewriteRule>();
         physicalRewritesTopLevel.add(new PushNestedOrderByUnderPreSortedGroupByRule());
         physicalRewritesTopLevel.add(new PushLimitDownRule());
+        physicalRewritesTopLevel.add(new IntroduceProjectsRule());
+        physicalRewritesTopLevel.add(new SetAlgebricksPhysicalOperatorsRule());
+        physicalRewritesTopLevel.add(new SetExecutionModeRule());
         return physicalRewritesTopLevel;
     }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixInlineVariablesRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixInlineVariablesRule.java
index 302d0d7..772a1f9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixInlineVariablesRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/AsterixInlineVariablesRule.java
@@ -14,354 +14,18 @@
  */
 package edu.uci.ics.asterix.optimizer.rules;
 
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
-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.EquivalenceClass;
-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.ILogicalPlan;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
-import edu.uci.ics.hyracks.algebricks.core.config.AlgebricksConfig;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.InlineVariablesRule;
 
-public class AsterixInlineVariablesRule implements IAlgebraicRewriteRule {
-
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
-        return false;
-    }
-
-    @Override
-    /**
-     * 
-     * Does one big DFS sweep over the plan.
-     * 
-     */
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        if (context.checkIfInDontApplySet(this, opRef.getValue())) {
-            return false;
-        }
-        VariableSubstitutionVisitor substVisitor = new VariableSubstitutionVisitor(false);
-        VariableSubstitutionVisitor substVisitorForWrites = new VariableSubstitutionVisitor(true);
-        substVisitor.setContext(context);
-        substVisitorForWrites.setContext(context);
-        Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(opRef, context, true,
-                new LinkedList<EquivalenceClass>(), substVisitor, substVisitorForWrites);
-        return bb.first;
-    }
-
-    private Pair<Boolean, Boolean> collectEqClassesAndRemoveRedundantOps(Mutable<ILogicalOperator> opRef,
-            IOptimizationContext context, boolean first, List<EquivalenceClass> equivClasses,
-            VariableSubstitutionVisitor substVisitor, VariableSubstitutionVisitor substVisitorForWrites)
-            throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (context.checkIfInDontApplySet(this, opRef.getValue())) {
-            new Pair<Boolean, Boolean>(false, false);
-        }
-        if (op.getOperatorTag() == LogicalOperatorTag.UNNEST_MAP) {
-            return new Pair<Boolean, Boolean>(false, false);
-        }
-        boolean modified = false;
-        boolean ecChange = false;
-        int cnt = 0;
-        for (Mutable<ILogicalOperator> i : op.getInputs()) {
-            boolean isOuterInputBranch = op.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN && cnt == 1;
-            List<EquivalenceClass> eqc = isOuterInputBranch ? new LinkedList<EquivalenceClass>() : equivClasses;
-
-            Pair<Boolean, Boolean> bb = (collectEqClassesAndRemoveRedundantOps(i, context, false, eqc, substVisitor,
-                    substVisitorForWrites));
-
-            if (bb.first) {
-                modified = true;
-            }
-            if (bb.second) {
-                ecChange = true;
-            }
-
-            if (isOuterInputBranch) {
-                if (AlgebricksConfig.DEBUG) {
-                    AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- Equivalence classes for inner branch of outer op.: "
-                            + eqc + "\n");
-                }
-                for (EquivalenceClass ec : eqc) {
-                    if (!ec.representativeIsConst()) {
-                        equivClasses.add(ec);
-                    }
-                }
-            }
-
-            ++cnt;
-        }
-        if (op.hasNestedPlans()) {
-            AbstractOperatorWithNestedPlans n = (AbstractOperatorWithNestedPlans) op;
-            List<EquivalenceClass> eqc = equivClasses;
-            if (n.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-                eqc = new LinkedList<EquivalenceClass>();
-            } else {
-                eqc = equivClasses;
-            }
-            for (ILogicalPlan p : n.getNestedPlans()) {
-                for (Mutable<ILogicalOperator> r : p.getRoots()) {
-                    Pair<Boolean, Boolean> bb = collectEqClassesAndRemoveRedundantOps(r, context, false, eqc,
-                            substVisitor, substVisitorForWrites);
-                    if (bb.first) {
-                        modified = true;
-                    }
-                    if (bb.second) {
-                        ecChange = true;
-                    }
-                }
-            }
-        }
-        // we assume a variable is assigned a value only once
-        if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AssignOperator a = (AssignOperator) op;
-            ILogicalExpression rhs = a.getExpressions().get(0).getValue();
-            if (rhs.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                LogicalVariable varLeft = a.getVariables().get(0);
-                VariableReferenceExpression varRef = (VariableReferenceExpression) rhs;
-                LogicalVariable varRight = varRef.getVariableReference();
-
-                EquivalenceClass ecRight = findEquivClass(varRight, equivClasses);
-                if (ecRight != null) {
-                    ecRight.addMember(varLeft);
-                } else {
-                    List<LogicalVariable> m = new LinkedList<LogicalVariable>();
-                    m.add(varRight);
-                    m.add(varLeft);
-                    EquivalenceClass ec = new EquivalenceClass(m, varRight);
-                    equivClasses.add(ec);
-                    if (AlgebricksConfig.DEBUG) {
-                        AlgebricksConfig.ALGEBRICKS_LOGGER.finest("--- New equivalence class: " + ec + "\n");
-                    }
-                }
-                ecChange = true;
-            } else if (((AbstractLogicalExpression) rhs).getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                LogicalVariable varLeft = a.getVariables().get(0);
-                List<LogicalVariable> m = new LinkedList<LogicalVariable>();
-                m.add(varLeft);
-                EquivalenceClass ec = new EquivalenceClass(m, (ConstantExpression) rhs);
-                // equivClassesForParent.add(ec);
-                equivClasses.add(ec);
-                ecChange = true;
-            }
-        } else if (op.getOperatorTag() == LogicalOperatorTag.GROUP && !(context.checkIfInDontApplySet(this, op))) {
-            GroupByOperator group = (GroupByOperator) op;
-            Pair<Boolean, Boolean> r1 = processVarExprPairs(group.getGroupByList(), equivClasses);
-            Pair<Boolean, Boolean> r2 = processVarExprPairs(group.getDecorList(), equivClasses);
-            modified = modified || r1.first || r2.first;
-            ecChange = r1.second || r2.second;
-        }
-        if (op.getOperatorTag() == LogicalOperatorTag.PROJECT) {
-            assignVarsNeededByProject((ProjectOperator) op, equivClasses, context);
-        } else {
-            boolean assignRecord = false;
-            if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-                AssignOperator assignOp = (AssignOperator) op;
-                List<Mutable<ILogicalExpression>> exprRefs = assignOp.getExpressions();
-                for (Mutable<ILogicalExpression> exprRef : exprRefs) {
-                    ILogicalExpression expr = exprRef.getValue();
-                    if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                        ScalarFunctionCallExpression funExpr = (ScalarFunctionCallExpression) expr;
-                        if (funExpr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)
-                                || funExpr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR)) {
-                            assignRecord = true;
-                            break;
-                        }
-
-                    }
-                }
-            }
-
-            if (op.getOperatorTag() == LogicalOperatorTag.WRITE
-                    || op.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE
-                    || op.getOperatorTag() == LogicalOperatorTag.INDEX_INSERT_DELETE
-                    || op.getOperatorTag() == LogicalOperatorTag.WRITE_RESULT || assignRecord) {
-                substVisitorForWrites.setEquivalenceClasses(equivClasses);
-                if (op.acceptExpressionTransform(substVisitorForWrites)) {
-                    modified = true;
-                }
-            } else {
-                substVisitor.setEquivalenceClasses(equivClasses);
-                if (op.acceptExpressionTransform(substVisitor)) {
-                    modified = true;
-                    if (op.getOperatorTag() == LogicalOperatorTag.GROUP) {
-                        GroupByOperator group = (GroupByOperator) op;
-                        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> gp : group.getGroupByList()) {
-                            if (gp.first != null
-                                    && gp.second.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                                LogicalVariable gv = ((VariableReferenceExpression) gp.second.getValue())
-                                        .getVariableReference();
-                                Iterator<Pair<LogicalVariable, Mutable<ILogicalExpression>>> iter = group.getDecorList()
-                                        .iterator();
-                                while (iter.hasNext()) {
-                                    Pair<LogicalVariable, Mutable<ILogicalExpression>> dp = iter.next();
-                                    if (dp.first == null
-                                            && dp.second.getValue().getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                                        LogicalVariable dv = ((VariableReferenceExpression) dp.second.getValue())
-                                                .getVariableReference();
-                                        if (dv == gv) {
-                                            // The decor variable is redundant,
-                                            // since it is
-                                            // propagated as a grouping
-                                            // variable.
-                                            EquivalenceClass ec1 = findEquivClass(gv, equivClasses);
-                                            if (ec1 != null) {
-                                                ec1.addMember(gp.first);
-                                                ec1.setVariableRepresentative(gp.first);
-                                            } else {
-                                                List<LogicalVariable> varList = new ArrayList<LogicalVariable>();
-                                                varList.add(gp.first);
-                                                varList.add(gv);
-                                                ec1 = new EquivalenceClass(varList, gp.first);
-                                            }
-                                            iter.remove();
-                                            break;
-                                        }
-                                    }
-                                }
-                            }
-                        }
-                    }
-                }
-            }
-        }
-        return new Pair<Boolean, Boolean>(modified, ecChange);
-    }
-
-    private Pair<Boolean, Boolean> processVarExprPairs(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> vePairs,
-            List<EquivalenceClass> equivClasses) {
-        boolean ecFromGroup = false;
-        boolean modified = false;
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> p : vePairs) {
-            ILogicalExpression expr = p.second.getValue();
-            if (p.first != null && expr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-                VariableReferenceExpression varRef = (VariableReferenceExpression) expr;
-                LogicalVariable rhsVar = varRef.getVariableReference();
-                ecFromGroup = true;
-                EquivalenceClass ecRight = findEquivClass(rhsVar, equivClasses);
-                if (ecRight != null) {
-                    LogicalVariable replacingVar = ecRight.getVariableRepresentative();
-                    if (replacingVar != null && replacingVar != rhsVar) {
-                        varRef.setVariable(replacingVar);
-                        modified = true;
-                    }
-                }
-            }
-        }
-        return new Pair<Boolean, Boolean>(modified, ecFromGroup);
-    }
-
-    // Instead of doing this, we could make Projection to be more expressive and
-    // also take constants (or even expression), at the expense of a more
-    // complex project push down.
-    private void assignVarsNeededByProject(ProjectOperator op, List<EquivalenceClass> equivClasses,
-            IOptimizationContext context) throws AlgebricksException {
-        List<LogicalVariable> prVars = op.getVariables();
-        int sz = prVars.size();
-        for (int i = 0; i < sz; i++) {
-            EquivalenceClass ec = findEquivClass(prVars.get(i), equivClasses);
-            if (ec != null) {
-                if (!ec.representativeIsConst()) {
-                    prVars.set(i, ec.getVariableRepresentative());
-                }
-            }
-        }
-    }
-
-    private final static EquivalenceClass findEquivClass(LogicalVariable var, List<EquivalenceClass> equivClasses) {
-        for (EquivalenceClass ec : equivClasses) {
-            if (ec.contains(var)) {
-                return ec;
-            }
-        }
-        return null;
-    }
-
-    private class VariableSubstitutionVisitor implements ILogicalExpressionReferenceTransform {
-        private List<EquivalenceClass> equivClasses;
-        private IOptimizationContext context;
-        private final boolean doNotSubstWithConst;
-
-        public VariableSubstitutionVisitor(boolean doNotSubstWithConst) {
-            this.doNotSubstWithConst = doNotSubstWithConst;
-        }
-
-        public void setContext(IOptimizationContext context) {
-            this.context = context;
-        }
-
-        public void setEquivalenceClasses(List<EquivalenceClass> equivClasses) {
-            this.equivClasses = equivClasses;
-        }
-
-        @Override
-        public boolean transform(Mutable<ILogicalExpression> exprRef) {
-            ILogicalExpression e = exprRef.getValue();
-            switch (((AbstractLogicalExpression) e).getExpressionTag()) {
-                case VARIABLE: {
-                    // look for a required substitution
-                    LogicalVariable var = ((VariableReferenceExpression) e).getVariableReference();
-                    if (context.shouldNotBeInlined(var)) {
-                        return false;
-                    }
-                    EquivalenceClass ec = findEquivClass(var, equivClasses);
-                    if (ec == null) {
-                        return false;
-                    }
-                    if (ec.representativeIsConst()) {
-                        if (doNotSubstWithConst) {
-                            return false;
-                        }
-                        exprRef.setValue(ec.getConstRepresentative());
-                        return true;
-                    } else {
-                        LogicalVariable r = ec.getVariableRepresentative();
-                        if (!r.equals(var)) {
-                            exprRef.setValue(new VariableReferenceExpression(r));
-                            return true;
-                        } else {
-                            return false;
-                        }
-                    }
-                }
-                case FUNCTION_CALL: {
-                    AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) e;
-                    boolean m = false;
-                    for (Mutable<ILogicalExpression> arg : fce.getArguments()) {
-                        if (transform(arg)) {
-                            m = true;
-                        }
-                    }
-                    return m;
-                }
-                default: {
-                    return false;
-                }
-            }
-        }
-
+public class AsterixInlineVariablesRule extends InlineVariablesRule {
+    
+    public AsterixInlineVariablesRule() {
+        // Do not inline field accesses because doing so would interfere with our access method rewrites.
+        // TODO: For now we must also exclude record constructor functions to avoid breaking our type casting rules
+        // IntroduceStaticTypeCastRule and IntroduceDynamicTypeCastRule. 
+        doNotInlineFuncs.add(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME);
+        doNotInlineFuncs.add(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX);
+        doNotInlineFuncs.add(AsterixBuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR);
+        doNotInlineFuncs.add(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR);
     }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
index c1d0fea..47e957b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -37,7 +37,10 @@
 import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
 import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
 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;
@@ -163,11 +166,23 @@
             if (!checkArgs(expr)) {
                 return new Pair<Boolean, ILogicalExpression>(changed, expr);
             }
-            // TODO: currently ARecord is always a closed record
+            //Current ARecord SerDe assumes a closed record, so we do not constant fold open record constructors
             if (expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)
                     || expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.CAST_RECORD)) {
                 return new Pair<Boolean, ILogicalExpression>(false, null);
             }
+            //Current List SerDe assumes a strongly typed list, so we do not constant fold the list constructors if they are not strongly typed
+            if (expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR)
+                    || expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)) {
+                AbstractCollectionType listType = (AbstractCollectionType) TypeComputerUtilities.getRequiredType(expr);
+                if (listType != null
+                        && (listType.getItemType().getTypeTag() == ATypeTag.ANY || listType.getItemType() instanceof AbstractCollectionType)) {
+                    //case1: listType == null,  could be a nested list inside a list<ANY>
+                    //case2: itemType = ANY
+                    //case3: itemType = a nested list
+                    return new Pair<Boolean, ILogicalExpression>(false, null);
+                }
+            }
             if (expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
                 ARecordType rt = (ARecordType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
                 String str = ((AString) ((AsterixConstantValue) ((ConstantExpression) expr.getArguments().get(1)
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java
index c770e9f..345f68a 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyEqRule.java
@@ -7,7 +7,6 @@
 import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -56,11 +55,10 @@
             return false;
         }
 
-        AqlCompiledMetadataDeclarations aqlMetadata = ((AqlMetadataProvider) context.getMetadataProvider())
-                .getMetadataDeclarations();
+        AqlMetadataProvider metadataProvider = ((AqlMetadataProvider) context.getMetadataProvider());
 
         IVariableTypeEnvironment env = context.getOutputTypeEnvironment(op);
-        if (expandFuzzyEq(expRef, context, env, aqlMetadata)) {
+        if (expandFuzzyEq(expRef, context, env, metadataProvider)) {
             context.computeAndSetTypeEnvironmentForOperator(op);
             return true;
         }
@@ -68,7 +66,7 @@
     }
 
     private boolean expandFuzzyEq(Mutable<ILogicalExpression> expRef, IOptimizationContext context,
-            IVariableTypeEnvironment env, AqlCompiledMetadataDeclarations aqlMetadata) throws AlgebricksException {
+            IVariableTypeEnvironment env, AqlMetadataProvider metadataProvider) throws AlgebricksException {
         ILogicalExpression exp = expRef.getValue();
 
         if (exp.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
@@ -85,7 +83,7 @@
             // We change the behavior of this rule for the specific cases of const-var, or for edit-distance functions.
             boolean useExprAsIs = false;
 
-            String simFuncName = FuzzyUtils.getSimFunction(aqlMetadata);
+            String simFuncName = FuzzyUtils.getSimFunction(metadataProvider);
             ArrayList<Mutable<ILogicalExpression>> similarityArgs = new ArrayList<Mutable<ILogicalExpression>>();
             List<ATypeTag> inputExprTypes = new ArrayList<ATypeTag>();
             for (int i = 0; i < 2; i++) {
@@ -101,9 +99,16 @@
                     inputExprTypes.add(t.getTypeTag());
                 } else if (inputExp.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                     // Hack to make sure that we will add the func call as is, without wrapping a tokenizer around.
-                    inputTypeTag = ATypeTag.UNORDEREDLIST;
+                    IAType type = (IAType) context.getExpressionTypeComputer().getType(inputExp, metadataProvider, env);
+                    inputTypeTag = type.getTypeTag();
+                    // Only auto-tokenize strings.
+                    if (inputTypeTag == ATypeTag.STRING) {
+                        // Strings will be auto-tokenized.
+                        inputTypeTag = ATypeTag.UNORDEREDLIST;
+                    } else {
+                        useExprAsIs = true;
+                    }
                     inputExprTypes.add(inputTypeTag);
-                    useExprAsIs = true;
                 } else if (inputExp.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
                     ConstantExpression inputConst = (ConstantExpression) inputExp;
                     AsterixConstantValue constVal = (AsterixConstantValue) inputConst.getValue();
@@ -140,9 +145,11 @@
             FunctionIdentifier simFunctionIdentifier = FuzzyUtils.getFunctionIdentifier(simFuncName);
             ScalarFunctionCallExpression similarityExp = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(simFunctionIdentifier), similarityArgs);
+            // Add annotations from the original fuzzy-eq function.
+            similarityExp.getAnnotations().putAll(funcExp.getAnnotations());
             ArrayList<Mutable<ILogicalExpression>> cmpArgs = new ArrayList<Mutable<ILogicalExpression>>();
             cmpArgs.add(new MutableObject<ILogicalExpression>(similarityExp));
-            IAObject simThreshold = FuzzyUtils.getSimThreshold(aqlMetadata, simFuncName);
+            IAObject simThreshold = FuzzyUtils.getSimThreshold(metadataProvider, simFuncName);
             cmpArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
                     simThreshold))));
             ScalarFunctionCallExpression cmpExpr = FuzzyUtils.getComparisonExpr(simFuncName, cmpArgs);
@@ -150,7 +157,7 @@
             return true;
         } else if (fi.equals(AlgebricksBuiltinFunctions.AND) || fi.equals(AlgebricksBuiltinFunctions.OR)) {
             for (int i = 0; i < 2; i++) {
-                if (expandFuzzyEq(funcExp.getArguments().get(i), context, env, aqlMetadata)) {
+                if (expandFuzzyEq(funcExp.getArguments().get(i), context, env, metadataProvider)) {
                     expanded = true;
                 }
             }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
index d5131a9..8936425 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/FuzzyJoinRule.java
@@ -15,7 +15,6 @@
 import edu.uci.ics.asterix.aqlplus.parser.AQLPlusParser;
 import edu.uci.ics.asterix.aqlplus.parser.ParseException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -33,6 +32,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IndexedNLJoinExpressionAnnotation;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -46,6 +46,11 @@
 
 public class FuzzyJoinRule implements IAlgebraicRewriteRule {
 
+    private static HashSet<FunctionIdentifier> simFuncs = new HashSet<FunctionIdentifier>();
+    static {
+        simFuncs.add(AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK);
+    }
+
     private static final String AQLPLUS = ""
             //
             // -- - Stage 3 - --
@@ -125,7 +130,8 @@
     private Collection<LogicalVariable> liveVars = new HashSet<LogicalVariable>();
 
     @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();
         // current opperator is join
         if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN
@@ -133,20 +139,30 @@
             return false;
         }
 
-        // find fuzzy join condition
+        // Find GET_ITEM function.
         AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
         Mutable<ILogicalExpression> expRef = joinOp.getCondition();
-        Mutable<ILogicalExpression> fuzzyExpRef = getSimilarityExpression(expRef);
-        if (fuzzyExpRef == null) {
+        Mutable<ILogicalExpression> getItemExprRef = getSimilarityExpression(expRef);
+        if (getItemExprRef == null) {
+            return false;
+        }
+        // Check if the GET_ITEM function is on one of the supported similarity-check functions.
+        AbstractFunctionCallExpression getItemFuncExpr = (AbstractFunctionCallExpression) getItemExprRef.getValue();
+        Mutable<ILogicalExpression> argRef = getItemFuncExpr.getArguments().get(0);
+        AbstractFunctionCallExpression simFuncExpr = (AbstractFunctionCallExpression) argRef.getValue();
+        if (!simFuncs.contains(simFuncExpr.getFunctionIdentifier())) {
+            return false;
+        }
+        // Skip this rule based on annotations.
+        if (simFuncExpr.getAnnotations().containsKey(IndexedNLJoinExpressionAnnotation.INSTANCE)) {
             return false;
         }
 
-        AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) fuzzyExpRef.getValue();
         List<Mutable<ILogicalOperator>> inputOps = joinOp.getInputs();
         ILogicalOperator leftInputOp = inputOps.get(0).getValue();
         ILogicalOperator rightInputOp = inputOps.get(1).getValue();
 
-        List<Mutable<ILogicalExpression>> inputExps = funcExp.getArguments();
+        List<Mutable<ILogicalExpression>> inputExps = simFuncExpr.getArguments();
 
         ILogicalExpression inputExp0 = inputExps.get(0).getValue();
         ILogicalExpression inputExp1 = inputExps.get(1).getValue();
@@ -175,6 +191,10 @@
 
         List<LogicalVariable> leftInputPKs = context.findPrimaryKey(leftInputVar);
         List<LogicalVariable> rightInputPKs = context.findPrimaryKey(rightInputVar);
+        // Bail if primary keys could not be inferred.
+        if (leftInputPKs == null || rightInputPKs == null) {
+            return false;
+        }
         // primary key has only one variable
         if (leftInputPKs.size() != 1 || rightInputPKs.size() != 1) {
             return false;
@@ -190,8 +210,7 @@
         //
         // -- - FIRE - --
         //
-        AqlCompiledMetadataDeclarations metadata = ((AqlMetadataProvider) context.getMetadataProvider())
-                .getMetadataDeclarations();
+        AqlMetadataProvider metadataProvider = ((AqlMetadataProvider) context.getMetadataProvider());
         FunctionIdentifier funcId = FuzzyUtils.getTokenizer(leftType.getTypeTag());
         String tokenizer;
         if (funcId == null) {
@@ -200,8 +219,8 @@
             tokenizer = funcId.getName();
         }
 
-        float simThreshold = FuzzyUtils.getSimThreshold(metadata);
-        String simFunction = FuzzyUtils.getSimFunction(metadata);
+        float simThreshold = FuzzyUtils.getSimThreshold(metadataProvider);
+        String simFunction = FuzzyUtils.getSimFunction(metadataProvider);
 
         // finalize AQL+ query
         String prepareJoin;
@@ -247,9 +266,8 @@
         }
         // The translator will compile metadata internally. Run this compilation
         // under the same transaction id as the "outer" compilation.
-        AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(mp.getTxnId(),
-                metadata.getMetadataTransactionContext(), counter, null);
+        AqlPlusExpressionToPlanTranslator translator = new AqlPlusExpressionToPlanTranslator(
+                metadataProvider.getJobTxnId(), metadataProvider, counter, null, null);
 
         LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
 
@@ -315,9 +333,9 @@
         ILogicalOperator outputOp = plan.getRoots().get(0).getValue();
 
         SelectOperator extraSelect = null;
-        if (fuzzyExpRef != expRef) {
+        if (getItemExprRef != expRef) {
             // more than one join condition
-            fuzzyExpRef.setValue(ConstantExpression.TRUE);
+            getItemExprRef.setValue(ConstantExpression.TRUE);
             switch (joinOp.getJoinKind()) {
                 case INNER: {
                     extraSelect = new SelectOperator(expRef);
@@ -343,19 +361,19 @@
         return true;
     }
 
-    /*
-     * look for FUZZY_EQ function call
+    /**
+     * Look for GET_ITEM function call.
      */
     private Mutable<ILogicalExpression> getSimilarityExpression(Mutable<ILogicalExpression> expRef) {
         ILogicalExpression exp = expRef.getValue();
         if (exp.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-            AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) exp;
-            if (funcExp.getFunctionIdentifier().equals(AsterixBuiltinFunctions.FUZZY_EQ)) {
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) exp;
+            if (funcExpr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.GET_ITEM)) {
                 return expRef;
-            } else if (funcExp.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)
-                    || funcExp.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.OR)) {
+            }
+            if (funcExpr.getFunctionIdentifier().equals(AlgebricksBuiltinFunctions.AND)) {
                 for (int i = 0; i < 2; i++) {
-                    Mutable<ILogicalExpression> expRefRet = getSimilarityExpression(funcExp.getArguments().get(i));
+                    Mutable<ILogicalExpression> expRefRet = getSimilarityExpression(funcExpr.getArguments().get(i));
                     if (expRefRet != null) {
                         return expRefRet;
                     }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/InlineAssignIntoAggregateRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/InlineAssignIntoAggregateRule.java
deleted file mode 100644
index 5f98a1b..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/InlineAssignIntoAggregateRule.java
+++ /dev/null
@@ -1,123 +0,0 @@
-package edu.uci.ics.asterix.optimizer.rules;
-
-import java.util.List;
-
-import org.apache.commons.lang3.mutable.Mutable;
-
-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;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.visitors.AbstractConstVarFunVisitor;
-import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
-
-
-
-public class InlineAssignIntoAggregateRule implements IAlgebraicRewriteRule {
-
-    @Override
-    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
-        return false;
-    }
-
-    @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
-        if (op.getOperatorTag() != LogicalOperatorTag.GROUP) {
-            return false;
-        }
-        boolean changed = false;
-        GroupByOperator gbyOp = (GroupByOperator) op;
-        for (ILogicalPlan p : gbyOp.getNestedPlans()) {
-            for (Mutable<ILogicalOperator> r : p.getRoots()) {
-                if (inlined(r)) {
-                    changed = true;
-                }
-            }
-        }
-        return changed;
-    }
-
-    private boolean inlined(Mutable<ILogicalOperator> r) throws AlgebricksException {
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) r.getValue();
-        if (op1.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
-            return false;
-        }
-        AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
-        if (op2.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-            return false;
-        }
-        AggregateOperator agg = (AggregateOperator) op1;
-        AssignOperator assign = (AssignOperator) op2;
-        VarExprSubstitution ves = new VarExprSubstitution(assign.getVariables(), assign.getExpressions());
-        for (Mutable<ILogicalExpression> exprRef : agg.getExpressions()) {
-            ILogicalExpression expr = exprRef.getValue();
-            Pair<Boolean, ILogicalExpression> p = expr.accept(ves, null);
-            if (p.first == true) {
-                exprRef.setValue(p.second);
-            }
-            // AbstractLogicalExpression ale = (AbstractLogicalExpression) expr;
-            // ale.accept(ves, null);
-        }
-        List<Mutable<ILogicalOperator>> op1InpList = op1.getInputs();
-        op1InpList.clear();
-        op1InpList.add(op2.getInputs().get(0));
-        return true;
-    }
-
-    private class VarExprSubstitution extends AbstractConstVarFunVisitor<Pair<Boolean, ILogicalExpression>, Void> {
-
-        private List<LogicalVariable> variables;
-        private List<Mutable<ILogicalExpression>> expressions;
-
-        public VarExprSubstitution(List<LogicalVariable> variables, List<Mutable<ILogicalExpression>> expressions) {
-            this.variables = variables;
-            this.expressions = expressions;
-        }
-
-        @Override
-        public Pair<Boolean, ILogicalExpression> visitConstantExpression(ConstantExpression expr, Void arg) {
-            return new Pair<Boolean, ILogicalExpression>(false, expr);
-        }
-
-        @Override
-        public Pair<Boolean, ILogicalExpression> visitFunctionCallExpression(AbstractFunctionCallExpression expr,
-                Void arg) throws AlgebricksException {
-            boolean changed = false;
-            for (Mutable<ILogicalExpression> eRef : expr.getArguments()) {
-                ILogicalExpression e = eRef.getValue();
-                Pair<Boolean, ILogicalExpression> p = e.accept(this, arg);
-                if (p.first) {
-                    eRef.setValue(p.second);
-                    changed = true;
-                }
-            }
-            return new Pair<Boolean, ILogicalExpression>(changed, expr);
-        }
-
-        @Override
-        public Pair<Boolean, ILogicalExpression> visitVariableReferenceExpression(VariableReferenceExpression expr,
-                Void arg) {
-            LogicalVariable v = expr.getVariableReference();
-            int idx = variables.indexOf(v);
-            if (idx < 0) {
-                return new Pair<Boolean, ILogicalExpression>(false, expr);
-            } else {
-                return new Pair<Boolean, ILogicalExpression>(true, expressions.get(idx).getValue());
-            }
-
-        }
-
-    }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/InlineUnnestFunctionRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/InlineUnnestFunctionRule.java
new file mode 100644
index 0000000..3e57ded
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/InlineUnnestFunctionRule.java
@@ -0,0 +1,174 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+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;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule is to inline unnest functions that are hold by variables.
+ * This rule is to fix issue 201.
+ */
+public class InlineUnnestFunctionRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+        if (context.checkIfInDontApplySet(this, op1))
+            return false;
+        context.addToDontApplySet(this, op1);
+        if (op1.getOperatorTag() != LogicalOperatorTag.UNNEST)
+            return false;
+        UnnestOperator unnestOperator = (UnnestOperator) op1;
+        AbstractFunctionCallExpression expr = (AbstractFunctionCallExpression) unnestOperator.getExpressionRef()
+                .getValue();
+        //we only inline for the scan-collection function
+        if (expr.getFunctionIdentifier() != AsterixBuiltinFunctions.SCAN_COLLECTION)
+            return false;
+
+        // inline all variables from an unnesting function call
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+        for (int i = 0; i < args.size(); i++) {
+            ILogicalExpression argExpr = args.get(i).getValue();
+            if (argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression varExpr = (VariableReferenceExpression) argExpr;
+                inlineVariable(varExpr.getVariableReference(), unnestOperator);
+            }
+        }
+        return true;
+    }
+
+    /**
+     * This method is to inline one variable
+     * 
+     * @param usedVar
+     *            A variable that is used by the scan-collection function in the unnest operator
+     * @param unnestOp
+     *            The unnest operator.
+     * @throws AlgebricksException
+     */
+    private void inlineVariable(LogicalVariable usedVar, UnnestOperator unnestOp) throws AlgebricksException {
+        AbstractFunctionCallExpression expr = (AbstractFunctionCallExpression) unnestOp.getExpressionRef().getValue();
+        List<Pair<AbstractFunctionCallExpression, Integer>> parentAndIndexList = new ArrayList<Pair<AbstractFunctionCallExpression, Integer>>();
+        getParentFunctionExpression(usedVar, expr, parentAndIndexList);
+        ILogicalExpression usedVarOrginExpr = findUsedVarOrigin(usedVar, unnestOp, (AbstractLogicalOperator) unnestOp
+                .getInputs().get(0).getValue());
+        if (usedVarOrginExpr != null) {
+            for (Pair<AbstractFunctionCallExpression, Integer> parentAndIndex : parentAndIndexList) {
+                //we only rewrite the top scan-collection function
+                if (parentAndIndex.first.getFunctionIdentifier() == AsterixBuiltinFunctions.SCAN_COLLECTION
+                        && parentAndIndex.first == expr) {
+                    unnestOp.getExpressionRef().setValue(usedVarOrginExpr);
+                }
+            }
+        }
+    }
+
+    private void getParentFunctionExpression(LogicalVariable usedVar, ILogicalExpression expr,
+            List<Pair<AbstractFunctionCallExpression, Integer>> parentAndIndexList) {
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+        for (int i = 0; i < args.size(); i++) {
+            ILogicalExpression argExpr = args.get(i).getValue();
+            if (argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                VariableReferenceExpression varExpr = (VariableReferenceExpression) argExpr;
+                if (varExpr.getVariableReference().equals(usedVar))
+                    parentAndIndexList.add(new Pair<AbstractFunctionCallExpression, Integer>(funcExpr, i));
+            }
+            if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                getParentFunctionExpression(usedVar, argExpr, parentAndIndexList);
+            }
+        }
+    }
+
+    private ILogicalExpression findUsedVarOrigin(LogicalVariable usedVar, AbstractLogicalOperator parentOp,
+            AbstractLogicalOperator currentOp) throws AlgebricksException {
+        ILogicalExpression ret = null;
+        if (currentOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
+            VariableUtilities.getProducedVariables(currentOp, producedVars);
+            if (producedVars.contains(usedVar)) {
+                AssignOperator assignOp = (AssignOperator) currentOp;
+                int index = assignOp.getVariables().indexOf(usedVar);
+                ILogicalExpression returnedExpr = assignOp.getExpressions().get(index).getValue();
+                if (returnedExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) returnedExpr;
+                    if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(funcExpr.getFunctionIdentifier())) {
+                        // we only inline for unnest functions
+                        removeUnecessaryAssign(parentOp, currentOp, assignOp, index);
+                        ret = returnedExpr;
+                    }
+                } else if (returnedExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+                    //recusively inline
+                    VariableReferenceExpression varExpr = (VariableReferenceExpression) returnedExpr;
+                    LogicalVariable var = varExpr.getVariableReference();
+                    ILogicalExpression finalExpr = findUsedVarOrigin(var, currentOp,
+                            (AbstractLogicalOperator) currentOp.getInputs().get(0).getValue());
+                    if (finalExpr != null) {
+                        removeUnecessaryAssign(parentOp, currentOp, assignOp, index);
+                        ret = finalExpr;
+                    }
+                }
+            }
+        } else {
+            for (Mutable<ILogicalOperator> child : currentOp.getInputs()) {
+                ILogicalExpression expr = findUsedVarOrigin(usedVar, currentOp,
+                        (AbstractLogicalOperator) child.getValue());
+                if (expr != null) {
+                    ret = expr;
+                }
+            }
+        }
+        return ret;
+    }
+
+    private void removeUnecessaryAssign(AbstractLogicalOperator parentOp, AbstractLogicalOperator currentOp,
+            AssignOperator assignOp, int index) {
+        assignOp.getVariables().remove(index);
+        assignOp.getExpressions().remove(index);
+        if (assignOp.getVariables().size() == 0) {
+            int opIndex = parentOp.getInputs().indexOf(new MutableObject<ILogicalOperator>(currentOp));
+            parentOp.getInputs().get(opIndex).setValue(assignOp.getInputs().get(0).getValue());
+        }
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
index c41d908..2dce5f6 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
@@ -48,22 +48,18 @@
  * recursive way. It enables: 1. bag-based fields in a record, 2. bidirectional
  * cast of a open field and a matched closed field, and 3. put in null fields
  * when necessary.
- * 
  * Here is an example: A record { "hobby": {{"music", "coding"}}, "id": "001",
  * "name": "Person Three"} which confirms to closed type ( id: string, name:
  * string, hobby: {{string}}? ) can be cast to an open type (id: string ), or
  * vice versa.
- * 
  * However, if the input record is a variable, then we don't know its exact
  * field layout at compile time. For example, records conforming to the same
  * type can have different field orderings and different open parts. That's why
  * we need dynamic type casting.
- * 
  * Note that as we can see in the example, the ordering of fields of a record is
  * not required. Since the open/closed part of a record has completely different
  * underlying memory/storage layout, a cast-record function will change the
  * layout as specified at runtime.
- * 
  * Implementation wise, this rule checks the target dataset type and the input
  * record type, and if the types are different, then it plugs in an assign with
  * a cast-record function, and projects away the original (uncast) field.
@@ -80,9 +76,7 @@
             throws AlgebricksException {
         /**
          * pattern match: sink insert assign
-         * 
          * resulting plan: sink-insert-project-assign
-         * 
          */
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
         if (op1.getOperatorTag() != LogicalOperatorTag.SINK)
@@ -90,6 +84,9 @@
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
         if (op2.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE)
             return false;
+        InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op2;
+        if (insertDeleteOp.getOperation() == InsertDeleteOperator.Kind.DELETE)
+            return false;
         AbstractLogicalOperator op3 = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
         if (op3.getOperatorTag() != LogicalOperatorTag.ASSIGN)
             return false;
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
new file mode 100644
index 0000000..2e8d530
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceEnforcedListTypeRule.java
@@ -0,0 +1,92 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.optimizer.rules.typecast.StaticTypeCastUtil;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractAssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This class is to enforce types for function expressions which contain list constructor function calls.
+ * The List constructor is very special because a nested list is of type List<ANY>.
+ * However, the bottom-up type inference (InferTypeRule in algebricks) did not infer that so we need this method to enforce the type.
+ * We do not want to break the generality of algebricks so this method is called in an ASTERIX rule: @ IntroduceEnforcedListTypeRule} .
+ */
+public class IntroduceEnforcedListTypeRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        if (context.checkIfInDontApplySet(this, opRef.getValue()))
+            return false;
+        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+        context.addToDontApplySet(this, opRef.getValue());
+        boolean changed = false;
+
+        /**
+         * rewrite list constructor types for list constructor functions
+         */
+        if (op1.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            AbstractAssignOperator assignOp = (AbstractAssignOperator) op1;
+            List<Mutable<ILogicalExpression>> expressions = assignOp.getExpressions();
+            IVariableTypeEnvironment env = assignOp.computeOutputTypeEnvironment(context);
+            changed = rewriteExpressions(expressions, env);
+        }
+        if (op1.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+            AbstractUnnestOperator unnestOp = (AbstractUnnestOperator) op1;
+            List<Mutable<ILogicalExpression>> expressions = Collections.singletonList(unnestOp.getExpressionRef());
+            IVariableTypeEnvironment env = unnestOp.computeOutputTypeEnvironment(context);
+            changed = rewriteExpressions(expressions, env);
+        }
+        return changed;
+    }
+
+    private boolean rewriteExpressions(List<Mutable<ILogicalExpression>> expressions, IVariableTypeEnvironment env)
+            throws AlgebricksException {
+        boolean changed = false;
+        for (Mutable<ILogicalExpression> exprRef : expressions) {
+            ILogicalExpression expr = exprRef.getValue();
+            if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) expr;
+                IAType exprType = (IAType) env.getType(argFuncExpr);
+                changed = changed || StaticTypeCastUtil.rewriteListExpr(argFuncExpr, exprType, exprType, env);
+            }
+        }
+        return changed;
+    }
+
+}
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 a9de14f..3310f4d 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
@@ -9,7 +9,6 @@
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.metadata.declared.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;
@@ -89,11 +88,11 @@
         InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
         AqlDataSource datasetSource = (AqlDataSource) insertOp.getDataSource();
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
+        String dataverseName = datasetSource.getId().getDataverseName();
         String datasetName = datasetSource.getId().getDatasetName();
-        Dataset dataset = metadata.findDataset(datasetName);
+        Dataset dataset = mp.findDataset(dataverseName, datasetName);
         if (dataset == null) {
-            throw new AlgebricksException("Unknown dataset " + datasetName);
+            throw new AlgebricksException("Unknown dataset " + datasetName + " in dataverse " + dataverseName);
         }
         if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
             return false;
@@ -103,12 +102,12 @@
         VariableUtilities.getUsedVariables(op1, projectVars);
         // Create operators for secondary index insert/delete.
         String itemTypeName = dataset.getItemTypeName();
-        IAType itemType = metadata.findType(itemTypeName);
+        IAType itemType = mp.findType(dataset.getDataverseName(), itemTypeName);
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             throw new AlgebricksException("Only record types can be indexed.");
         }
         ARecordType recType = (ARecordType) itemType;
-        List<Index> indexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         ILogicalOperator currentTop = op1;
         boolean hasSecondaryIndex = false;
         for (Index index : indexes) {
@@ -154,7 +153,7 @@
                 }
                 Mutable<ILogicalExpression> filterExpression = createFilterExpression(secondaryKeyVars,
                         context.getOutputTypeEnvironment(assign), false);
-                AqlIndex dataSourceIndex = new AqlIndex(index, metadata, datasetName);
+                AqlIndex dataSourceIndex = new AqlIndex(index, dataverseName, datasetName, mp);
                 IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
                         insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
                         insertOp.getOperation());
@@ -196,7 +195,7 @@
                 boolean forceFilter = keyPairType.second;
                 Mutable<ILogicalExpression> filterExpression = createFilterExpression(keyVarList,
                         context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
-                AqlIndex dataSourceIndex = new AqlIndex(index, metadata, datasetName);
+                AqlIndex dataSourceIndex = new AqlIndex(index, dataverseName, datasetName, mp);
                 IndexInsertDeleteOperator indexUpdate = new IndexInsertDeleteOperator(dataSourceIndex,
                         insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
                         insertOp.getOperation());
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
index 7dc35fe..3aae2dd 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
@@ -16,26 +16,14 @@
 package edu.uci.ics.asterix.optimizer.rules;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
-import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
-import edu.uci.ics.asterix.om.base.ANull;
-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.typecomputer.base.TypeComputerUtilities;
-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.AbstractCollectionType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
-import edu.uci.ics.asterix.runtime.pointables.base.DefaultOpenFieldType;
+import edu.uci.ics.asterix.optimizer.rules.typecast.StaticTypeCastUtil;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -44,9 +32,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 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.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InsertDeleteOperator;
@@ -58,22 +44,17 @@
  * recursive way. It enables: 1. bag-based fields in a record, 2. bidirectional
  * cast of a open field and a matched closed field, and 3. put in null fields
  * when necessary. It should be fired before the constant folding rule.
- * 
  * This rule is not responsible for type casting between primitive types.
- * 
  * Here is an example: A record { "hobby": {{"music", "coding"}}, "id": "001",
  * "name": "Person Three"} which confirms to closed type ( id: string, name:
  * string, hobby: {{string}}? ) can be cast to an open type (id: string ), or
  * vice versa.
- * 
  * Implementation wise: first, we match the record's type and its target dataset
  * type to see if it is "cast-able"; second, if the types are cast-able, we
  * embed the required type into the original producer expression. If the types
  * are not cast-able, we throw a compile time exception.
- * 
  * Then, at runtime (not in this rule), the corresponding record/list
  * constructors know what to do by checking the required output type.
- * 
  * TODO: right now record/list constructor of the cast result is not done in the
  * ConstantFoldingRule and has to go to the runtime, because the
  * ConstantFoldingRule uses ARecordSerializerDeserializer which seems to have
@@ -96,28 +77,31 @@
         if (context.checkIfInDontApplySet(this, opRef.getValue()))
             return false;
         context.addToDontApplySet(this, opRef.getValue());
+
         AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
+        List<LogicalVariable> producedVariables = new ArrayList<LogicalVariable>();
+        LogicalVariable oldRecordVariable;
+
         if (op1.getOperatorTag() != LogicalOperatorTag.SINK)
             return false;
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
         if (op2.getOperatorTag() != LogicalOperatorTag.INSERT_DELETE)
             return false;
-        AbstractLogicalOperator op3 = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
-        if (op3.getOperatorTag() != LogicalOperatorTag.ASSIGN)
+        InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op2;
+        if (insertDeleteOp.getOperation() == InsertDeleteOperator.Kind.DELETE)
             return false;
-
+        AbstractLogicalOperator assignOp = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
+        if (assignOp.getOperatorTag() != LogicalOperatorTag.ASSIGN)
+            return false;
         /**
          * get required record type
          */
         InsertDeleteOperator insertDeleteOperator = (InsertDeleteOperator) op2;
-        AssignOperator topAssignOperator = (AssignOperator) op3;
         AqlDataSource dataSource = (AqlDataSource) insertDeleteOperator.getDataSource();
         IAType[] schemaTypes = (IAType[]) dataSource.getSchemaTypes();
-        ARecordType requiredRecordType = (ARecordType) schemaTypes[schemaTypes.length - 1];
+        IAType requiredRecordType = schemaTypes[schemaTypes.length - 1];
 
-        /**
-         * get input record type to the insert operator
-         */
+        AssignOperator topAssignOperator = (AssignOperator) assignOp;
         List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
         VariableUtilities.getUsedVariables(topAssignOperator, usedVariables);
 
@@ -126,14 +110,12 @@
         // empty
         if (usedVariables.size() == 0)
             return false;
-        LogicalVariable oldRecordVariable = usedVariables.get(0);
+        oldRecordVariable = usedVariables.get(0);
         LogicalVariable inputRecordVar = usedVariables.get(0);
         IVariableTypeEnvironment env = topAssignOperator.computeOutputTypeEnvironment(context);
-        ARecordType inputRecordType = (ARecordType) env.getVarType(inputRecordVar);
+        IAType inputRecordType = (IAType) env.getVarType(inputRecordVar);
 
-        AbstractLogicalOperator currentOperator = topAssignOperator;
-        List<LogicalVariable> producedVariables = new ArrayList<LogicalVariable>();
-
+        AbstractLogicalOperator currentOperator = assignOp;
         /**
          * find the assign operator for the "input record" to the insert_delete
          * operator
@@ -153,13 +135,15 @@
                     List<Mutable<ILogicalExpression>> expressionRefs = originalAssign.getExpressions();
                     ILogicalExpression expr = expressionRefs.get(position).getValue();
                     if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                        ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) expr;
+                        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
                         // that expression has been rewritten, and it will not
                         // fail but just return false
-                        if (TypeComputerUtilities.getRequiredType(funcExpr) != null)
+                        if (TypeComputerUtilities.getRequiredType(funcExpr) != null) {
+                            context.computeAndSetTypeEnvironmentForOperator(assignOp);
                             return false;
-                        IVariableTypeEnvironment assignEnv = topAssignOperator.computeOutputTypeEnvironment(context);
-                        rewriteFuncExpr(funcExpr, requiredRecordType, inputRecordType, assignEnv);
+                        }
+                        IVariableTypeEnvironment assignEnv = assignOp.computeOutputTypeEnvironment(context);
+                        StaticTypeCastUtil.rewriteFuncExpr(funcExpr, requiredRecordType, inputRecordType, assignEnv);
                     }
                     context.computeAndSetTypeEnvironmentForOperator(originalAssign);
                 }
@@ -172,249 +156,4 @@
         return true;
     }
 
-    private void rewriteFuncExpr(ScalarFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
-            IVariableTypeEnvironment env) throws AlgebricksException {
-        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR) {
-            rewriteListFuncExpr(funcExpr, (AbstractCollectionType) reqType, (AbstractCollectionType) inputType, env);
-        } else if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR) {
-            rewriteListFuncExpr(funcExpr, (AbstractCollectionType) reqType, (AbstractCollectionType) inputType, env);
-        } else if (reqType.getTypeTag().equals(ATypeTag.RECORD)) {
-            rewriteRecordFuncExpr(funcExpr, (ARecordType) reqType, (ARecordType) inputType, env);
-        }
-    }
-
-    /**
-     * only called when funcExpr is record constructor
-     * 
-     * @param funcExpr
-     *            record constructor function expression
-     * @param requiredListType
-     *            required record type
-     * @param inputRecordType
-     * @param env
-     *            type environment
-     * @throws AlgebricksException
-     */
-    private void rewriteRecordFuncExpr(ScalarFunctionCallExpression funcExpr, ARecordType requiredRecordType,
-            ARecordType inputRecordType, IVariableTypeEnvironment env) throws AlgebricksException {
-        // if already rewritten, the required type is not null
-        if (TypeComputerUtilities.getRequiredType(funcExpr) != null)
-            return;
-        TypeComputerUtilities.setRequiredAndInputTypes(funcExpr, requiredRecordType, inputRecordType);
-        staticRecordTypeCast(funcExpr, requiredRecordType, inputRecordType, env);
-    }
-
-    /**
-     * only called when funcExpr is list constructor
-     * 
-     * @param funcExpr
-     *            list constructor function expression
-     * @param requiredListType
-     *            required list type
-     * @param inputListType
-     * @param env
-     *            type environment
-     * @throws AlgebricksException
-     */
-    private void rewriteListFuncExpr(ScalarFunctionCallExpression funcExpr, AbstractCollectionType requiredListType,
-            AbstractCollectionType inputListType, IVariableTypeEnvironment env) throws AlgebricksException {
-        if (TypeComputerUtilities.getRequiredType(funcExpr) != null)
-            return;
-
-        TypeComputerUtilities.setRequiredAndInputTypes(funcExpr, requiredListType, inputListType);
-        List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
-
-        IAType itemType = requiredListType.getItemType();
-        if (itemType == null || itemType.getTypeTag().equals(ATypeTag.ANY))
-            return;
-        IAType inputItemType = inputListType.getItemType();
-        for (int j = 0; j < args.size(); j++) {
-            ILogicalExpression arg = args.get(j).getValue();
-            if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
-                IAType currentItemType = (IAType) env.getType(argFunc);
-                if (inputItemType == null || inputItemType == BuiltinType.ANY) {
-                    currentItemType = (IAType) env.getType(argFunc);
-                    rewriteFuncExpr(argFunc, itemType, currentItemType, env);
-                } else {
-                    rewriteFuncExpr(argFunc, itemType, inputItemType, env);
-                }
-            }
-        }
-    }
-
-    private void staticRecordTypeCast(ScalarFunctionCallExpression func, ARecordType reqType, ARecordType inputType,
-            IVariableTypeEnvironment env) throws AlgebricksException {
-        IAType[] reqFieldTypes = reqType.getFieldTypes();
-        String[] reqFieldNames = reqType.getFieldNames();
-        IAType[] inputFieldTypes = inputType.getFieldTypes();
-        String[] inputFieldNames = inputType.getFieldNames();
-
-        int[] fieldPermutation = new int[reqFieldTypes.length];
-        boolean[] nullFields = new boolean[reqFieldTypes.length];
-        boolean[] openFields = new boolean[inputFieldTypes.length];
-
-        Arrays.fill(nullFields, false);
-        Arrays.fill(openFields, true);
-        Arrays.fill(fieldPermutation, -1);
-
-        // forward match: match from actual to required
-        boolean matched = false;
-        for (int i = 0; i < inputFieldNames.length; i++) {
-            String fieldName = inputFieldNames[i];
-            IAType fieldType = inputFieldTypes[i];
-
-            if (2 * i + 1 > func.getArguments().size())
-                throw new AlgebricksException("expression index out of bound");
-
-            // 2*i+1 is the index of field value expression
-            ILogicalExpression arg = func.getArguments().get(2 * i + 1).getValue();
-            matched = false;
-            for (int j = 0; j < reqFieldNames.length; j++) {
-                String reqFieldName = reqFieldNames[j];
-                IAType reqFieldType = reqFieldTypes[j];
-                if (fieldName.equals(reqFieldName)) {
-                    if (fieldType.equals(reqFieldType)) {
-                        fieldPermutation[j] = i;
-                        openFields[i] = false;
-                        matched = true;
-
-                        if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                            ScalarFunctionCallExpression scalarFunc = (ScalarFunctionCallExpression) arg;
-                            rewriteFuncExpr(scalarFunc, reqFieldType, fieldType, env);
-                        }
-                        break;
-                    }
-
-                    // match the optional field
-                    if (reqFieldType.getTypeTag() == ATypeTag.UNION
-                            && NonTaggedFormatUtil.isOptionalField((AUnionType) reqFieldType)) {
-                        IAType itemType = ((AUnionType) reqFieldType).getUnionList().get(
-                                NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
-                        reqFieldType = itemType;
-                        if (fieldType.equals(BuiltinType.ANULL) || fieldType.equals(itemType)) {
-                            fieldPermutation[j] = i;
-                            openFields[i] = false;
-                            matched = true;
-
-                            // rewrite record expr
-                            if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                                ScalarFunctionCallExpression scalarFunc = (ScalarFunctionCallExpression) arg;
-                                rewriteFuncExpr(scalarFunc, reqFieldType, fieldType, env);
-                            }
-                            break;
-                        }
-                    }
-
-                    // match the record field: need cast
-                    if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                        ScalarFunctionCallExpression scalarFunc = (ScalarFunctionCallExpression) arg;
-                        rewriteFuncExpr(scalarFunc, reqFieldType, fieldType, env);
-                        fieldPermutation[j] = i;
-                        openFields[i] = false;
-                        matched = true;
-                        break;
-                    }
-                }
-            }
-            // the input has extra fields
-            if (!matched && !reqType.isOpen())
-                throw new AlgebricksException("static type mismatch: including an extra closed field " + fieldName);
-        }
-
-        // backward match: match from required to actual
-        for (int i = 0; i < reqFieldNames.length; i++) {
-            String reqFieldName = reqFieldNames[i];
-            IAType reqFieldType = reqFieldTypes[i];
-            matched = false;
-            for (int j = 0; j < inputFieldNames.length; j++) {
-                String fieldName = inputFieldNames[j];
-                IAType fieldType = inputFieldTypes[j];
-                if (!fieldName.equals(reqFieldName))
-                    continue;
-                // should check open field here
-                // because number of entries in fieldPermuations is the
-                // number of required schema fields
-                // here we want to check if an input field is matched
-                // the entry index of fieldPermuatons is req field index
-                if (!openFields[j]) {
-                    matched = true;
-                    break;
-                }
-
-                // match the optional field
-                if (reqFieldType.getTypeTag() == ATypeTag.UNION
-                        && NonTaggedFormatUtil.isOptionalField((AUnionType) reqFieldType)) {
-                    IAType itemType = ((AUnionType) reqFieldType).getUnionList().get(
-                            NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
-                    if (fieldType.equals(BuiltinType.ANULL) || fieldType.equals(itemType)) {
-                        matched = true;
-                        break;
-                    }
-                }
-            }
-            if (matched)
-                continue;
-
-            if (reqFieldType.getTypeTag() == ATypeTag.UNION
-                    && NonTaggedFormatUtil.isOptionalField((AUnionType) reqFieldType)) {
-                // add a null field
-                nullFields[i] = true;
-            } else {
-                // no matched field in the input for a required closed field
-                throw new AlgebricksException("static type mismatch: miss a required closed field " + reqFieldName);
-            }
-        }
-
-        List<Mutable<ILogicalExpression>> arguments = func.getArguments();
-        List<Mutable<ILogicalExpression>> originalArguments = new ArrayList<Mutable<ILogicalExpression>>();
-        originalArguments.addAll(arguments);
-        arguments.clear();
-        // re-order the closed part and fill in null fields
-        for (int i = 0; i < fieldPermutation.length; i++) {
-            int pos = fieldPermutation[i];
-            if (pos >= 0) {
-                arguments.add(originalArguments.get(2 * pos));
-                arguments.add(originalArguments.get(2 * pos + 1));
-            }
-            if (nullFields[i]) {
-                // add a null field
-                arguments.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                        new AString(reqFieldNames[i])))));
-                arguments.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
-                        ANull.NULL))));
-            }
-        }
-
-        // add the open part
-        for (int i = 0; i < openFields.length; i++) {
-            if (openFields[i]) {
-                arguments.add(originalArguments.get(2 * i));
-                Mutable<ILogicalExpression> fExprRef = originalArguments.get(2 * i + 1);
-                ILogicalExpression argExpr = fExprRef.getValue();
-
-                // we need to handle open fields recursively by their default
-                // types
-                // for list, their item type is any
-                // for record, their
-                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    IAType reqFieldType = inputFieldTypes[i];
-                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.RECORD) {
-                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
-                    }
-                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.ORDEREDLIST) {
-                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
-                    }
-                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.UNORDEREDLIST) {
-                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
-                    }
-                    if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null) {
-                        ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
-                        rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
-                    }
-                }
-                arguments.add(fExprRef);
-            }
-        }
-    }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
new file mode 100644
index 0000000..c5a1cb0
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
@@ -0,0 +1,154 @@
+/*
+ * 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.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Pushes aggregate functions into a stand alone aggregate operator (no group by). 
+ */
+public class PushAggFuncIntoStandaloneAggregateRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        // Pattern to match: assign <-- aggregate <-- !(group-by)
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
+            return false;
+        }
+        Mutable<ILogicalOperator> opRef2 = op.getInputs().get(0);
+        AbstractLogicalOperator op2 = (AbstractLogicalOperator) opRef2.getValue();
+        if (op2.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+            return false;
+        }
+        // If there's a group by below the agg, then we want to have the agg pushed into the group by.
+        Mutable<ILogicalOperator> opRef3 = op2.getInputs().get(0);
+        AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
+        if (op3.getOperatorTag() == LogicalOperatorTag.GROUP) {
+            return false;
+        }
+        
+        AssignOperator assignOp = (AssignOperator) op;
+        AggregateOperator aggOp = (AggregateOperator) op2;
+        if (aggOp.getVariables().size() != 1) {
+            return false;
+        }
+
+        // Make sure the agg expr is a listify.
+        ILogicalExpression aggExpr = aggOp.getExpressions().get(0).getValue();
+        if (aggExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return false;
+        }
+        AbstractFunctionCallExpression origAggFuncExpr = (AbstractFunctionCallExpression) aggExpr;
+        if (origAggFuncExpr.getFunctionIdentifier() != AsterixBuiltinFunctions.LISTIFY) {
+            return false;
+        }
+        
+        LogicalVariable aggVar = aggOp.getVariables().get(0);
+        List<LogicalVariable> used = new LinkedList<LogicalVariable>();
+        VariableUtilities.getUsedVariables(assignOp, used);
+        if (!used.contains(aggVar)) {
+            return false;
+        }
+        
+        Mutable<ILogicalExpression> srcAssignExprRef = fingAggFuncExprRef(assignOp.getExpressions(), aggVar);
+        if (srcAssignExprRef == null) {
+        	return false;
+        }
+        AbstractFunctionCallExpression assignFuncExpr = (AbstractFunctionCallExpression) srcAssignExprRef.getValue();
+        FunctionIdentifier aggFuncIdent = AsterixBuiltinFunctions.getAggregateFunction(assignFuncExpr.getFunctionIdentifier());
+        
+        // Push the agg func into the agg op.                
+        AbstractFunctionCallExpression aggOpExpr = (AbstractFunctionCallExpression) aggOp.getExpressions().get(0).getValue();
+        List<Mutable<ILogicalExpression>> aggArgs = new ArrayList<Mutable<ILogicalExpression>>();
+        aggArgs.add(aggOpExpr.getArguments().get(0));
+        AggregateFunctionCallExpression aggFuncExpr = AsterixBuiltinFunctions.makeAggregateFunctionExpression(aggFuncIdent, aggArgs);
+        aggOp.getExpressions().get(0).setValue(aggFuncExpr);
+        
+        // The assign now just "renames" the variable to make sure the upstream plan still works.
+        srcAssignExprRef.setValue(new VariableReferenceExpression(aggVar));
+
+        // Create a new assign for a TRUE variable.
+        LogicalVariable trueVar = context.newVar();
+        AssignOperator trueAssignOp = new AssignOperator(trueVar, new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
+        
+        ILogicalOperator aggInput = aggOp.getInputs().get(0).getValue();
+        aggOp.getInputs().get(0).setValue(trueAssignOp);
+        trueAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(aggInput));
+        
+        // Set partitioning variable.
+        aggOp.setPartitioningVariable(trueVar);
+        
+        context.computeAndSetTypeEnvironmentForOperator(trueAssignOp);
+        context.computeAndSetTypeEnvironmentForOperator(aggOp);
+        context.computeAndSetTypeEnvironmentForOperator(assignOp);
+        
+        return true;
+    }
+    
+    private Mutable<ILogicalExpression> fingAggFuncExprRef(List<Mutable<ILogicalExpression>> exprRefs, LogicalVariable aggVar) {
+    	for (Mutable<ILogicalExpression> exprRef : exprRefs) {
+            ILogicalExpression expr = exprRef.getValue();
+            if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                continue;
+            }
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            FunctionIdentifier funcIdent = AsterixBuiltinFunctions.getAggregateFunction(funcExpr.getFunctionIdentifier());
+            if (funcIdent == null) {
+            	// Recursively look in func args.
+            	return fingAggFuncExprRef(funcExpr.getArguments(), aggVar);
+            }
+            // Check if this is the expr that uses aggVar.
+            Collection<LogicalVariable> usedVars = new HashSet<LogicalVariable>();
+            funcExpr.getUsedVariables(usedVars);
+            if (usedVars.contains(aggVar)) {
+            	return exprRef;
+            }
+    	}
+    	return null;
+    }
+}
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 de0ee38..da96872 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,7 +11,6 @@
 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.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;
@@ -115,9 +114,8 @@
             return false;
         }
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-        AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
         AqlSourceId asid = ((IDataSource<AqlSourceId>) scan.getDataSource()).getId();
-        Dataset dataset = metadata.findDataset(asid.getDatasetName());
+        Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
         if (dataset == null) {
             throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
         }
@@ -136,7 +134,7 @@
         } else {
             int pos = ((AInt32) obj).getIntegerValue();
             String tName = dataset.getItemTypeName();
-            IAType t = metadata.findType(tName);
+            IAType t = mp.findType(dataset.getDataverseName(), tName);
             if (t.getTypeTag() != ATypeTag.RECORD) {
                 return false;
             }
@@ -147,7 +145,7 @@
             fldName = rt.getFieldNames()[pos];
         }
 
-        List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        List<Index> datasetIndexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         boolean hasSecondaryIndex = false;
         for (Index index : datasetIndexes) {
             if (index.isSecondaryIndex()) {
@@ -292,8 +290,7 @@
                             IDataSource<AqlSourceId> dataSource = (IDataSource<AqlSourceId>) scan.getDataSource();
                             AqlSourceId asid = dataSource.getId();
                             AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-                            AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                            Dataset dataset = metadata.findDataset(asid.getDatasetName());
+                            Dataset dataset = mp.findDataset(asid.getDataverseName(), asid.getDatasetName());
                             if (dataset == null) {
                                 throw new AlgebricksException("Dataset " + asid.getDatasetName() + " not found.");
                             }
@@ -310,7 +307,7 @@
                             } else {
                                 int pos = ((AInt32) obj).getIntegerValue();
                                 String tName = dataset.getItemTypeName();
-                                IAType t = metadata.findType(tName);
+                                IAType t = mp.findType(dataset.getDataverseName(), tName);
                                 if (t.getTypeTag() != ATypeTag.RECORD) {
                                     return false;
                                 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushSimilarityFunctionsBelowJoin.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushSimilarityFunctionsBelowJoin.java
new file mode 100644
index 0000000..c1aa1ac
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushSimilarityFunctionsBelowJoin.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2009-2012 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.optimizer.rules;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.rewriter.rules.PushFunctionsBelowJoin;
+
+/**
+ * Pushes similarity function-call expressions below a join if possible.
+ * Assigns the similarity function-call expressions to new variables, and replaces the original
+ * expression with a corresponding variable reference expression.
+ * This rule can help reduce the cost of computing expensive similarity functions by pushing them below
+ * a join (which may blow up the cardinality).
+ * Also, this rule may help to enable other rules such as common subexpression elimination, again to reduce
+ * the number of calls to expensive similarity functions.
+ * 
+ * Example:
+ * 
+ * Before plan:
+ * assign [$$10] <- [funcA(funcB(simFuncX($$3, $$4)))]
+ *   join (some condition) 
+ *     join_branch_0 where $$3 and $$4 are not live
+ *       ...
+ *     join_branch_1 where $$3 and $$4 are live
+ *       ...
+ * 
+ * After plan:
+ * assign [$$10] <- [funcA(funcB($$11))]
+ *   join (some condition) 
+ *     join_branch_0 where $$3 and $$4 are not live
+ *       ...
+ *     join_branch_1 where $$3 and $$4 are live
+ *       assign[$$11] <- [simFuncX($$3, $$4)]
+ *         ...
+ */
+public class PushSimilarityFunctionsBelowJoin extends PushFunctionsBelowJoin {
+
+    private static final Set<FunctionIdentifier> simFuncIdents = new HashSet<FunctionIdentifier>();
+    static {
+        simFuncIdents.add(AsterixBuiltinFunctions.SIMILARITY_JACCARD);
+        simFuncIdents.add(AsterixBuiltinFunctions.SIMILARITY_JACCARD_CHECK);
+        simFuncIdents.add(AsterixBuiltinFunctions.EDIT_DISTANCE);
+        simFuncIdents.add(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK);
+    }
+
+    public PushSimilarityFunctionsBelowJoin() {
+        super(simFuncIdents);
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
new file mode 100644
index 0000000..9089c42
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/RemoveUnusedOneToOneEquiJoinRule.java
@@ -0,0 +1,211 @@
+/*
+ * Copyright 2009-2012 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.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
+import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * Removes join operators for which all of the following conditions are true:
+ * 1. The live variables of one input branch of the join are not used in the upstream plan
+ * 2. The join is an inner equi join
+ * 3. The join condition only uses variables that correspond to primary keys of the same dataset    
+ * Notice that the last condition implies a 1:1 join, i.e., the join does not change the result cardinality.
+ * 
+ * Joins that satisfy the above conditions may be introduced by other rules 
+ * which use surrogate optimizations. Such an optimization aims to reduce data copies and communication costs by
+ * using the primary keys as surrogates for the desired data items. Typically,
+ * such a surrogate-based plan introduces a top-level join to finally resolve
+ * the surrogates to the desired data items. 
+ * In case the upstream plan does not require the original data items at all, such a top-level join is unnecessary.
+ * The purpose of this rule is to remove such unnecessary joins.
+ */
+public class RemoveUnusedOneToOneEquiJoinRule implements IAlgebraicRewriteRule {
+
+    private final Set<LogicalVariable> parentsUsedVars = new HashSet<LogicalVariable>();
+    private final List<LogicalVariable> usedVars = new ArrayList<LogicalVariable>();
+    private final List<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
+    private final List<LogicalVariable> pkVars = new ArrayList<LogicalVariable>();
+    private final List<DataSourceScanOperator> dataScans = new ArrayList<DataSourceScanOperator>();
+    private boolean hasRun = false;
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        if (hasRun) {
+            return false;
+        }
+        hasRun = true;
+        if (removeUnusedJoin(opRef)) {
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        return false;
+    }
+
+    private boolean removeUnusedJoin(Mutable<ILogicalOperator> opRef) throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        boolean modified = false;
+
+        usedVars.clear();
+        VariableUtilities.getUsedVariables(op, usedVars);
+        // Propagate used variables from parents downwards.
+        parentsUsedVars.addAll(usedVars);
+
+        int numInputs = op.getInputs().size();
+        for (int i = 0; i < numInputs; i++) {
+            Mutable<ILogicalOperator> childOpRef = op.getInputs().get(i);
+            int unusedJoinBranchIndex = removeJoinFromInputBranch(childOpRef);
+            if (unusedJoinBranchIndex >= 0) {
+                int usedBranchIndex = (unusedJoinBranchIndex == 0) ? 1 : 0;
+                // Remove join at input index i, by hooking up op's input i with 
+                // the join's branch at unusedJoinBranchIndex.
+                AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) childOpRef.getValue();
+                op.getInputs().set(i, joinOp.getInputs().get(usedBranchIndex));
+                modified = true;
+            }
+            // Descend into children.
+            if (removeUnusedJoin(childOpRef)) {
+                modified = true;
+            }
+        }
+        return modified;
+    }
+
+    private int removeJoinFromInputBranch(Mutable<ILogicalOperator> opRef) throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.INNERJOIN) {
+            return -1;
+        }
+
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) op;
+        // Make sure the join is an equi-join.
+        if (!isEquiJoin(joinOp.getCondition())) {
+            return -1;
+        }
+
+        int unusedJoinBranchIndex = -1;
+        for (int i = 0; i < joinOp.getInputs().size(); i++) {
+            liveVars.clear();
+            VariableUtilities.getLiveVariables(joinOp.getInputs().get(i).getValue(), liveVars);
+            liveVars.retainAll(parentsUsedVars);
+            if (liveVars.isEmpty()) {
+                // None of the live variables from this branch are used by its parents.
+                unusedJoinBranchIndex = i;
+                break;
+            }
+        }
+        if (unusedJoinBranchIndex < 0) {
+            // The variables from both branches are used in the upstream plan. We cannot remove this join.
+            return -1;
+        }
+
+        // Check whether one of the join branches is unused.
+        usedVars.clear();
+        VariableUtilities.getUsedVariables(joinOp, usedVars);
+
+        // Check whether all used variables originate from primary keys of exactly the same dataset.
+        // Collect a list of datascans whose primary key variables are used in the join condition.
+        gatherProducingDataScans(opRef, usedVars, dataScans);
+
+        // Check that all datascans scan the same dataset, and that the join condition
+        // only used primary key variables of those datascans.
+        for (int i = 0; i < dataScans.size(); i++) {
+            if (i > 0) {
+                AqlDataSource prevAqlDataSource = (AqlDataSource) dataScans.get(i - 1).getDataSource();
+                AqlDataSource currAqlDataSource = (AqlDataSource) dataScans.get(i).getDataSource();
+                if (!prevAqlDataSource.getDataset().equals(currAqlDataSource.getDataset())) {
+                    return -1;
+                }
+            }
+            // Remove from the used variables all the primary key vars of this dataset.
+            fillPKVars(dataScans.get(i), pkVars);
+            usedVars.removeAll(pkVars);
+        }
+        if (!usedVars.isEmpty()) {
+            // The join condition also uses some other variables that are not primary
+            // keys from datasource scans of the same dataset.
+            return -1;
+        }
+        return unusedJoinBranchIndex;
+    }
+
+    private void gatherProducingDataScans(Mutable<ILogicalOperator> opRef, List<LogicalVariable> joinUsedVars,
+            List<DataSourceScanOperator> dataScans) {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.DATASOURCESCAN) {
+            for (Mutable<ILogicalOperator> inputOp : op.getInputs()) {
+                gatherProducingDataScans(inputOp, joinUsedVars, dataScans);
+            }
+            return;
+        }
+        DataSourceScanOperator dataScan = (DataSourceScanOperator) op;
+        fillPKVars(dataScan, pkVars);
+        // Check if join uses all PK vars.
+        if (joinUsedVars.containsAll(pkVars)) {
+            dataScans.add(dataScan);
+        }
+    }
+
+    private void fillPKVars(DataSourceScanOperator dataScan, List<LogicalVariable> pkVars) {
+        pkVars.clear();
+        AqlDataSource aqlDataSource = (AqlDataSource) dataScan.getDataSource();
+        int numPKs = DatasetUtils.getPartitioningKeys(aqlDataSource.getDataset()).size();
+        pkVars.clear();
+        for (int i = 0; i < numPKs; i++) {
+            pkVars.add(dataScan.getVariables().get(i));
+        }
+    }
+
+    private boolean isEquiJoin(Mutable<ILogicalExpression> conditionExpr) {
+        AbstractLogicalExpression expr = (AbstractLogicalExpression) conditionExpr.getValue();
+        if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            FunctionIdentifier funcIdent = funcExpr.getFunctionIdentifier();
+            if (funcIdent != AlgebricksBuiltinFunctions.AND && funcIdent != AlgebricksBuiltinFunctions.EQ) {
+                return false;
+            }
+        }
+        return true;
+    }
+}
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 74f790f..19791e3 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
@@ -14,6 +14,7 @@
 import edu.uci.ics.asterix.metadata.declared.AqlSourceId;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.optimizer.rules.am.AccessMethodJobGenParams;
+import edu.uci.ics.asterix.optimizer.rules.am.BTreeJobGenParams;
 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;
@@ -149,8 +150,8 @@
                         AccessMethodJobGenParams jobGenParams = new AccessMethodJobGenParams();
                         jobGenParams.readFromFuncArgs(f.getArguments());
                         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-                        String dataverseName = mp.getMetadataDeclarations().getDataverseName();
-                        AqlSourceId dataSourceId = new AqlSourceId(dataverseName, jobGenParams.getDatasetName());
+                        AqlSourceId dataSourceId = new AqlSourceId(jobGenParams.getDataverseName(),
+                                jobGenParams.getDatasetName());
                         IDataSourceIndex<String, AqlSourceId> dsi = mp.findDataSourceIndex(jobGenParams.getIndexName(),
                                 dataSourceId);
                         if (dsi == null) {
@@ -161,7 +162,11 @@
                         boolean requiresBroadcast = jobGenParams.getRequiresBroadcast();
                         switch (indexType) {
                             case BTREE: {
-                                op.setPhysicalOperator(new BTreeSearchPOperator(dsi, requiresBroadcast));
+                                BTreeJobGenParams btreeJobGenParams = new BTreeJobGenParams();
+                                btreeJobGenParams.readFromFuncArgs(f.getArguments());
+                                op.setPhysicalOperator(new BTreeSearchPOperator(dsi, requiresBroadcast,
+                                        btreeJobGenParams.isPrimaryIndex(), btreeJobGenParams.getLowKeyVarList(),
+                                        btreeJobGenParams.getHighKeyVarList()));
                                 break;
                             }
                             case RTREE: {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
index 0e7a220..3dad464 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SetClosedRecordConstructorsRule.java
@@ -92,47 +92,55 @@
             boolean changed = false;
             if (expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)) {
                 ARecordType reqType = (ARecordType) TypeComputerUtilities.getRequiredType(expr);
-                if (reqType != null) {
-                    if (reqType.isOpen())
-                        allClosed = false;
-                }
-                int n = expr.getArguments().size();
-                if (n % 2 > 0) {
-                    throw new AlgebricksException("Record constructor expected to have an even number of arguments: "
-                            + expr);
-                }
-                for (int i = 0; i < n / 2; i++) {
-                    ILogicalExpression a0 = expr.getArguments().get(2 * i).getValue();
-                    if (a0.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                        allClosed = false;
+                if (reqType == null || !reqType.isOpen()) {
+                    int n = expr.getArguments().size();
+                    if (n % 2 > 0) {
+                        throw new AlgebricksException(
+                                "Record constructor expected to have an even number of arguments: " + expr);
                     }
-                    Mutable<ILogicalExpression> aRef1 = expr.getArguments().get(2 * i + 1);
-                    ILogicalExpression a1 = aRef1.getValue();
-                    ClosedDataInfo cdi = a1.accept(this, arg);
-                    if (!cdi.dataIsClosed) {
-                        allClosed = false;
+                    for (int i = 0; i < n / 2; i++) {
+                        ILogicalExpression a0 = expr.getArguments().get(2 * i).getValue();
+                        if (a0.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+                            allClosed = false;
+                        }
+                        Mutable<ILogicalExpression> aRef1 = expr.getArguments().get(2 * i + 1);
+                        ILogicalExpression a1 = aRef1.getValue();
+                        ClosedDataInfo cdi = a1.accept(this, arg);
+                        if (!cdi.dataIsClosed) {
+                            allClosed = false;
+                        }
+                        if (cdi.expressionChanged) {
+                            aRef1.setValue(cdi.expression);
+                            changed = true;
+                        }
                     }
-                    if (cdi.expressionChanged) {
-                        aRef1.setValue(cdi.expression);
+                    if (allClosed) {
+                        expr.setFunctionInfo(FunctionUtils
+                                .getFunctionInfo(AsterixBuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR));
+                        GlobalConfig.ASTERIX_LOGGER.finest("Switching to CLOSED record constructor in " + expr + ".\n");
                         changed = true;
                     }
                 }
-                if (allClosed) {
-                    expr.setFunctionInfo(FunctionUtils
-                            .getFunctionInfo(AsterixBuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR));
-                    GlobalConfig.ASTERIX_LOGGER.finest("Switching to CLOSED record constructor in " + expr + ".\n");
-                    changed = true;
-                }
             } else {
-                for (Mutable<ILogicalExpression> e : expr.getArguments()) {
-                    ILogicalExpression ale = e.getValue();
-                    ClosedDataInfo cdi = ale.accept(this, arg);
-                    if (!cdi.dataIsClosed) {
-                        allClosed = false;
+                boolean rewrite = true;
+                if (expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)
+                        || (expr.getFunctionIdentifier().equals(AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR))) {
+                    IAType reqType = TypeComputerUtilities.getRequiredType(expr);
+                    if (reqType == null) {
+                        rewrite = false;
                     }
-                    if (cdi.expressionChanged) {
-                        e.setValue(cdi.expression);
-                        changed = true;
+                }
+                if (rewrite) {
+                    for (Mutable<ILogicalExpression> e : expr.getArguments()) {
+                        ILogicalExpression ale = e.getValue();
+                        ClosedDataInfo cdi = ale.accept(this, arg);
+                        if (!cdi.dataIsClosed) {
+                            allClosed = false;
+                        }
+                        if (cdi.expressionChanged) {
+                            e.setValue(cdi.expression);
+                            changed = true;
+                        }
                     }
                 }
             }
@@ -142,7 +150,12 @@
         @Override
         public ClosedDataInfo visitVariableReferenceExpression(VariableReferenceExpression expr, Void arg)
                 throws AlgebricksException {
-            boolean dataIsClosed = isClosedRec((IAType) env.getVarType(expr.getVariableReference()));
+            Object varType = env.getVarType(expr.getVariableReference());
+            if (varType == null) {
+                throw new AlgebricksException("Could not infer type for variable '" + expr.getVariableReference()
+                        + "'.");
+            }
+            boolean dataIsClosed = isClosedRec((IAType) varType);
             return new ClosedDataInfo(false, dataIsClosed, expr);
         }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SimilarityCheckRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SimilarityCheckRule.java
index 60e5a3e..e88e5b0 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SimilarityCheckRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/SimilarityCheckRule.java
@@ -7,8 +7,10 @@
 import org.apache.commons.lang3.mutable.MutableObject;
 
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
+import edu.uci.ics.asterix.om.base.ADouble;
 import edu.uci.ics.asterix.om.base.AFloat;
 import edu.uci.ics.asterix.om.base.AInt32;
+import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -31,35 +33,38 @@
 
 /**
  * Looks for a select operator, containing a condition:
- * 
  * similarity-function GE/GT/LE/LE constant/variable
- * 
  * Rewrites the select condition (and possibly the assign expr) with the equivalent similarity-check function.
- * 
  */
 public class SimilarityCheckRule implements IAlgebraicRewriteRule {
 
     @Override
-    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
-    	AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         // Look for select.
         if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
             return false;
         }
         SelectOperator select = (SelectOperator) op;
         Mutable<ILogicalExpression> condExpr = select.getCondition();
-        
+
         // Gather assigns below this select.
         List<AssignOperator> assigns = new ArrayList<AssignOperator>();
         AbstractLogicalOperator childOp = (AbstractLogicalOperator) select.getInputs().get(0).getValue();
+        // Skip selects.
+        while (childOp.getOperatorTag() == LogicalOperatorTag.SELECT) {
+            childOp = (AbstractLogicalOperator) childOp.getInputs().get(0).getValue();
+        }
         while (childOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-        	assigns.add((AssignOperator) childOp);
-        	childOp = (AbstractLogicalOperator) childOp.getInputs().get(0).getValue();
+            assigns.add((AssignOperator) childOp);
+            childOp = (AbstractLogicalOperator) childOp.getInputs().get(0).getValue();
         }
         return replaceSelectConditionExprs(condExpr, assigns, context);
     }
 
-    private boolean replaceSelectConditionExprs(Mutable<ILogicalExpression> expRef, List<AssignOperator> assigns, IOptimizationContext context) throws AlgebricksException {
+    private boolean replaceSelectConditionExprs(Mutable<ILogicalExpression> expRef, List<AssignOperator> assigns,
+            IOptimizationContext context) throws AlgebricksException {
         ILogicalExpression expr = expRef.getValue();
         if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
             return false;
@@ -77,10 +82,10 @@
             }
             return found;
         }
-        
+
         // Look for GE/GT/LE/LT.
-        if (funcIdent != AlgebricksBuiltinFunctions.GE && funcIdent != AlgebricksBuiltinFunctions.GT &&
-                funcIdent != AlgebricksBuiltinFunctions.LE && funcIdent != AlgebricksBuiltinFunctions.LT) {
+        if (funcIdent != AlgebricksBuiltinFunctions.GE && funcIdent != AlgebricksBuiltinFunctions.GT
+                && funcIdent != AlgebricksBuiltinFunctions.LE && funcIdent != AlgebricksBuiltinFunctions.LT) {
             return false;
         }
 
@@ -92,8 +97,8 @@
         // Normalized GE/GT/LE/LT as if constant was on the right hand side.
         FunctionIdentifier normFuncIdent = null;
         // One of the args must be a constant.
-        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {                    	
-        	ConstantExpression constExpr = (ConstantExpression) arg1;
+        if (arg1.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+            ConstantExpression constExpr = (ConstantExpression) arg1;
             constVal = (AsterixConstantValue) constExpr.getValue();
             nonConstExpr = arg2;
             // Get func ident as if swapping lhs and rhs.            
@@ -107,91 +112,101 @@
         } else {
             return false;
         }
-        
+
         // The other arg is a function call. We can directly replace the select condition with an equivalent similarity check expression.
         if (nonConstExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-        	return replaceWithFunctionCallArg(expRef, normFuncIdent, constVal, (AbstractFunctionCallExpression) nonConstExpr);
+            return replaceWithFunctionCallArg(expRef, normFuncIdent, constVal,
+                    (AbstractFunctionCallExpression) nonConstExpr);
         }
         // The other arg ist a variable. We may have to introduce an assign operator that assigns the result of a similarity-check function to a variable.
         if (nonConstExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-        	return replaceWithVariableArg(expRef, normFuncIdent, constVal, (VariableReferenceExpression) nonConstExpr, assigns, context);
+            return replaceWithVariableArg(expRef, normFuncIdent, constVal, (VariableReferenceExpression) nonConstExpr,
+                    assigns, context);
         }
         return false;
     }
-    
+
     private boolean replaceWithVariableArg(Mutable<ILogicalExpression> expRef, FunctionIdentifier normFuncIdent,
-    		AsterixConstantValue constVal, VariableReferenceExpression varRefExpr, List<AssignOperator> assigns, IOptimizationContext context) throws AlgebricksException {
-    	
-    	// Find variable in assigns to determine its originating function.    	
-    	LogicalVariable var = varRefExpr.getVariableReference();
-    	Mutable<ILogicalExpression> simFuncExprRef = null;
-    	ScalarFunctionCallExpression simCheckFuncExpr = null;
-    	AssignOperator matchingAssign = null;
-    	for (int i = 0; i < assigns.size(); i++) {
-    		AssignOperator assign = assigns.get(i);
-    		for (int j = 0; j < assign.getVariables().size(); j++) {
-    			// Check if variables match.
-    			if (var != assign.getVariables().get(j)) {
-    				continue;
-    			}
-    			// Check if corresponding expr is a function call.
-    			if (assign.getExpressions().get(j).getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-    				continue;
-    			}
-    			simFuncExprRef = assign.getExpressions().get(j);
-    			// Analyze function expression and get equivalent similarity check function.
-    			simCheckFuncExpr = getSimilarityCheckExpr(normFuncIdent, constVal, (AbstractFunctionCallExpression) simFuncExprRef.getValue());
-    			matchingAssign = assign;
-    			break;
-    		}
-    		if (simCheckFuncExpr != null) {
-    			break;
-    		}
-    	}
-    	
-    	// Only non-null if we found that varRefExpr refers to an optimizable similarity function call. 
-    	if (simCheckFuncExpr != null) {
-    		// Create a new assign under matchingAssign which assigns the result of our similarity-check function to a variable.
-    		LogicalVariable newVar = context.newVar();
-    		AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(simCheckFuncExpr));
-    		// Hook up inputs. 
-    		newAssign.getInputs().add(new MutableObject<ILogicalOperator>(matchingAssign.getInputs().get(0).getValue()));
-    		matchingAssign.getInputs().get(0).setValue(newAssign);    		
-    		
-    		// Replace select condition with a get-item on newVar.
+            AsterixConstantValue constVal, VariableReferenceExpression varRefExpr, List<AssignOperator> assigns,
+            IOptimizationContext context) throws AlgebricksException {
+
+        // Find variable in assigns to determine its originating function.    	
+        LogicalVariable var = varRefExpr.getVariableReference();
+        Mutable<ILogicalExpression> simFuncExprRef = null;
+        ScalarFunctionCallExpression simCheckFuncExpr = null;
+        AssignOperator matchingAssign = null;
+        for (int i = 0; i < assigns.size(); i++) {
+            AssignOperator assign = assigns.get(i);
+            for (int j = 0; j < assign.getVariables().size(); j++) {
+                // Check if variables match.
+                if (var != assign.getVariables().get(j)) {
+                    continue;
+                }
+                // Check if corresponding expr is a function call.
+                if (assign.getExpressions().get(j).getValue().getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                    continue;
+                }
+                simFuncExprRef = assign.getExpressions().get(j);
+                // Analyze function expression and get equivalent similarity check function.
+                simCheckFuncExpr = getSimilarityCheckExpr(normFuncIdent, constVal,
+                        (AbstractFunctionCallExpression) simFuncExprRef.getValue());
+                matchingAssign = assign;
+                break;
+            }
+            if (simCheckFuncExpr != null) {
+                break;
+            }
+        }
+
+        // Only non-null if we found that varRefExpr refers to an optimizable similarity function call. 
+        if (simCheckFuncExpr != null) {
+            // Create a new assign under matchingAssign which assigns the result of our similarity-check function to a variable.
+            LogicalVariable newVar = context.newVar();
+            AssignOperator newAssign = new AssignOperator(newVar, new MutableObject<ILogicalExpression>(
+                    simCheckFuncExpr));
+            // Hook up inputs. 
+            newAssign.getInputs()
+                    .add(new MutableObject<ILogicalOperator>(matchingAssign.getInputs().get(0).getValue()));
+            matchingAssign.getInputs().get(0).setValue(newAssign);
+
+            // Replace select condition with a get-item on newVar.
             List<Mutable<ILogicalExpression>> selectGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
             // First arg is a variable reference expr on newVar.
             selectGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
             // Second arg is the item index to be accessed, here 0.
-            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
-            ILogicalExpression selectGetItemExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), selectGetItemArgs);
+            selectGetItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(
+                    new AsterixConstantValue(new AInt32(0)))));
+            ILogicalExpression selectGetItemExpr = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), selectGetItemArgs);
             // Replace the old similarity function call with the new getItemExpr.
             expRef.setValue(selectGetItemExpr);
-    		
+
             // Replace expr corresponding to original variable in the original assign with a get-item on newVar.
             List<Mutable<ILogicalExpression>> assignGetItemArgs = new ArrayList<Mutable<ILogicalExpression>>();
             // First arg is a variable reference expr on newVar.
             assignGetItemArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(newVar)));
             // Second arg is the item index to be accessed, here 1.
-            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(1)))));
-            ILogicalExpression assignGetItemExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), assignGetItemArgs);
+            assignGetItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(
+                    new AsterixConstantValue(new AInt32(1)))));
+            ILogicalExpression assignGetItemExpr = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), assignGetItemArgs);
             // Replace the original assign expr with the get-item expr.
             simFuncExprRef.setValue(assignGetItemExpr);
-    		
+
             context.computeAndSetTypeEnvironmentForOperator(newAssign);
             context.computeAndSetTypeEnvironmentForOperator(matchingAssign);
-            
-    		return true;
-    	}
-    	
-    	return false;
+
+            return true;
+        }
+
+        return false;
     }
-    
+
     private boolean replaceWithFunctionCallArg(Mutable<ILogicalExpression> expRef, FunctionIdentifier normFuncIdent,
-    		AsterixConstantValue constVal, AbstractFunctionCallExpression funcExpr) {
-    	// Analyze func expr to see if it is an optimizable similarity function.
-        ScalarFunctionCallExpression simCheckFuncExpr = getSimilarityCheckExpr(normFuncIdent, constVal, funcExpr); 
-        
+            AsterixConstantValue constVal, AbstractFunctionCallExpression funcExpr) {
+        // Analyze func expr to see if it is an optimizable similarity function.
+        ScalarFunctionCallExpression simCheckFuncExpr = getSimilarityCheckExpr(normFuncIdent, constVal, funcExpr);
+
         // Replace the expr in the select condition.
         if (simCheckFuncExpr != null) {
             // Get item 0 from var.
@@ -199,8 +214,10 @@
             // First arg is the similarity-check function call.
             getItemArgs.add(new MutableObject<ILogicalExpression>(simCheckFuncExpr));
             // Second arg is the item index to be accessed.
-            getItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(0)))));
-            ILogicalExpression getItemExpr = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), getItemArgs);
+            getItemArgs.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                    new AInt32(0)))));
+            ILogicalExpression getItemExpr = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.GET_ITEM), getItemArgs);
             // Replace the old similarity function call with the new getItemExpr.
             expRef.setValue(getItemExpr);
             return true;
@@ -208,20 +225,29 @@
 
         return false;
     }
-    
+
     private ScalarFunctionCallExpression getSimilarityCheckExpr(FunctionIdentifier normFuncIdent,
-    		AsterixConstantValue constVal, AbstractFunctionCallExpression funcExpr) {
-    	// Remember args from original similarity function to add them to the similarity-check function later.
+            AsterixConstantValue constVal, AbstractFunctionCallExpression funcExpr) {
+        // Remember args from original similarity function to add them to the similarity-check function later.
         ArrayList<Mutable<ILogicalExpression>> similarityArgs = null;
         ScalarFunctionCallExpression simCheckFuncExpr = null; 
         // Look for jaccard function call, and GE or GT.
         if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.SIMILARITY_JACCARD) {
-            AFloat aFloat = (AFloat) constVal.getObject();
-            AFloat jaccThresh;
-            if (normFuncIdent == AlgebricksBuiltinFunctions.GE) {
-                jaccThresh = aFloat;
+            IAObject jaccThresh;
+            if (normFuncIdent == AlgebricksBuiltinFunctions.GE) {                
+                if (constVal.getObject() instanceof AFloat) {
+                    jaccThresh = constVal.getObject();                    
+                } else {
+                    jaccThresh = new AFloat((float)((ADouble) constVal.getObject()).getDoubleValue());
+                }
             } else if (normFuncIdent == AlgebricksBuiltinFunctions.GT) {
-                float f = aFloat.getFloatValue() + Float.MIN_VALUE;
+                float threshVal = 0.0f;            
+                if (constVal.getObject() instanceof AFloat) {
+                    threshVal = ((AFloat) constVal.getObject()).getFloatValue();
+                } else {
+                    threshVal = (float)((ADouble) constVal.getObject()).getDoubleValue();                    
+                }
+                float f = threshVal + Float.MIN_VALUE;
                 if (f > 1.0f) f = 1.0f;
                 jaccThresh = new AFloat(f);
             } else {
@@ -253,9 +279,13 @@
             simCheckFuncExpr = new ScalarFunctionCallExpression(
                     FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK), similarityArgs);
         }
+        // Preserve all annotations.
+        if (simCheckFuncExpr != null) {
+            simCheckFuncExpr.getAnnotations().putAll(funcExpr.getAnnotations());
+        }
         return simCheckFuncExpr;
     }
-    
+
     private FunctionIdentifier getLhsAndRhsSwappedFuncIdent(FunctionIdentifier oldFuncIdent) {
         if (oldFuncIdent == AlgebricksBuiltinFunctions.GE) {
             return AlgebricksBuiltinFunctions.LE;
@@ -271,7 +301,7 @@
         }
         throw new IllegalStateException();
     }
-    
+
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
         return false;
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 f25a671..34c5739 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,12 +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.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.entities.Dataverse;
 import edu.uci.ics.asterix.metadata.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -19,6 +19,7 @@
 import edu.uci.ics.asterix.om.types.ATypeTag;
 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;
 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;
@@ -73,16 +74,19 @@
                 if (acv2.getObject().getType().getTypeTag() != ATypeTag.STRING) {
                     return false;
                 }
-                String datasetName = ((AString) acv2.getObject()).getStringValue();
+                String datasetArg = ((AString) acv2.getObject()).getStringValue();
 
-                AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-                AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                Dataset dataset = metadata.findDataset(datasetName);
+                AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+                Pair<String, String> datasetReference = parseDatasetReference(metadataProvider, datasetArg);
+                String dataverseName = datasetReference.first;
+                String datasetName = datasetReference.second;
+                Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
                 if (dataset == null) {
-                    throw new AlgebricksException("Could not find dataset " + datasetName);
+                    throw new AlgebricksException("Could not find dataset " + datasetName + " in dataverse "
+                            + dataverseName);
                 }
 
-                AqlSourceId asid = new AqlSourceId(metadata.getDataverseName(), datasetName);
+                AqlSourceId asid = new AqlSourceId(dataverseName, datasetName);
 
                 ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
 
@@ -94,7 +98,7 @@
                 }
                 v.add(unnest.getVariable());
 
-                DataSourceScanOperator scan = new DataSourceScanOperator(v, mp.findDataSource(asid));
+                DataSourceScanOperator scan = new DataSourceScanOperator(v, metadataProvider.findDataSource(asid));
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
                 opRef.setValue(scan);
@@ -121,12 +125,13 @@
                 if (acv2.getObject().getType().getTypeTag() != ATypeTag.STRING) {
                     return false;
                 }
-                String datasetName = ((AString) acv2.getObject()).getStringValue();
+                String datasetArg = ((AString) acv2.getObject()).getStringValue();
 
-                AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
-                AqlCompiledMetadataDeclarations metadata = mp.getMetadataDeclarations();
-                Dataset dataset = metadata.findDataset(datasetName);
-
+                AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
+                Pair<String, String> datasetReference = parseDatasetReference(metadataProvider, datasetArg);
+                String dataverseName = datasetReference.first;
+                String datasetName = datasetReference.second;
+                Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
                 if (dataset == null) {
                     throw new AlgebricksException("Could not find dataset " + datasetName);
                 }
@@ -135,20 +140,12 @@
                     throw new IllegalArgumentException("invalid dataset type:" + dataset.getDatasetType());
                 }
 
-                AqlSourceId asid = new AqlSourceId(metadata.getDataverseName(), datasetName);
-
+                AqlSourceId asid = new AqlSourceId(dataverseName, datasetName);
                 ArrayList<LogicalVariable> v = new ArrayList<LogicalVariable>();
-
-                /*
-                int numPrimaryKeys = DatasetUtils.getPartitioningFunctions(acdd).size();
-                for (int i = 0; i < numPrimaryKeys; i++) {
-                    v.add(context.newVar());
-                }*/
-
                 v.add(unnest.getVariable());
 
                 DataSourceScanOperator scan = new DataSourceScanOperator(v, createDummyFeedDataSource(asid, dataset,
-                        metadata));
+                        metadataProvider));
 
                 List<Mutable<ILogicalOperator>> scanInpList = scan.getInputs();
                 scanInpList.addAll(unnest.getInputs());
@@ -163,18 +160,6 @@
         return false;
     }
 
-    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
-            AqlCompiledMetadataDeclarations metadata) throws AlgebricksException {
-        if (!aqlId.getDataverseName().equals(metadata.getDataverseName())) {
-            return null;
-        }
-        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));
@@ -183,4 +168,37 @@
         FunctionalDependency pk = new FunctionalDependency(head, tail);
         context.addPrimaryKey(pk);
     }
+
+    private AqlDataSource createDummyFeedDataSource(AqlSourceId aqlId, Dataset dataset,
+            AqlMetadataProvider metadataProvider) throws AlgebricksException {
+        if (!aqlId.getDataverseName().equals(
+                metadataProvider.getDefaultDataverse() == null ? null : metadataProvider.getDefaultDataverse()
+                        .getDataverseName())) {
+            return null;
+        }
+        String tName = dataset.getItemTypeName();
+        IAType itemType = metadataProvider.findType(dataset.getDataverseName(), tName);
+        ExternalFeedDataSource extDataSource = new ExternalFeedDataSource(aqlId, dataset, itemType,
+                AqlDataSource.AqlDataSourceType.EXTERNAL_FEED);
+        return extDataSource;
+    }
+
+    private Pair<String, String> parseDatasetReference(AqlMetadataProvider metadataProvider, String datasetArg)
+            throws AlgebricksException {
+        String[] datasetNameComponents = datasetArg.split("\\.");
+        String dataverseName;
+        String datasetName;
+        if (datasetNameComponents.length == 1) {
+            Dataverse defaultDataverse = metadataProvider.getDefaultDataverse();
+            if (defaultDataverse == null) {
+                throw new AlgebricksException("Unresolved dataset " + datasetArg + " Dataverse not specified.");
+            }
+            dataverseName = defaultDataverse.getDataverseName();
+            datasetName = datasetNameComponents[0];
+        } else {
+            dataverseName = datasetNameComponents[0];
+            datasetName = datasetNameComponents[1];
+        }
+        return new Pair<String, String>(dataverseName, datasetName);
+    }
 }
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 b378ed7..8bfef17 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,7 +7,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
-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;
@@ -37,7 +36,7 @@
  */
 public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
 
-    private AqlCompiledMetadataDeclarations metadata;
+    private AqlMetadataProvider metadataProvider;
 
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
 
@@ -60,23 +59,18 @@
     }
 
     protected void setMetadataDeclarations(IOptimizationContext context) {
-        AqlMetadataProvider metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
-        metadata = metadataProvider.getMetadataDeclarations();
+        metadataProvider = (AqlMetadataProvider) context.getMetadataProvider();
     }
 
     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();
         Iterator<Map.Entry<IAccessMethod, AccessMethodAnalysisContext>> amIt = analyzedAMs.entrySet().iterator();
         // Check applicability of indexes by access method type.
         while (amIt.hasNext()) {
             Map.Entry<IAccessMethod, AccessMethodAnalysisContext> entry = amIt.next();
             AccessMethodAnalysisContext amCtx = entry.getValue();
-            // For the current access method type, map variables from the assign op to applicable indexes.
-            fillAllIndexExprs(varList, subTree, amCtx);
+            // For the current access method type, map variables to applicable indexes.
+            fillAllIndexExprs(subTree, amCtx);
         }
     }
 
@@ -124,13 +118,13 @@
         Iterator<Map.Entry<Index, List<Integer>>> it = analysisCtx.indexExprs.entrySet().iterator();
         while (it.hasNext()) {
             Map.Entry<Index, List<Integer>> entry = it.next();
-            Index index = entry.getKey();
-            Iterator<Integer> exprsIter = entry.getValue().iterator();
+            Index index = entry.getKey();            
             boolean allUsed = true;
             int lastFieldMatched = -1;
             for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
                 String keyField = index.getKeyFieldNames().get(i);
                 boolean foundKeyField = false;
+                Iterator<Integer> exprsIter = entry.getValue().iterator();
                 while (exprsIter.hasNext()) {
                     Integer ix = exprsIter.next();
                     IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(ix);
@@ -156,12 +150,10 @@
             // If the access method requires all exprs to be matched but they are not, remove this candidate.
             if (!allUsed && accessMethod.matchAllIndexExprs()) {
                 it.remove();
-                return;
             }
             // A prefix of the index exprs may have been matched.
             if (lastFieldMatched < 0 && accessMethod.matchPrefixIndexExprs()) {
                 it.remove();
-                return;
             }
         }
     }
@@ -240,7 +232,8 @@
      */
     protected boolean fillIndexExprs(String fieldName, int matchedFuncExprIndex, Dataset dataset,
             AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
-        List<Index> datasetIndexes = metadata.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
+        List<Index> datasetIndexes = metadataProvider.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) {
@@ -259,37 +252,43 @@
         return true;
     }
 
-    protected void fillAllIndexExprs(List<LogicalVariable> varList, OptimizableOperatorSubTree subTree,
+    protected void fillAllIndexExprs(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);
-                IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(optFuncExprIndex);
+            IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(optFuncExprIndex);
+            // Try to match variables from optFuncExpr to assigns.
+            for (int assignIndex = 0; assignIndex < subTree.assigns.size(); assignIndex++) {
+                AssignOperator assignOp = subTree.assigns.get(assignIndex);
+                List<LogicalVariable> varList = assignOp.getVariables();
+                for (int varIndex = 0; varIndex < varList.size(); varIndex++) {
+                    LogicalVariable var = varList.get(varIndex);
+                    int funcVarIndex = optFuncExpr.findLogicalVar(var);
+                    // No matching var in optFuncExpr.
+                    if (funcVarIndex == -1) {
+                        continue;
+                    }
+                    // At this point we have matched the optimizable func expr at optFuncExprIndex to an assigned variable.
+                    String fieldName = getFieldNameOfFieldAccess(assignOp, subTree.recordType, varIndex);
+                    if (fieldName == null) {
+                        continue;
+                    }
+                    // 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.dataset, analysisCtx);
+                }
+            }
+            // Try to match variables from optFuncExpr to datasourcescan if not already matched in assigns.
+            List<LogicalVariable> dsVarList = subTree.dataSourceScan.getVariables();
+            for (int varIndex = 0; varIndex < dsVarList.size(); varIndex++) {
+                LogicalVariable var = dsVarList.get(varIndex);
                 int funcVarIndex = optFuncExpr.findLogicalVar(var);
                 // No matching var in optFuncExpr.
                 if (funcVarIndex == -1) {
                     continue;
                 }
-                // At this point we have matched the optimizable func expr at optFuncExprIndex to an assigned variable.
-                String fieldName = null;
-                if (!subTree.assigns.isEmpty()) {
-                    // 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);
-                    if (fieldName == null) {
-                        continue;
-                    }
-                } else {
-                    // We don't have an assign, only a datasource scan.
-                    // The last var. is the record itself, so skip it.
-                    if (varIndex >= varList.size() - 1) {
-                        break;
-                    }
-                    // The variable value is one of the partitioning fields.
-                    fieldName = DatasetUtils.getPartitioningKeys(subTree.dataset).get(varIndex);
-                }
+                // The variable value is one of the partitioning fields.
+                String 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);
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 af30163..e3a9e91 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
@@ -20,27 +20,32 @@
 public class AccessMethodJobGenParams {
     protected String indexName;
     protected IndexType indexType;
+    protected String dataverseName;
     protected String datasetName;
     protected boolean retainInput;
     protected boolean requiresBroadcast;
+    protected boolean isPrimaryIndex;
 
-    private final int NUM_PARAMS = 5;
+    private final int NUM_PARAMS = 6;
 
     public AccessMethodJobGenParams() {
     }
 
-    public AccessMethodJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
-            boolean requiresBroadcast) {
+    public AccessMethodJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
+            boolean retainInput, boolean requiresBroadcast) {
         this.indexName = indexName;
         this.indexType = indexType;
+        this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.retainInput = retainInput;
         this.requiresBroadcast = requiresBroadcast;
+        this.isPrimaryIndex = datasetName.equals(indexName);
     }
 
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(indexName)));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createInt32Constant(indexType.ordinal())));
+        funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(dataverseName)));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createStringConstant(datasetName)));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(retainInput)));
         funcArgs.add(new MutableObject<ILogicalExpression>(AccessMethodUtils.createBooleanConstant(requiresBroadcast)));
@@ -49,9 +54,12 @@
     public void readFromFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
         indexName = AccessMethodUtils.getStringConstant(funcArgs.get(0));
         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));
+        dataverseName = AccessMethodUtils.getStringConstant(funcArgs.get(2));
+        datasetName = AccessMethodUtils.getStringConstant(funcArgs.get(3));
+        retainInput = AccessMethodUtils.getBooleanConstant(funcArgs.get(4));
+        requiresBroadcast = AccessMethodUtils.getBooleanConstant(funcArgs.get(5));
+        isPrimaryIndex = datasetName.equals(indexName);
+        isPrimaryIndex = datasetName.equals(indexName);
     }
 
     public String getIndexName() {
@@ -62,6 +70,10 @@
         return indexType;
     }
 
+    public String getDataverseName() {
+        return dataverseName;
+    }
+
     public String getDatasetName() {
         return datasetName;
     }
@@ -100,4 +112,8 @@
     protected int getNumParams() {
         return NUM_PARAMS;
     }
+
+    public boolean isPrimaryIndex() {
+        return isPrimaryIndex;
+    }
 }
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 cd3712d..6651ea3 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
@@ -109,6 +109,24 @@
         analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr, fieldVar, constFilterVal));
         return true;
     }
+    
+    public static boolean analyzeFuncExprArgsForTwoVars(AbstractFunctionCallExpression funcExpr,
+            AccessMethodAnalysisContext analysisCtx) {
+        LogicalVariable fieldVar1 = null;
+        LogicalVariable fieldVar2 = null;
+        ILogicalExpression arg1 = funcExpr.getArguments().get(0).getValue();
+        ILogicalExpression arg2 = funcExpr.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() == LogicalExpressionTag.VARIABLE
+                && arg2.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
+            fieldVar1 = ((VariableReferenceExpression) arg1).getVariableReference();
+            fieldVar2 = ((VariableReferenceExpression) arg2).getVariableReference();
+        } else {
+            return false;
+        }
+        analysisCtx.matchedFuncExprs.add(new OptimizableFuncExpr(funcExpr,
+                new LogicalVariable[] { fieldVar1, fieldVar2 }, null));
+        return true;
+    }
 
     public static int getNumSecondaryKeys(Index index, ARecordType recordType) throws AlgebricksException {
         switch (index.getIndexType()) {
@@ -190,6 +208,35 @@
         return primaryKeyVars;
     }
 
+    /**
+     * Returns the search key expression which feeds a secondary-index search. If we are optimizing a selection query then this method returns
+     * the a ConstantExpression from the first constant value in the optimizable function expression.
+     * If we are optimizing a join, then this method returns the VariableReferenceExpression that should feed the secondary index probe.
+     */
+    public static ILogicalExpression createSearchKeyExpr(IOptimizableFuncExpr optFuncExpr,
+            OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree) {
+        if (probeSubTree == null) {
+            // We are optimizing a selection query. Search key is a constant.
+            return new ConstantExpression(optFuncExpr.getConstantVal(0));
+        } else {
+            // We are optimizing a join query. Determine which variable feeds the secondary index. 
+            if (optFuncExpr.getOperatorSubTree(0) == null || optFuncExpr.getOperatorSubTree(0) == probeSubTree) {
+                return new VariableReferenceExpression(optFuncExpr.getLogicalVar(0));
+            } else {
+                return new VariableReferenceExpression(optFuncExpr.getLogicalVar(1));
+            }
+        }
+    }
+
+    /**
+     *  Returns the first expr optimizable by this index.
+     */
+    public static IOptimizableFuncExpr chooseFirstOptFuncExpr(Index chosenIndex, AccessMethodAnalysisContext analysisCtx) {
+        List<Integer> indexExprs = analysisCtx.getIndexExprs(chosenIndex);
+        int firstExprIndex = indexExprs.get(0);
+        return analysisCtx.matchedFuncExprs.get(firstExprIndex);
+    }
+    
     public static UnnestMapOperator createSecondaryIndexUnnestMap(Dataset dataset, ARecordType recordType, Index index,
             ILogicalOperator inputOp, AccessMethodJobGenParams jobGenParams, IOptimizationContext context,
             boolean outputPrimaryKeysOnly, boolean retainInput) throws AlgebricksException {
@@ -240,7 +287,7 @@
         // The job gen parameters are transferred to the actual job gen via the UnnestMapOperator's function arguments. 
         List<Mutable<ILogicalExpression>> primaryIndexFuncArgs = new ArrayList<Mutable<ILogicalExpression>>();
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(dataset.getDatasetName(), IndexType.BTREE,
-                dataset.getDatasetName(), retainInput, requiresBroadcast);
+                dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
         // Set low/high inclusive to true for a point lookup.
         jobGenParams.setLowKeyInclusive(true);
         jobGenParams.setHighKeyInclusive(true);
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 414dca3..1379bf4 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
@@ -1,6 +1,7 @@
 package edu.uci.ics.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -21,13 +22,14 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
-import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IndexedNLJoinExpressionAnnotation;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
 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.operators.logical.AbstractBinaryJoinOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -57,7 +59,6 @@
         funcIdents.add(AlgebricksBuiltinFunctions.GE);
         funcIdents.add(AlgebricksBuiltinFunctions.LT);
         funcIdents.add(AlgebricksBuiltinFunctions.GT);
-        funcIdents.add(AlgebricksBuiltinFunctions.NEQ);
     }
 
     public static BTreeAccessMethod INSTANCE = new BTreeAccessMethod();
@@ -70,7 +71,11 @@
     @Override
     public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
             AccessMethodAnalysisContext analysisCtx) {
-        return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
+        boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
+        if (!matches) {
+            matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVars(funcExpr, analysisCtx);
+        }
+        return matches;
     }
 
     @Override
@@ -88,20 +93,90 @@
     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;
+        Mutable<ILogicalExpression> conditionRef = select.getCondition();
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(selectRef, conditionRef, subTree, null,
+                chosenIndex, analysisCtx, false, false, context);
+        if (primaryIndexUnnestOp == null) {
+            return false;
+        }
         Mutable<ILogicalOperator> assignRef = (subTree.assignRefs.isEmpty()) ? null : subTree.assignRefs.get(0);
         AssignOperator assign = null;
         if (assignRef != null) {
             assign = (AssignOperator) assignRef.getValue();
         }
+        // Generate new select using the new condition.
+        if (conditionRef.getValue() != null) {
+            select.getInputs().clear();
+            if (assign != null) {
+                subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
+                select.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+            } else {
+                select.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestOp));
+            }
+        } else {
+            ((AbstractLogicalOperator) primaryIndexUnnestOp).setExecutionMode(ExecutionMode.PARTITIONED);
+            if (assign != null) {
+                subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
+                selectRef.setValue(assign);
+            } else {
+                selectRef.setValue(primaryIndexUnnestOp);
+            }
+        }
+        return true;
+    }
+
+    @Override
+    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
+        Mutable<ILogicalExpression> conditionRef = joinOp.getCondition();
+        // Determine if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
+        Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
+        // Determine probe and index subtrees based on chosen index.
+        OptimizableOperatorSubTree indexSubTree = null;
+        OptimizableOperatorSubTree probeSubTree = null;
+        if (leftSubTree.dataset != null && dataset.getDatasetName().equals(leftSubTree.dataset.getDatasetName())) {
+            indexSubTree = leftSubTree;
+            probeSubTree = rightSubTree;
+        } else if (rightSubTree.dataset != null
+                && dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
+            indexSubTree = rightSubTree;
+            probeSubTree = leftSubTree;
+        }
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(joinRef, conditionRef, indexSubTree,
+                probeSubTree, chosenIndex, analysisCtx, true, true, context);
+        if (primaryIndexUnnestOp == null) {
+            return false;
+        }
+        // If there are conditions left, add a new select operator on top.
+        indexSubTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
+        if (conditionRef.getValue() != null) {
+            SelectOperator topSelect = new SelectOperator(conditionRef);
+            topSelect.getInputs().add(indexSubTree.rootRef);
+            topSelect.setExecutionMode(ExecutionMode.LOCAL);
+            context.computeAndSetTypeEnvironmentForOperator(topSelect);
+            // Replace the original join with the new subtree rooted at the select op.
+            joinRef.setValue(topSelect);
+        } else {
+            joinRef.setValue(indexSubTree.rootRef.getValue());
+        }
+        return true;
+    }
+
+    private ILogicalOperator createSecondaryToPrimaryPlan(Mutable<ILogicalOperator> topOpRef,
+            Mutable<ILogicalExpression> conditionRef, OptimizableOperatorSubTree indexSubTree,
+            OptimizableOperatorSubTree probeSubTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
+            boolean retainInput, boolean requiresBroadcast, IOptimizationContext context) throws AlgebricksException {
+        Dataset dataset = indexSubTree.dataset;
+        ARecordType recordType = indexSubTree.recordType;
+        DataSourceScanOperator dataSourceScan = indexSubTree.dataSourceScan;
         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];
+        ILogicalExpression[] lowKeyExprs = new ILogicalExpression[numSecondaryKeys];
+        ILogicalExpression[] highKeyExprs = new ILogicalExpression[numSecondaryKeys];
         LimitType[] lowKeyLimits = new LimitType[numSecondaryKeys];
         LimitType[] highKeyLimits = new LimitType[numSecondaryKeys];
         boolean[] lowKeyInclusive = new boolean[numSecondaryKeys];
@@ -116,6 +191,9 @@
         // If we can't figure out how to integrate a certain funcExpr into the current predicate, we just bail by setting this flag.
         boolean couldntFigureOut = false;
         boolean doneWithExprs = false;
+        // TODO: For now don't consider prefix searches.
+        BitSet setLowKeys = new BitSet(numSecondaryKeys);
+        BitSet setHighKeys = new BitSet(numSecondaryKeys);
         // Go through the func exprs listed as optimizable by the chosen index, 
         // and formulate a range predicate on the secondary-index keys.
         for (Integer exprIndex : exprList) {
@@ -123,26 +201,40 @@
             IOptimizableFuncExpr optFuncExpr = matchedFuncExprs.get(exprIndex);
             int keyPos = indexOf(optFuncExpr.getFieldName(0), chosenIndex.getKeyFieldNames());
             if (keyPos < 0) {
-                throw new InternalError();
+                if (optFuncExpr.getNumLogicalVars() > 1) {
+                    // If we are optimizing a join, the matching field may be the second field name.
+                    keyPos = indexOf(optFuncExpr.getFieldName(1), chosenIndex.getKeyFieldNames());
+                }                
             }
+            if (keyPos < 0) {
+                throw new AlgebricksException(
+                        "Could not match optimizable function expression to any index field name.");
+            }
+            ILogicalExpression searchKeyExpr = AccessMethodUtils.createSearchKeyExpr(optFuncExpr, indexSubTree,
+                    probeSubTree);
             LimitType limit = getLimitType(optFuncExpr);
             switch (limit) {
                 case EQUAL: {
                     if (lowKeyLimits[keyPos] == null && highKeyLimits[keyPos] == null) {
                         lowKeyLimits[keyPos] = highKeyLimits[keyPos] = limit;
                         lowKeyInclusive[keyPos] = highKeyInclusive[keyPos] = true;
-                        lowKeyConstants[keyPos] = highKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        lowKeyExprs[keyPos] = highKeyExprs[keyPos] = searchKeyExpr;
+                        setLowKeys.set(keyPos);
+                        setHighKeys.set(keyPos);
                     } else {
                         couldntFigureOut = true;
                     }
-                    // Mmmm, we would need an inference system here.
-                    doneWithExprs = true;
+                    // TODO: For now don't consider prefix searches.
+                    // If high and low keys are set, we exit for now.
+                    if (setLowKeys.cardinality() == numSecondaryKeys && setHighKeys.cardinality() == numSecondaryKeys) {
+                        doneWithExprs = true;
+                    }
                     break;
                 }
                 case HIGH_EXCLUSIVE: {
                     if (highKeyLimits[keyPos] == null || (highKeyLimits[keyPos] != null && highKeyInclusive[keyPos])) {
                         highKeyLimits[keyPos] = limit;
-                        highKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        highKeyExprs[keyPos] = searchKeyExpr;
                         highKeyInclusive[keyPos] = false;
                     } else {
                         couldntFigureOut = true;
@@ -153,7 +245,7 @@
                 case HIGH_INCLUSIVE: {
                     if (highKeyLimits[keyPos] == null) {
                         highKeyLimits[keyPos] = limit;
-                        highKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        highKeyExprs[keyPos] = searchKeyExpr;
                         highKeyInclusive[keyPos] = true;
                     } else {
                         couldntFigureOut = true;
@@ -164,7 +256,7 @@
                 case LOW_EXCLUSIVE: {
                     if (lowKeyLimits[keyPos] == null || (lowKeyLimits[keyPos] != null && lowKeyInclusive[keyPos])) {
                         lowKeyLimits[keyPos] = limit;
-                        lowKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        lowKeyExprs[keyPos] = searchKeyExpr;
                         lowKeyInclusive[keyPos] = false;
                     } else {
                         couldntFigureOut = true;
@@ -175,7 +267,7 @@
                 case LOW_INCLUSIVE: {
                     if (lowKeyLimits[keyPos] == null) {
                         lowKeyLimits[keyPos] = limit;
-                        lowKeyConstants[keyPos] = optFuncExpr.getConstantVal(0);
+                        lowKeyExprs[keyPos] = searchKeyExpr;
                         lowKeyInclusive[keyPos] = true;
                     } else {
                         couldntFigureOut = true;
@@ -196,22 +288,22 @@
             }
         }
         if (couldntFigureOut) {
-            return false;
+            return null;
         }
 
         // Rule out the cases unsupported by the current btree search
         // implementation.
         for (int i = 1; i < numSecondaryKeys; i++) {
             if (lowKeyInclusive[i] != lowKeyInclusive[0] || highKeyInclusive[i] != highKeyInclusive[0]) {
-                return false;
+                return null;
             }
             if (lowKeyLimits[0] == null && lowKeyLimits[i] != null || lowKeyLimits[0] != null
                     && lowKeyLimits[i] == null) {
-                return false;
+                return null;
             }
             if (highKeyLimits[0] == null && highKeyLimits[i] != null || highKeyLimits[0] != null
                     && highKeyLimits[i] == null) {
-                return false;
+                return null;
             }
         }
         if (lowKeyLimits[0] == null) {
@@ -224,97 +316,92 @@
         // 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>();
-        // List of expressions for the assign.
-        ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
-        int numLowKeys = createKeyVarsAndExprs(lowKeyLimits, lowKeyConstants, keyExprList, keyVarList, context);
-        int numHighKeys = createKeyVarsAndExprs(highKeyLimits, highKeyConstants, keyExprList, keyVarList, context);
+        // List of variables and expressions for the assign.
+        ArrayList<LogicalVariable> assignKeyVarList = new ArrayList<LogicalVariable>();
+        ArrayList<Mutable<ILogicalExpression>> assignKeyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+        int numLowKeys = createKeyVarsAndExprs(lowKeyLimits, lowKeyExprs, assignKeyVarList, assignKeyExprList,
+                keyVarList, context);
+        int numHighKeys = createKeyVarsAndExprs(highKeyLimits, highKeyExprs, assignKeyVarList, assignKeyExprList,
+                keyVarList, context);
 
         BTreeJobGenParams jobGenParams = new BTreeJobGenParams(chosenIndex.getIndexName(), IndexType.BTREE,
-                dataset.getDatasetName(), false, false);
+                dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
         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());
+        ILogicalOperator inputOp = null;
+        if (!assignKeyVarList.isEmpty()) {
+            // Assign operator that sets the constant secondary-index search-key fields if necessary.
+            AssignOperator assignConstantSearchKeys = new AssignOperator(assignKeyVarList, assignKeyExprList);
+            // Input to this assign is the EmptyTupleSource (which the dataSourceScan also must have had as input).
+            assignConstantSearchKeys.getInputs().add(dataSourceScan.getInputs().get(0));
+            assignConstantSearchKeys.setExecutionMode(dataSourceScan.getExecutionMode());
+            inputOp = assignConstantSearchKeys;
+        } else {
+            // All index search keys are variables.
+            inputOp = probeSubTree.root;
+        }
 
         UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
-                chosenIndex, assignSearchKeys, jobGenParams, context, false, false);
+                chosenIndex, inputOp, jobGenParams, context, false, retainInput);
 
         // Generate the rest of the upstream plan which feeds the search results into the primary index.        
         UnnestMapOperator primaryIndexUnnestOp;
         boolean isPrimaryIndex = chosenIndex.getIndexName().equals(dataset.getDatasetName());
         if (!isPrimaryIndex) {
             primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset, recordType,
-                    secondaryIndexUnnestOp, context, true, false, false);
+                    secondaryIndexUnnestOp, context, true, retainInput, false);
         } else {
             List<Object> primaryIndexOutputTypes = new ArrayList<Object>();
             AccessMethodUtils.appendPrimaryIndexTypes(dataset, recordType, primaryIndexOutputTypes);
             primaryIndexUnnestOp = new UnnestMapOperator(dataSourceScan.getVariables(),
-                    secondaryIndexUnnestOp.getExpressionRef(), primaryIndexOutputTypes, false);
-            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(assignSearchKeys));
+                    secondaryIndexUnnestOp.getExpressionRef(), primaryIndexOutputTypes, retainInput);
+            primaryIndexUnnestOp.getInputs().add(new MutableObject<ILogicalOperator>(inputOp));
         }
 
         List<Mutable<ILogicalExpression>> remainingFuncExprs = new ArrayList<Mutable<ILogicalExpression>>();
-        getNewSelectExprs(select, replacedFuncExprs, remainingFuncExprs);
-        // Generate new select using the new condition.
+        getNewConditionExprs(conditionRef, replacedFuncExprs, remainingFuncExprs);
+        // Generate new condition.
         if (!remainingFuncExprs.isEmpty()) {
             ILogicalExpression pulledCond = createSelectCondition(remainingFuncExprs);
-            SelectOperator selectRest = new SelectOperator(new MutableObject<ILogicalExpression>(pulledCond));
-            if (assign != null) {
-                subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
-                selectRest.getInputs().add(new MutableObject<ILogicalOperator>(assign));
-            } else {
-                selectRest.getInputs().add(new MutableObject<ILogicalOperator>(primaryIndexUnnestOp));
-            }
-            selectRest.setExecutionMode(((AbstractLogicalOperator) selectRef.getValue()).getExecutionMode());
-            selectRef.setValue(selectRest);
+            conditionRef.setValue(pulledCond);
         } else {
-            primaryIndexUnnestOp.setExecutionMode(ExecutionMode.PARTITIONED);
-            if (assign != null) {
-                subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
-                selectRef.setValue(assign);
-            } else {
-                selectRef.setValue(primaryIndexUnnestOp);
-            }
+            conditionRef.setValue(null);
         }
-        return true;
+        return primaryIndexUnnestOp;
     }
 
-    @Override
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            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) {
+    private int createKeyVarsAndExprs(LimitType[] keyLimits, ILogicalExpression[] searchKeyExprs,
+            ArrayList<LogicalVariable> assignKeyVarList, ArrayList<Mutable<ILogicalExpression>> assignKeyExprList,
+            ArrayList<LogicalVariable> keyVarList, IOptimizationContext context) {
         if (keyLimits[0] == null) {
             return 0;
         }
         int numKeys = keyLimits.length;
         for (int i = 0; i < numKeys; i++) {
-            LogicalVariable keyVar = context.newVar();
+            ILogicalExpression searchKeyExpr = searchKeyExprs[i];
+            LogicalVariable keyVar = null;
+            if (searchKeyExpr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+                keyVar = context.newVar();
+                assignKeyExprList.add(new MutableObject<ILogicalExpression>(searchKeyExpr));
+                assignKeyVarList.add(keyVar);
+            } else {
+                keyVar = ((VariableReferenceExpression) searchKeyExpr).getVariableReference();
+            }
             keyVarList.add(keyVar);
-            keyExprList.add(new MutableObject<ILogicalExpression>(new ConstantExpression(keyConstants[i])));
         }
         return numKeys;
     }
 
-    private void getNewSelectExprs(SelectOperator select, Set<ILogicalExpression> replacedFuncExprs,
-            List<Mutable<ILogicalExpression>> remainingFuncExprs) {
+    private void getNewConditionExprs(Mutable<ILogicalExpression> conditionRef,
+            Set<ILogicalExpression> replacedFuncExprs, List<Mutable<ILogicalExpression>> remainingFuncExprs) {
         remainingFuncExprs.clear();
         if (replacedFuncExprs.isEmpty()) {
             return;
         }
-        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) select.getCondition().getValue();
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) conditionRef.getValue();
         if (replacedFuncExprs.size() == 1) {
             Iterator<ILogicalExpression> it = replacedFuncExprs.iterator();
             if (!it.hasNext()) {
@@ -405,6 +492,12 @@
 
     @Override
     public boolean exprIsOptimizable(Index index, IOptimizableFuncExpr optFuncExpr) {
+        // If we are optimizing a join, check for the indexed nested-loop join hint.
+        if (optFuncExpr.getNumLogicalVars() == 2) {
+            if (!optFuncExpr.getFuncExpr().getAnnotations().containsKey(IndexedNLJoinExpressionAnnotation.INSTANCE)) {
+                return false;
+            }
+        }
         // 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 c377a34..9a735c9 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
@@ -27,9 +27,9 @@
         super();
     }
 
-    public BTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
+    public BTreeJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName, boolean retainInput,
             boolean requiresBroadcast) {
-        super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
+        super(indexName, indexType, dataverseName, datasetName, retainInput, requiresBroadcast);
     }
 
     public void setLowKeyVarList(List<LogicalVariable> keyVarList, int startIndex, int numKeys) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IOptimizableFuncExpr.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
index dd91fc1..aa38ce9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/IOptimizableFuncExpr.java
@@ -22,4 +22,5 @@
     
     public int findLogicalVar(LogicalVariable var);
     public int findFieldName(String fieldName);
+    public void substituteVar(LogicalVariable original, LogicalVariable substitution);
 }
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 e4555aa..d6f0279 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
@@ -57,12 +57,15 @@
     // Register access methods.
     protected static Map<FunctionIdentifier, List<IAccessMethod>> accessMethods = new HashMap<FunctionIdentifier, List<IAccessMethod>>();
     static {
-        registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);
+        registerAccessMethod(BTreeAccessMethod.INSTANCE, accessMethods);
+        registerAccessMethod(RTreeAccessMethod.INSTANCE, accessMethods);
+        registerAccessMethod(InvertedIndexAccessMethod.INSTANCE, accessMethods);        
     }
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
+        clear();
         setMetadataDeclarations(context);
 
         // Match operator pattern and initialize optimizable sub trees.
@@ -116,7 +119,7 @@
         boolean res = chosenIndex.first.applyJoinPlanTransformation(joinRef, leftSubTree, rightSubTree,
                 chosenIndex.second, analysisCtx, context);
         if (res) {
-            OperatorPropertiesUtil.typeOpRec(opRef, context);
+            OperatorPropertiesUtil.typeOpRec(opRef, context);            
         }
         context.addToDontApplySet(this, join);
         return res;
@@ -153,4 +156,10 @@
     public Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods() {
         return accessMethods;
     }
+    
+    private void clear() {
+        joinRef = null;
+        join = null;
+        joinCond = null;
+    }
 }
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 59b11fc..abfcfb3 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
@@ -70,6 +70,7 @@
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
+        clear();
         setMetadataDeclarations(context);
 
         // Match operator pattern and initialize operator members.
@@ -134,4 +135,10 @@
     public Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods() {
         return accessMethods;
     }
+    
+    private void clear() {
+        selectRef = null;
+        select = null;
+        selectCond = null;
+    }
 }
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 dbd92c0..84a86b1 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
@@ -1,8 +1,10 @@
 package edu.uci.ics.asterix.optimizer.rules.am;
 
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -44,13 +46,13 @@
 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.expressions.VariableReferenceExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
@@ -321,7 +323,8 @@
         DataSourceScanOperator dataSourceScan = indexSubTree.dataSourceScan;
 
         InvertedIndexJobGenParams jobGenParams = new InvertedIndexJobGenParams(chosenIndex.getIndexName(),
-                chosenIndex.getIndexType(), dataset.getDatasetName(), retainInput, requiresBroadcast);
+                chosenIndex.getIndexType(), dataset.getDataverseName(), 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.
@@ -377,7 +380,7 @@
     public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
             OptimizableOperatorSubTree subTree, Index chosenIndex, AccessMethodAnalysisContext analysisCtx,
             IOptimizationContext context) throws AlgebricksException {
-        IOptimizableFuncExpr optFuncExpr = chooseOptFuncExpr(chosenIndex, analysisCtx);
+        IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
         ILogicalOperator indexPlanRootOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr, false,
                 false, context);
         // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
@@ -401,22 +404,34 @@
             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).
+        IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
         InnerJoinOperator join = (InnerJoinOperator) joinRef.getValue();
+
+        // Remember the original probe subtree, and its primary-key variables,
+        // so we can later retrieve the missing attributes via an equi join.
+        List<LogicalVariable> originalSubTreePKs = new ArrayList<LogicalVariable>();
+        // Remember the primary-keys of the new probe subtree for the top-level equi join.
+        List<LogicalVariable> surrogateSubTreePKs = new ArrayList<LogicalVariable>();
+
+        // Copy probe subtree, replacing their variables with new ones. We will use the original variables
+        // to stitch together a top-level equi join.
+        Mutable<ILogicalOperator> originalProbeSubTreeRootRef = copyAndReinitProbeSubTree(probeSubTree, join
+                .getCondition().getValue(), optFuncExpr, originalSubTreePKs, surrogateSubTreePKs, context);
+
+        // Remember original live variables from the index sub tree.
+        List<LogicalVariable> indexSubTreeLiveVars = new ArrayList<LogicalVariable>();
+        VariableUtilities.getLiveVariables(indexSubTree.root, indexSubTreeLiveVars);
+
+        // Clone the original join condition because we may have to modify it (and we also need the original).        
         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;
+        Map<LogicalVariable, LogicalVariable> panicVarMap = null;
         if (optFuncExpr.getFuncExpr().getFunctionIdentifier() == AsterixBuiltinFunctions.EDIT_DISTANCE_CHECK) {
             panicJoinRef = new MutableObject<ILogicalOperator>(joinRef.getValue());
+            panicVarMap = new HashMap<LogicalVariable, LogicalVariable>();
             Mutable<ILogicalOperator> newProbeRootRef = createPanicNestedLoopJoinPlan(panicJoinRef, indexSubTree,
-                    probeSubTree, optFuncExpr, chosenIndex, context);
+                    probeSubTree, optFuncExpr, chosenIndex, panicVarMap, context);
             probeSubTree.rootRef.setValue(newProbeRootRef.getValue());
             probeSubTree.root = newProbeRootRef.getValue();
         }
@@ -430,52 +445,150 @@
         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);
+        ILogicalOperator topOp = topSelect;
 
         // 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);
+            LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
+            indexSubTreeLiveVars.addAll(originalSubTreePKs);
+            indexSubTreeLiveVars.add(inputSearchVar);
             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)));
+            for (int i = 0; i < indexSubTreeLiveVars.size(); i++) {
+                LogicalVariable indexSubTreeVar = indexSubTreeLiveVars.get(i);
+                LogicalVariable panicPlanVar = panicVarMap.get(indexSubTreeVar);
+                if (panicPlanVar == null) {
+                    panicPlanVar = indexSubTreeVar;
+                }
+                varMap.add(new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(indexSubTreeVar, panicPlanVar,
+                        indexSubTreeVar));
             }
             UnionAllOperator unionAllOp = new UnionAllOperator(varMap);
-            unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(joinRef.getValue()));
+            unionAllOp.getInputs().add(new MutableObject<ILogicalOperator>(topOp));
             unionAllOp.getInputs().add(panicJoinRef);
             unionAllOp.setExecutionMode(ExecutionMode.PARTITIONED);
             context.computeAndSetTypeEnvironmentForOperator(unionAllOp);
-            joinRef.setValue(unionAllOp);
+            topOp = unionAllOp;
         }
+
+        // Place a top-level equi-join on top to retrieve the missing variables from the original probe subtree.
+        // The inner (build) branch of the join is the subtree with the data scan, since the result of the similarity join could potentially be big.
+        // This choice may not always be the most efficient, but it seems more robust than the alternative.
+        Mutable<ILogicalExpression> eqJoinConditionRef = createPrimaryKeysEqJoinCondition(originalSubTreePKs,
+                surrogateSubTreePKs);
+        InnerJoinOperator topEqJoin = new InnerJoinOperator(eqJoinConditionRef, originalProbeSubTreeRootRef,
+                new MutableObject<ILogicalOperator>(topOp));
+        topEqJoin.setExecutionMode(ExecutionMode.PARTITIONED);
+        joinRef.setValue(topEqJoin);
+        context.computeAndSetTypeEnvironmentForOperator(topEqJoin);
+
         return true;
     }
 
-    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);
-        int firstExprIndex = indexExprs.get(0);
-        return analysisCtx.matchedFuncExprs.get(firstExprIndex);
+    /**
+     * Copies the probeSubTree (using new variables), and reinitializes the probeSubTree to it.
+     * Accordingly replaces the variables in the given joinCond, and the optFuncExpr.
+     * Returns a reference to the original plan root.
+     */
+    private Mutable<ILogicalOperator> copyAndReinitProbeSubTree(OptimizableOperatorSubTree probeSubTree,
+            ILogicalExpression joinCond, IOptimizableFuncExpr optFuncExpr, List<LogicalVariable> originalSubTreePKs,
+            List<LogicalVariable> surrogateSubTreePKs, IOptimizationContext context) throws AlgebricksException {
+
+        probeSubTree.getPrimaryKeyVars(originalSubTreePKs);
+
+        // Create two copies of the original probe subtree.
+        // The first copy, which becomes the new probe subtree, will retain the primary-key and secondary-search key variables,
+        // but have all other variables replaced with new ones.
+        // The second copy, which will become an input to the top-level equi-join to resolve the surrogates, 
+        // will have all primary-key and secondary-search keys replaced, but retains all other original variables.
+
+        // Variable replacement map for the first copy.
+        Map<LogicalVariable, LogicalVariable> newProbeSubTreeVarMap = new HashMap<LogicalVariable, LogicalVariable>();
+        // Variable replacement map for the second copy.
+        Map<LogicalVariable, LogicalVariable> joinInputSubTreeVarMap = new HashMap<LogicalVariable, LogicalVariable>();
+        // Init with all live vars.
+        List<LogicalVariable> liveVars = new ArrayList<LogicalVariable>();
+        VariableUtilities.getLiveVariables(probeSubTree.root, liveVars);
+        for (LogicalVariable var : liveVars) {
+            joinInputSubTreeVarMap.put(var, var);
+        }
+        // Fill variable replacement maps.
+        for (int i = 0; i < optFuncExpr.getNumLogicalVars(); i++) {
+            joinInputSubTreeVarMap.put(optFuncExpr.getLogicalVar(i), context.newVar());
+            newProbeSubTreeVarMap.put(optFuncExpr.getLogicalVar(i), optFuncExpr.getLogicalVar(i));
+        }
+        for (int i = 0; i < originalSubTreePKs.size(); i++) {            
+            LogicalVariable newPKVar = context.newVar();
+            surrogateSubTreePKs.add(newPKVar);
+            joinInputSubTreeVarMap.put(originalSubTreePKs.get(i), newPKVar);
+            newProbeSubTreeVarMap.put(originalSubTreePKs.get(i), originalSubTreePKs.get(i));
+        }
+        
+        // Create first copy.
+        Counter firstCounter = new Counter(context.getVarCounter());
+        LogicalOperatorDeepCopyVisitor firstDeepCopyVisitor = new LogicalOperatorDeepCopyVisitor(firstCounter,
+                newProbeSubTreeVarMap);
+        ILogicalOperator newProbeSubTree = firstDeepCopyVisitor.deepCopy(probeSubTree.root, null);
+        inferTypes(newProbeSubTree, context);
+        Mutable<ILogicalOperator> newProbeSubTreeRootRef = new MutableObject<ILogicalOperator>(newProbeSubTree);        
+        context.setVarCounter(firstCounter.get());
+        // Create second copy.
+        Counter secondCounter = new Counter(context.getVarCounter());
+        LogicalOperatorDeepCopyVisitor secondDeepCopyVisitor = new LogicalOperatorDeepCopyVisitor(secondCounter,
+                joinInputSubTreeVarMap);        
+        ILogicalOperator joinInputSubTree = secondDeepCopyVisitor.deepCopy(probeSubTree.root, null);
+        inferTypes(joinInputSubTree, context);
+        probeSubTree.rootRef.setValue(joinInputSubTree);
+        context.setVarCounter(secondCounter.get());
+        
+        // Remember the original probe subtree reference so we can return it.
+        Mutable<ILogicalOperator> originalProbeSubTreeRootRef = probeSubTree.rootRef;
+
+        // Replace the original probe subtree with its copy.
+        Dataset origDataset = probeSubTree.dataset;
+        ARecordType origRecordType = probeSubTree.recordType;
+        probeSubTree.initFromSubTree(newProbeSubTreeRootRef);        
+        probeSubTree.dataset = origDataset;
+        probeSubTree.recordType = origRecordType;
+
+        // Replace the variables in the join condition based on the mapping of variables
+        // in the new probe subtree.
+        Map<LogicalVariable, LogicalVariable> varMapping = firstDeepCopyVisitor.getVariableMapping();
+        for (Map.Entry<LogicalVariable, LogicalVariable> varMapEntry : varMapping.entrySet()) {
+            if (varMapEntry.getKey() != varMapEntry.getValue()) {
+                joinCond.substituteVar(varMapEntry.getKey(), varMapEntry.getValue());
+            }
+        }
+        return originalProbeSubTreeRootRef;
+    }
+
+    private Mutable<ILogicalExpression> createPrimaryKeysEqJoinCondition(List<LogicalVariable> originalSubTreePKs,
+            List<LogicalVariable> surrogateSubTreePKs) {
+        List<Mutable<ILogicalExpression>> eqExprs = new ArrayList<Mutable<ILogicalExpression>>();
+        int numPKVars = originalSubTreePKs.size();
+        for (int i = 0; i < numPKVars; i++) {
+            List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(surrogateSubTreePKs.get(i))));
+            args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(originalSubTreePKs.get(i))));                   
+            ILogicalExpression eqFunc = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.EQ), args);
+            eqExprs.add(new MutableObject<ILogicalExpression>(eqFunc));
+        }
+        if (eqExprs.size() == 1) {
+            return eqExprs.get(0);
+        } else {
+            ILogicalExpression andFunc = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.AND), eqExprs);
+            return new MutableObject<ILogicalExpression>(andFunc);
+        }
     }
 
     private Mutable<ILogicalOperator> createPanicNestedLoopJoinPlan(Mutable<ILogicalOperator> joinRef,
             OptimizableOperatorSubTree indexSubTree, OptimizableOperatorSubTree probeSubTree,
-            IOptimizableFuncExpr optFuncExpr, Index chosenIndex, IOptimizationContext context)
-            throws AlgebricksException {
+            IOptimizableFuncExpr optFuncExpr, Index chosenIndex, Map<LogicalVariable, LogicalVariable> panicVarMap,
+            IOptimizationContext context) throws AlgebricksException {
         LogicalVariable inputSearchVar = getInputSearchVar(optFuncExpr, indexSubTree);
 
         // We split the plan into two "branches", and add selections on each side.
@@ -485,8 +598,8 @@
         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);
+        IVariableTypeEnvironment probeTypeEnv = context.getOutputTypeEnvironment(probeSubTree.root);
+        IAType inputSearchVarType = (IAType) probeTypeEnv.getVarType(inputSearchVar);
         Mutable<ILogicalOperator> isFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         Mutable<ILogicalOperator> isNotFilterableSelectOpRef = new MutableObject<ILogicalOperator>();
         createIsFilterableSelectOps(replicateOp, inputSearchVar, inputSearchVarType, optFuncExpr, chosenIndex, context,
@@ -500,6 +613,8 @@
         LogicalOperatorDeepCopyVisitor deepCopyVisitor = new LogicalOperatorDeepCopyVisitor(counter);
         ILogicalOperator scanSubTree = deepCopyVisitor.deepCopy(indexSubTree.root, null);
         context.setVarCounter(counter.get());
+        Map<LogicalVariable, LogicalVariable> copyVarMap = deepCopyVisitor.getVariableMapping();
+        panicVarMap.putAll(copyVarMap);
 
         List<LogicalVariable> copyLiveVars = new ArrayList<LogicalVariable>();
         VariableUtilities.getLiveVariables(scanSubTree, copyLiveVars);
@@ -508,14 +623,8 @@
         // 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>();
-        VariableUtilities.getUsedVariables(joinOp, joinCondUsedVars);
-        for (int i = 0; i < joinCondUsedVars.size(); i++) {
-            int ix = originalLiveVars.indexOf(joinCondUsedVars.get(i));
-            if (ix >= 0) {
-                joinOp.getCondition().getValue().substituteVar(originalLiveVars.get(ix), copyLiveVars.get(ix));
-            }
+        for (Map.Entry<LogicalVariable, LogicalVariable> entry : copyVarMap.entrySet()) {
+            joinOp.getCondition().getValue().substituteVar(entry.getKey(), entry.getValue());
         }
         joinOp.getInputs().clear();
         joinOp.getInputs().add(new MutableObject<ILogicalOperator>(scanSubTree));
@@ -831,4 +940,11 @@
             }
         }
     }
+
+    private void inferTypes(ILogicalOperator op, IOptimizationContext context) throws AlgebricksException {
+        for (Mutable<ILogicalOperator> childOpRef : op.getInputs()) {
+            inferTypes(childOpRef.getValue(), context);
+        }
+        context.computeAndSetTypeEnvironmentForOperator(op);
+    }
 }
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 530606e..65473c7 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
@@ -30,9 +30,9 @@
     public InvertedIndexJobGenParams() {
     }
 
-    public InvertedIndexJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
-            boolean requiresBroadcast) {
-        super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
+    public InvertedIndexJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
+            boolean retainInput, boolean requiresBroadcast) {
+        super(indexName, indexType, dataverseName, datasetName, retainInput, requiresBroadcast);
     }
 
     public void setSearchModifierType(SearchModifierType searchModifierType) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableFuncExpr.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableFuncExpr.java
index 13e515a..dbbe6d9 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableFuncExpr.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/am/OptimizableFuncExpr.java
@@ -96,4 +96,16 @@
     public OptimizableOperatorSubTree getOperatorSubTree(int index) {
         return subTrees[index];
     }
+
+    @Override
+    public void substituteVar(LogicalVariable original, LogicalVariable substitution) {
+        if (logicalVars != null) {
+            for (int i = 0; i < logicalVars.length; i++) {
+                if (logicalVars[i] == original) {
+                    logicalVars[i] = substitution;
+                    break;
+                }
+            }
+        }
+    }
 }
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 80f8cc1..fd1b89d 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,16 +6,18 @@
 import org.apache.commons.lang3.mutable.Mutable;
 
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-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.utils.DatasetUtils;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.optimizer.base.AnalysisUtil;
 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.ILogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
@@ -27,8 +29,8 @@
  * (select)? <-- (datasource scan)
  */
 public class OptimizableOperatorSubTree {
-    public ILogicalOperator root;
-    public Mutable<ILogicalOperator> rootRef;
+    public ILogicalOperator root = null;
+    public Mutable<ILogicalOperator> rootRef = null;
     public final List<Mutable<ILogicalOperator>> assignRefs = new ArrayList<Mutable<ILogicalOperator>>();
     public final List<AssignOperator> assigns = new ArrayList<AssignOperator>();
     public Mutable<ILogicalOperator> dataSourceScanRef = null;
@@ -38,6 +40,7 @@
     public ARecordType recordType = null;
 
     public boolean initFromSubTree(Mutable<ILogicalOperator> subTreeOpRef) {
+        reset();
         rootRef = subTreeOpRef;
         root = subTreeOpRef.getValue();
         // Examine the op's children to match the expected patterns.
@@ -87,12 +90,13 @@
             return false;
         }
         // Find the dataset corresponding to the datasource scan in the metadata.
-        String datasetName = AnalysisUtil.getDatasetName(dataSourceScan);
-        if (datasetName == null) {
+        Pair<String, String> datasetInfo = AnalysisUtil.getDatasetInfo(dataSourceScan);
+        String dataverseName = datasetInfo.first;
+        String datasetName = datasetInfo.second;
+        if (dataverseName == null || datasetName == null) {
             return false;
         }
-        AqlCompiledMetadataDeclarations metadata = metadataProvider.getMetadataDeclarations();
-        dataset = metadata.findDataset(datasetName);
+        dataset = metadataProvider.findDataset(dataverseName, datasetName);
         if (dataset == null) {
             throw new AlgebricksException("No metadata for dataset " + datasetName);
         }
@@ -100,7 +104,7 @@
             return false;
         }
         // Get the record type for that dataset.
-        IAType itemType = metadata.findType(dataset.getItemTypeName());
+        IAType itemType = metadataProvider.findType(dataverseName, dataset.getItemTypeName());
         if (itemType.getTypeTag() != ATypeTag.RECORD) {
             return false;
         }
@@ -111,4 +115,22 @@
     public boolean hasDataSourceScan() {
         return dataSourceScan != null;
     }
+    
+    public void reset() {
+        root = null;
+        rootRef = null;
+        assignRefs.clear();
+        assigns.clear();
+        dataSourceScanRef = null;
+        dataSourceScan = null;
+        dataset = null;
+        recordType = null;
+    }
+    
+    public void getPrimaryKeyVars(List<LogicalVariable> target) {
+        int numPrimaryKeys = DatasetUtils.getPartitioningKeys(dataset).size();
+        for (int i = 0; i < numPrimaryKeys; i++) {
+            target.add(dataSourceScan.getVariables().get(i));
+        }
+    }
 }
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 dfd3ff7..45cce9c 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
@@ -26,8 +26,11 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 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.operators.logical.AbstractBinaryJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.DataSourceScanOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 
 /**
@@ -50,7 +53,11 @@
     @Override
     public boolean analyzeFuncExprArgs(AbstractFunctionCallExpression funcExpr, List<AssignOperator> assigns,
             AccessMethodAnalysisContext analysisCtx) {
-        return AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
+        boolean matches = AccessMethodUtils.analyzeFuncExprArgsForOneConstAndVar(funcExpr, analysisCtx);
+        if (!matches) {
+            matches = AccessMethodUtils.analyzeFuncExprArgsForTwoVars(funcExpr, analysisCtx);
+        }
+        return matches;
     }
 
     @Override
@@ -65,27 +72,71 @@
 
     @Override
     public boolean applySelectPlanTransformation(Mutable<ILogicalOperator> selectRef,
-            OptimizableOperatorSubTree subTree, Index index, AccessMethodAnalysisContext analysisCtx,
+            OptimizableOperatorSubTree subTree, Index chosenIndex, 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(index);
-        int firstExprIndex = indexExprs.get(0);
-        IOptimizableFuncExpr optFuncExpr = analysisCtx.matchedFuncExprs.get(firstExprIndex);
+        IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(subTree, null, chosenIndex, optFuncExpr,
+                false, false, context);
+        if (primaryIndexUnnestOp == null) {
+            return false;
+        }
+        // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
+        subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
+        return true;
+    }
 
-        // Get the number of dimensions corresponding to the field indexed by
-        // chosenIndex.
+    @Override
+    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
+            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
+            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
+        // Determine if the index is applicable on the left or right side (if both, we arbitrarily prefer the left side).
+        Dataset dataset = analysisCtx.indexDatasetMap.get(chosenIndex);
+        // Determine probe and index subtrees based on chosen index.
+        OptimizableOperatorSubTree indexSubTree = null;
+        OptimizableOperatorSubTree probeSubTree = null;
+        if (leftSubTree.dataset != null && dataset.getDatasetName().equals(leftSubTree.dataset.getDatasetName())) {
+            indexSubTree = leftSubTree;
+            probeSubTree = rightSubTree;
+        } else if (rightSubTree.dataset != null
+                && dataset.getDatasetName().equals(rightSubTree.dataset.getDatasetName())) {
+            indexSubTree = rightSubTree;
+            probeSubTree = leftSubTree;
+        }
+        // TODO: We can probably do something smarter here based on selectivity or MBR area.
+        IOptimizableFuncExpr optFuncExpr = AccessMethodUtils.chooseFirstOptFuncExpr(chosenIndex, analysisCtx);
+        ILogicalOperator primaryIndexUnnestOp = createSecondaryToPrimaryPlan(indexSubTree, probeSubTree, chosenIndex,
+                optFuncExpr, true, true, context);
+        if (primaryIndexUnnestOp == null) {
+            return false;
+        }
+        indexSubTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
+        // Change join into a select with the same condition.
+        AbstractBinaryJoinOperator joinOp = (AbstractBinaryJoinOperator) joinRef.getValue();
+        SelectOperator topSelect = new SelectOperator(joinOp.getCondition());
+        topSelect.getInputs().add(indexSubTree.rootRef);
+        topSelect.setExecutionMode(ExecutionMode.LOCAL);
+        context.computeAndSetTypeEnvironmentForOperator(topSelect);
+        // Replace the original join with the new subtree rooted at the select op.
+        joinRef.setValue(topSelect);
+        return true;
+    }
+
+    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;
+
+        // Get the number of dimensions corresponding to the field indexed by chosenIndex.
         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(index.getIndexName(), IndexType.RTREE,
-                dataset.getDatasetName(), false, false);
+        DataSourceScanOperator dataSourceScan = indexSubTree.dataSourceScan;
+        RTreeJobGenParams jobGenParams = new RTreeJobGenParams(chosenIndex.getIndexName(), IndexType.RTREE,
+                dataset.getDataverseName(), dataset.getDatasetName(), retainInput, requiresBroadcast);
         // 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).
@@ -93,13 +144,14 @@
         ArrayList<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
         // List of expressions for the assign.
         ArrayList<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+        ILogicalExpression searchKeyExpr = AccessMethodUtils.createSearchKeyExpr(optFuncExpr, indexSubTree,
+                probeSubTree);
         for (int i = 0; i < numSecondaryKeys; i++) {
             // The create MBR function "extracts" one field of an MBR around the given spatial object.
             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>(searchKeyExpr));
             // The number of dimensions.
             createMBR.getArguments().add(
                     new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(new AInt32(
@@ -117,26 +169,23 @@
 
         // Assign operator that "extracts" the MBR fields from the func-expr constant into a tuple.
         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());
+        if (probeSubTree == null) {
+            // We are optimizing a selection query.
+            // 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());
+        } else {
+            // We are optimizing a join, place the assign op top of the probe subtree.
+            assignSearchKeys.getInputs().add(probeSubTree.rootRef);
+        }
 
         UnnestMapOperator secondaryIndexUnnestOp = AccessMethodUtils.createSecondaryIndexUnnestMap(dataset, recordType,
-                index, assignSearchKeys, jobGenParams, context, false, false);
+                chosenIndex, assignSearchKeys, jobGenParams, context, false, retainInput);
         // Generate the rest of the upstream plan which feeds the search results into the primary index.
         UnnestMapOperator primaryIndexUnnestOp = AccessMethodUtils.createPrimaryIndexUnnestMap(dataSourceScan, dataset,
-                recordType, secondaryIndexUnnestOp, context, true, false, false);
-        // Replace the datasource scan with the new plan rooted at primaryIndexUnnestMap.
-        subTree.dataSourceScanRef.setValue(primaryIndexUnnestOp);
-        return true;
-    }
+                recordType, secondaryIndexUnnestOp, context, true, retainInput, false);
 
-    @Override
-    public boolean applyJoinPlanTransformation(Mutable<ILogicalOperator> joinRef,
-            OptimizableOperatorSubTree leftSubTree, OptimizableOperatorSubTree rightSubTree, Index chosenIndex,
-            AccessMethodAnalysisContext analysisCtx, IOptimizationContext context) throws AlgebricksException {
-        // TODO Implement this.
-        return false;
+        return primaryIndexUnnestOp;
     }
 
     @Override
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 b3153f9..846bcb6 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
@@ -20,9 +20,9 @@
     public RTreeJobGenParams() {
     }
 
-    public RTreeJobGenParams(String indexName, IndexType indexType, String datasetName, boolean retainInput,
-            boolean requiresBroadcast) {
-        super(indexName, indexType, datasetName, retainInput, requiresBroadcast);
+    public RTreeJobGenParams(String indexName, IndexType indexType, String dataverseName, String datasetName,
+            boolean retainInput, boolean requiresBroadcast) {
+        super(indexName, indexType, dataverseName, datasetName, retainInput, requiresBroadcast);
     }
 
     public void writeToFuncArgs(List<Mutable<ILogicalExpression>> funcArgs) {
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
new file mode 100644
index 0000000..0c6f2ea
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -0,0 +1,408 @@
+/*
+ * 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.optimizer.rules.typecast;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.asterix.om.base.ANull;
+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.pointables.base.DefaultOpenFieldType;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
+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.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+
+/**
+ * This class is utility to do type cast.
+ * It offers two public methods:
+ * 1. public static boolean rewriteListExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
+ * IVariableTypeEnvironment env) throws AlgebricksException, which only enforces the list type recursively.
+ * 2. public static boolean rewriteFuncExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
+ * IVariableTypeEnvironment env) throws AlgebricksException, which enforces the list type and the record type recursively.
+ * 
+ * @author yingyib
+ */
+public class StaticTypeCastUtil {
+
+    /**
+     * This method is only called when funcExpr contains list constructor function calls.
+     * The List constructor is very special because a nested list is of type List<ANY>.
+     * However, the bottom-up type inference (InferTypeRule in algebricks) did not infer that so we need this method to enforce the type.
+     * We do not want to break the generality of algebricks so this method is called in an ASTERIX rule: @ IntroduceEnforcedListTypeRule} .
+     * 
+     * @param funcExpr
+     *            record constructor function expression
+     * @param requiredListType
+     *            required record type
+     * @param inputRecordType
+     * @param env
+     *            type environment
+     * @throws AlgebricksException
+     */
+    public static boolean rewriteListExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
+            IVariableTypeEnvironment env) throws AlgebricksException {
+        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR) {
+            if (reqType.equals(BuiltinType.ANY)) {
+                reqType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+            }
+            return rewriteListFuncExpr(funcExpr, (AbstractCollectionType) reqType, (AbstractCollectionType) inputType,
+                    env);
+        } else if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR) {
+            if (reqType.equals(BuiltinType.ANY)) {
+                reqType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+            }
+            return rewriteListFuncExpr(funcExpr, (AbstractCollectionType) reqType, (AbstractCollectionType) inputType,
+                    env);
+        } else {
+            List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+            boolean changed = false;
+            for (Mutable<ILogicalExpression> arg : args) {
+                ILogicalExpression argExpr = arg.getValue();
+                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
+                    IAType exprType = (IAType) env.getType(argFuncExpr);
+                    changed = changed || rewriteListExpr(argFuncExpr, exprType, exprType, env);
+                }
+            }
+            return changed;
+        }
+    }
+
+    /**
+     * This method is to recursively enforce required types, for the list type and the record type.
+     * The List constructor is very special because
+     * 1. a nested list in a list is of type List<ANY>;
+     * 2. a nested record in a list is of type Open_Record{}.
+     * The open record constructor is very special because
+     * 1. a nested list in the open part is of type List<ANY>;
+     * 2. a nested record in the open part is of type Open_Record{}.
+     * However, the bottom-up type inference (InferTypeRule in algebricks) did not infer that so we need this method to enforce the type.
+     * We do not want to break the generality of algebricks so this method is called in an ASTERIX rule: @ IntroduceStaticTypeCastRule} .
+     * 
+     * @param funcExpr
+     *            the function expression whose type needs to be top-down enforced
+     * @param reqType
+     *            the required type inferred from parent operators/expressions
+     * @param inputType
+     *            the current inferred
+     * @param env
+     *            the type environment
+     * @return true if the type is casted; otherwise, false.
+     * @throws AlgebricksException
+     */
+    public static boolean rewriteFuncExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
+            IVariableTypeEnvironment env) throws AlgebricksException {
+        if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR) {
+            if (reqType.equals(BuiltinType.ANY)) {
+                reqType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+            }
+            return rewriteListFuncExpr(funcExpr, (AbstractCollectionType) reqType, (AbstractCollectionType) inputType,
+                    env);
+        } else if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR) {
+            if (reqType.equals(BuiltinType.ANY)) {
+                reqType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+            }
+            return rewriteListFuncExpr(funcExpr, (AbstractCollectionType) reqType, (AbstractCollectionType) inputType,
+                    env);
+        } else if (inputType.getTypeTag().equals(ATypeTag.RECORD)) {
+            if (reqType.equals(BuiltinType.ANY)) {
+                reqType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+            }
+            return rewriteRecordFuncExpr(funcExpr, (ARecordType) reqType, (ARecordType) inputType, env);
+        } else {
+            List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+            boolean changed = false;
+            for (Mutable<ILogicalExpression> arg : args) {
+                ILogicalExpression argExpr = arg.getValue();
+                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
+                    IAType exprType = (IAType) env.getType(argFuncExpr);
+                    changed = changed || rewriteFuncExpr(argFuncExpr, exprType, exprType, env);
+                }
+            }
+            return changed;
+        }
+    }
+
+    /**
+     * only called when funcExpr is record constructor
+     * 
+     * @param funcExpr
+     *            record constructor function expression
+     * @param requiredListType
+     *            required record type
+     * @param inputRecordType
+     * @param env
+     *            type environment
+     * @throws AlgebricksException
+     */
+    private static boolean rewriteRecordFuncExpr(AbstractFunctionCallExpression funcExpr,
+            ARecordType requiredRecordType, ARecordType inputRecordType, IVariableTypeEnvironment env)
+            throws AlgebricksException {
+        // if already rewritten, the required type is not null
+        if (TypeComputerUtilities.getRequiredType(funcExpr) != null)
+            return false;
+        TypeComputerUtilities.setRequiredAndInputTypes(funcExpr, requiredRecordType, inputRecordType);
+        staticRecordTypeCast(funcExpr, requiredRecordType, inputRecordType, env);
+        return true;
+    }
+
+    /**
+     * only called when funcExpr is list constructor
+     * 
+     * @param funcExpr
+     *            list constructor function expression
+     * @param requiredListType
+     *            required list type
+     * @param inputListType
+     * @param env
+     *            type environment
+     * @throws AlgebricksException
+     */
+    private static boolean rewriteListFuncExpr(AbstractFunctionCallExpression funcExpr,
+            AbstractCollectionType requiredListType, AbstractCollectionType inputListType, IVariableTypeEnvironment env)
+            throws AlgebricksException {
+        if (TypeComputerUtilities.getRequiredType(funcExpr) != null)
+            return false;
+
+        TypeComputerUtilities.setRequiredAndInputTypes(funcExpr, requiredListType, inputListType);
+        List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
+
+        IAType itemType = requiredListType.getItemType();
+        IAType inputItemType = inputListType.getItemType();
+        for (int j = 0; j < args.size(); j++) {
+            ILogicalExpression arg = args.get(j).getValue();
+            if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
+                IAType currentItemType = (IAType) env.getType(argFunc);
+                if (inputItemType == null || inputItemType == BuiltinType.ANY) {
+                    currentItemType = (IAType) env.getType(argFunc);
+                    rewriteFuncExpr(argFunc, itemType, currentItemType, env);
+                } else {
+                    rewriteFuncExpr(argFunc, itemType, inputItemType, env);
+                }
+            }
+        }
+        return true;
+    }
+
+    /**
+     * This method statically cast the type of records from their current type to the required type.
+     * 
+     * @param func
+     *            The record constructor expression.
+     * @param reqType
+     *            The required type.
+     * @param inputType
+     *            The current type.
+     * @param env
+     *            The type environment.
+     * @throws AlgebricksException
+     */
+    private static void staticRecordTypeCast(AbstractFunctionCallExpression func, ARecordType reqType,
+            ARecordType inputType, IVariableTypeEnvironment env) throws AlgebricksException {
+        IAType[] reqFieldTypes = reqType.getFieldTypes();
+        String[] reqFieldNames = reqType.getFieldNames();
+        IAType[] inputFieldTypes = inputType.getFieldTypes();
+        String[] inputFieldNames = inputType.getFieldNames();
+
+        int[] fieldPermutation = new int[reqFieldTypes.length];
+        boolean[] nullFields = new boolean[reqFieldTypes.length];
+        boolean[] openFields = new boolean[inputFieldTypes.length];
+
+        Arrays.fill(nullFields, false);
+        Arrays.fill(openFields, true);
+        Arrays.fill(fieldPermutation, -1);
+
+        // forward match: match from actual to required
+        boolean matched = false;
+        for (int i = 0; i < inputFieldNames.length; i++) {
+            String fieldName = inputFieldNames[i];
+            IAType fieldType = inputFieldTypes[i];
+
+            if (2 * i + 1 > func.getArguments().size())
+                throw new AlgebricksException("expression index out of bound");
+
+            // 2*i+1 is the index of field value expression
+            ILogicalExpression arg = func.getArguments().get(2 * i + 1).getValue();
+            matched = false;
+            for (int j = 0; j < reqFieldNames.length; j++) {
+                String reqFieldName = reqFieldNames[j];
+                IAType reqFieldType = reqFieldTypes[j];
+                if (fieldName.equals(reqFieldName)) {
+                    if (fieldType.equals(reqFieldType)) {
+                        fieldPermutation[j] = i;
+                        openFields[i] = false;
+                        matched = true;
+
+                        if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                            ScalarFunctionCallExpression scalarFunc = (ScalarFunctionCallExpression) arg;
+                            rewriteFuncExpr(scalarFunc, reqFieldType, fieldType, env);
+                        }
+                        break;
+                    }
+
+                    // match the optional field
+                    if (reqFieldType.getTypeTag() == ATypeTag.UNION
+                            && NonTaggedFormatUtil.isOptionalField((AUnionType) reqFieldType)) {
+                        IAType itemType = ((AUnionType) reqFieldType).getUnionList().get(
+                                NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+                        reqFieldType = itemType;
+                        if (fieldType.equals(BuiltinType.ANULL) || fieldType.equals(itemType)) {
+                            fieldPermutation[j] = i;
+                            openFields[i] = false;
+                            matched = true;
+
+                            // rewrite record expr
+                            if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                                ScalarFunctionCallExpression scalarFunc = (ScalarFunctionCallExpression) arg;
+                                rewriteFuncExpr(scalarFunc, reqFieldType, fieldType, env);
+                            }
+                            break;
+                        }
+                    }
+
+                    // match the record field: need cast
+                    if (arg.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        ScalarFunctionCallExpression scalarFunc = (ScalarFunctionCallExpression) arg;
+                        rewriteFuncExpr(scalarFunc, reqFieldType, fieldType, env);
+                        fieldPermutation[j] = i;
+                        openFields[i] = false;
+                        matched = true;
+                        break;
+                    }
+                }
+            }
+            // the input has extra fields
+            if (!matched && !reqType.isOpen())
+                throw new AlgebricksException("static type mismatch: including an extra closed field " + fieldName);
+        }
+
+        // backward match: match from required to actual
+        for (int i = 0; i < reqFieldNames.length; i++) {
+            String reqFieldName = reqFieldNames[i];
+            IAType reqFieldType = reqFieldTypes[i];
+            matched = false;
+            for (int j = 0; j < inputFieldNames.length; j++) {
+                String fieldName = inputFieldNames[j];
+                IAType fieldType = inputFieldTypes[j];
+                if (!fieldName.equals(reqFieldName))
+                    continue;
+                // should check open field here
+                // because number of entries in fieldPermuations is the
+                // number of required schema fields
+                // here we want to check if an input field is matched
+                // the entry index of fieldPermuatons is req field index
+                if (!openFields[j]) {
+                    matched = true;
+                    break;
+                }
+
+                // match the optional field
+                if (reqFieldType.getTypeTag() == ATypeTag.UNION
+                        && NonTaggedFormatUtil.isOptionalField((AUnionType) reqFieldType)) {
+                    IAType itemType = ((AUnionType) reqFieldType).getUnionList().get(
+                            NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+                    if (fieldType.equals(BuiltinType.ANULL) || fieldType.equals(itemType)) {
+                        matched = true;
+                        break;
+                    }
+                }
+            }
+            if (matched)
+                continue;
+
+            if (reqFieldType.getTypeTag() == ATypeTag.UNION
+                    && NonTaggedFormatUtil.isOptionalField((AUnionType) reqFieldType)) {
+                // add a null field
+                nullFields[i] = true;
+            } else {
+                // no matched field in the input for a required closed field
+                throw new AlgebricksException("static type mismatch: miss a required closed field " + reqFieldName);
+            }
+        }
+
+        List<Mutable<ILogicalExpression>> arguments = func.getArguments();
+        List<Mutable<ILogicalExpression>> originalArguments = new ArrayList<Mutable<ILogicalExpression>>();
+        originalArguments.addAll(arguments);
+        arguments.clear();
+        // re-order the closed part and fill in null fields
+        for (int i = 0; i < fieldPermutation.length; i++) {
+            int pos = fieldPermutation[i];
+            if (pos >= 0) {
+                arguments.add(originalArguments.get(2 * pos));
+                arguments.add(originalArguments.get(2 * pos + 1));
+            }
+            if (nullFields[i]) {
+                // add a null field
+                arguments.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                        new AString(reqFieldNames[i])))));
+                arguments.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                        ANull.NULL))));
+            }
+        }
+
+        // add the open part
+        for (int i = 0; i < openFields.length; i++) {
+            if (openFields[i]) {
+                arguments.add(originalArguments.get(2 * i));
+                Mutable<ILogicalExpression> fExprRef = originalArguments.get(2 * i + 1);
+                ILogicalExpression argExpr = fExprRef.getValue();
+
+                // we need to handle open fields recursively by their default
+                // types
+                // for list, their item type is any
+                // for record, their
+                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                    IAType reqFieldType = inputFieldTypes[i];
+                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.RECORD) {
+                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+                    }
+                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.ORDEREDLIST) {
+                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+                    }
+                    if (inputFieldTypes[i].getTypeTag() == ATypeTag.UNORDEREDLIST) {
+                        reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+                    }
+                    if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null) {
+                        ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+                        rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
+                    }
+                }
+                arguments.add(fExprRef);
+            }
+        }
+    }
+
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
index 5f31bd1..9163e35 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/AbstractAqlTranslator.java
@@ -1,90 +1,103 @@
+/*
+ * Copyright 2009-2012 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.translator;
 
-import java.io.File;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.expression.DataverseDecl;
-import edu.uci.ics.asterix.aql.expression.SetStatement;
-import edu.uci.ics.asterix.aql.expression.TypeDecl;
-import edu.uci.ics.asterix.aql.expression.WriteStatement;
-import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.aql.expression.DataverseDropStatement;
+import edu.uci.ics.asterix.aql.expression.DeleteStatement;
+import edu.uci.ics.asterix.aql.expression.DropStatement;
+import edu.uci.ics.asterix.aql.expression.InsertStatement;
+import edu.uci.ics.asterix.aql.expression.NodeGroupDropStatement;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
-import edu.uci.ics.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
-import edu.uci.ics.hyracks.api.io.FileReference;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
+import edu.uci.ics.asterix.metadata.entities.AsterixBuiltinTypeMap;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.asterix.om.types.BuiltinType;
 
+/**
+ * Base class for AQL translators. 
+ * Contains the common validation logic for AQL statements.
+ */
 public abstract class AbstractAqlTranslator {
 
-    public AqlCompiledMetadataDeclarations compileMetadata(MetadataTransactionContext mdTxnCtx,
-            List<Statement> statements, boolean online) throws AlgebricksException, MetadataException {
-        List<TypeDecl> typeDeclarations = new ArrayList<TypeDecl>();
-        Map<String, String> config = new HashMap<String, String>();
+    protected static final Map<String, BuiltinType> builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
 
-        FileSplit outputFile = null;
-        IAWriterFactory writerFactory = null;
-        String dataverseName = MetadataConstants.METADATA_DATAVERSE_NAME;
-        for (Statement stmt : statements) {
-            switch (stmt.getKind()) {
-                case TYPE_DECL: {
-                    typeDeclarations.add((TypeDecl) stmt);
-                    break;
+    public void validateOperation(Dataverse defaultDataverse, Statement stmt) throws AsterixException {
+        boolean invalidOperation = false;
+        String message = null;
+        String dataverse = defaultDataverse != null ? defaultDataverse.getDataverseName() : null;
+        switch (stmt.getKind()) {
+            case INSERT:
+                InsertStatement insertStmt = (InsertStatement) stmt;
+                if (insertStmt.getDataverseName() != null) {
+                    dataverse = insertStmt.getDataverseName().getValue();
                 }
-                case DATAVERSE_DECL: {
-                    DataverseDecl dstmt = (DataverseDecl) stmt;
-                    dataverseName = dstmt.getDataverseName().toString();
-                    break;
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverse);
+                if (invalidOperation) {
+                    message = "Insert operation is not permitted in dataverse "
+                            + MetadataConstants.METADATA_DATAVERSE_NAME;
                 }
-                case WRITE: {
-                    if (outputFile != null) {
-                        throw new AlgebricksException("Multiple 'write' statements.");
-                    }
-                    WriteStatement ws = (WriteStatement) stmt;
-                    File f = new File(ws.getFileName());
-                    outputFile = new FileSplit(ws.getNcName().getValue(), new FileReference(f));
-                    if (ws.getWriterClassName() != null) {
-                        try {
-                            writerFactory = (IAWriterFactory) Class.forName(ws.getWriterClassName()).newInstance();
-                        } catch (Exception e) {
-                            throw new AlgebricksException(e);
-                        }
-                    }
-                    break;
+                break;
+
+            case DELETE:
+                DeleteStatement deleteStmt = (DeleteStatement) stmt;
+                if (deleteStmt.getDataverseName() != null) {
+                    dataverse = deleteStmt.getDataverseName().getValue();
                 }
-                case SET: {
-                    SetStatement ss = (SetStatement) stmt;
-                    String pname = ss.getPropName();
-                    String pvalue = ss.getPropValue();
-                    config.put(pname, pvalue);
-                    break;
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverse);
+                if (invalidOperation) {
+                    message = "Delete operation is not permitted in dataverse "
+                            + MetadataConstants.METADATA_DATAVERSE_NAME;
                 }
-            }
-        }
-        if (writerFactory == null) {
-            writerFactory = PrinterBasedWriterFactory.INSTANCE;
+                break;
+
+            case NODEGROUP_DROP:
+                String nodegroupName = ((NodeGroupDropStatement) stmt).getNodeGroupName().getValue();
+                invalidOperation = MetadataConstants.METADATA_DEFAULT_NODEGROUP_NAME.equals(nodegroupName);
+                if (invalidOperation) {
+                    message = "Cannot drop nodegroup:" + nodegroupName;
+                }
+                break;
+
+            case DATAVERSE_DROP:
+                DataverseDropStatement dvDropStmt = (DataverseDropStatement) stmt;
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dvDropStmt.getDataverseName()
+                        .getValue());
+                if (invalidOperation) {
+                    message = "Cannot drop dataverse:" + dvDropStmt.getDataverseName().getValue();
+                }
+                break;
+
+            case DATASET_DROP:
+                DropStatement dropStmt = (DropStatement) stmt;
+                if (dropStmt.getDataverseName() != null) {
+                    dataverse = dropStmt.getDataverseName().getValue();
+                }
+                invalidOperation = MetadataConstants.METADATA_DATAVERSE_NAME.equals(dataverse);
+                if (invalidOperation) {
+                    message = "Cannot drop a dataset belonging to the dataverse:"
+                            + MetadataConstants.METADATA_DATAVERSE_NAME;
+                }
+                break;
+
         }
 
-        MetadataDeclTranslator metadataTranslator = new MetadataDeclTranslator(mdTxnCtx, dataverseName, outputFile,
-                writerFactory, config, typeDeclarations);
-        return metadataTranslator.computeMetadataDeclarations(online);
-    }
-
-    public void validateOperation(AqlCompiledMetadataDeclarations compiledDeclarations, Statement stmt)
-            throws AlgebricksException {
-        if (compiledDeclarations.getDataverseName() != null
-                && compiledDeclarations.getDataverseName().equals(MetadataConstants.METADATA_DATAVERSE_NAME)) {
-            if (stmt.getKind() == Statement.Kind.INSERT || stmt.getKind() == Statement.Kind.UPDATE
-                    || stmt.getKind() == Statement.Kind.DELETE) {
-                throw new AlgebricksException(" Operation  " + stmt.getKind() + " not permitted in system dataverse-"
-                        + MetadataConstants.METADATA_DATAVERSE_NAME);
-            }
+        if (invalidOperation) {
+            throw new AsterixException("Invalid operation - " + message);
         }
     }
-}
\ No newline at end of file
+}
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 35fb3ae..fdf9c0b 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
@@ -12,7 +12,6 @@
 import edu.uci.ics.asterix.aql.base.Clause;
 import edu.uci.ics.asterix.aql.base.Expression;
 import edu.uci.ics.asterix.aql.base.Expression.Kind;
-import edu.uci.ics.asterix.aql.base.Statement;
 import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
 import edu.uci.ics.asterix.aql.expression.CallExpr;
 import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
@@ -76,27 +75,28 @@
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
 import edu.uci.ics.asterix.metadata.MetadataException;
-import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
-import edu.uci.ics.asterix.metadata.declared.AqlLogicalPlanAndMetadataImpl;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 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.entities.Function;
 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.functions.AsterixFunction;
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 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;
@@ -105,7 +105,6 @@
 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.ILogicalPlan;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
@@ -115,6 +114,7 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.BroadcastExpressionAnnotation.BroadcastSide;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
@@ -154,1507 +154,1297 @@
  * source for the current subtree.
  */
 
-public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator
-		implements
-		IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlExpressionToPlanTranslator extends AbstractAqlTranslator implements
+        IAqlExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
 
-	private final MetadataTransactionContext mdTxnCtx;
-	private final long txnId;
-	private TranslationContext context;
-	private String outputDatasetName;
-	private Statement.Kind dmlKind;
-	private static AtomicLong outputFileID = new AtomicLong(0);
-	private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
+    private final AqlMetadataProvider metadataProvider;
+    private final TranslationContext context;
+    private final String outputDatasetName;
+    private final ICompiledDmlStatement stmt;
+    private static AtomicLong outputFileID = new AtomicLong(0);
+    private static final String OUTPUT_FILE_PREFIX = "OUTPUT_";
 
-	private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+    private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-	public AqlExpressionToPlanTranslator(long txnId,
-			MetadataTransactionContext mdTxnCtx, int currentVarCounter,
-			String outputDatasetName, Statement.Kind dmlKind) {
-		this.mdTxnCtx = mdTxnCtx;
-		this.txnId = txnId;
-		this.context = new TranslationContext(new Counter(currentVarCounter));
-		this.outputDatasetName = outputDatasetName;
-		this.dmlKind = dmlKind;
-	}
+    public AqlExpressionToPlanTranslator(AqlMetadataProvider metadataProvider, int currentVarCounter,
+            String outputDatasetName, ICompiledDmlStatement stmt) {
+        this.context = new TranslationContext(new Counter(currentVarCounter));
+        this.outputDatasetName = outputDatasetName;
+        this.stmt = stmt;
+        this.metadataProvider = metadataProvider;
+    }
 
-	public int getVarCounter() {
-		return context.getVarCounter();
-	}
+    public int getVarCounter() {
+        return context.getVarCounter();
+    }
 
-	public ILogicalPlanAndMetadata translate(Query expr,
-			AqlCompiledMetadataDeclarations compiledDeclarations)
-			throws AlgebricksException, AsterixException {
-		if (expr == null) {
-			return null;
-		}
-		if (compiledDeclarations == null) {
-			compiledDeclarations = compileMetadata(mdTxnCtx,
-					expr.getPrologDeclList(), true);
-		}
-		if (!compiledDeclarations.isConnectedToDataverse())
-			compiledDeclarations.connectToDataverse(compiledDeclarations
-					.getDataverseName());
-		IDataFormat format = compiledDeclarations.getFormat();
-		if (format == null) {
-			throw new AlgebricksException("Data format has not been set.");
-		}
-		format.registerRuntimeFunctions();
-		Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-				new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator()));
+    public ILogicalPlan translate(Query expr) throws AlgebricksException, AsterixException {
+        IDataFormat format = FormatUtils.getDefaultFormat();
+        format.registerRuntimeFunctions();
 
-		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+        Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
+                new EmptyTupleSourceOperator()));
 
-		boolean isTransactionalWrite = false;
-		ILogicalOperator topOp = p.first;
-		ProjectOperator project = (ProjectOperator) topOp;
-		LogicalVariable resVar = project.getVariables().get(0);
-		if (outputDatasetName == null) {
-			FileSplit outputFileSplit = compiledDeclarations.getOutputFile();
-			if (outputFileSplit == null) {
-				outputFileSplit = getDefaultOutputFileLocation();
-			}
-			compiledDeclarations.setOutputFile(outputFileSplit);
-			List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			writeExprList.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(resVar)));
-			FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
-			FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
-			topOp = new WriteOperator(writeExprList, sink);
-			topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
-		} else {
-			String dataVerseName = compiledDeclarations.getDataverseName();
-			Dataset dataset = compiledDeclarations
-					.findDataset(outputDatasetName);
-			if (dataset == null) {
-				throw new AlgebricksException("Cannot find dataset "
-						+ outputDatasetName);
-			}
+        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-			AqlSourceId sourceId = new AqlSourceId(dataVerseName,
-					outputDatasetName);
-			String itemTypeName = dataset.getItemTypeName();
-			IAType itemType = compiledDeclarations.findType(itemTypeName);
-			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>>();
+        boolean isTransactionalWrite = false;
+        ILogicalOperator topOp = p.first;
+        ProjectOperator project = (ProjectOperator) topOp;
+        LogicalVariable resVar = project.getVariables().get(0);
+        if (outputDatasetName == null) {
+            FileSplit outputFileSplit = metadataProvider.getOutputFile();
+            if (outputFileSplit == null) {
+                outputFileSplit = getDefaultOutputFileLocation();
+            }
+            metadataProvider.setOutputFile(outputFileSplit);
+            List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+            FileSplitSinkId fssi = new FileSplitSinkId(outputFileSplit);
+            FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+            topOp = new WriteOperator(writeExprList, sink);
+            topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+        } else {
 
-			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)))));
-				f.substituteVar(METADATA_DUMMY_VAR, resVar);
-				exprs.add(new MutableObject<ILogicalExpression>(f));
-				LogicalVariable v = context.newVar();
-				vars.add(v);
-				varRefsForLoading.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(v)));
-			}
-			AssignOperator assign = new AssignOperator(vars, exprs);
-			assign.getInputs()
-					.add(new MutableObject<ILogicalOperator>(project));
+            AqlDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
+                    stmt.getDatasetName());
 
-			Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(resVar));
-			ILogicalOperator load = null;
+            ArrayList<LogicalVariable> vars = new ArrayList<LogicalVariable>();
+            ArrayList<Mutable<ILogicalExpression>> exprs = new ArrayList<Mutable<ILogicalExpression>>();
+            List<Mutable<ILogicalExpression>> varRefsForLoading = new ArrayList<Mutable<ILogicalExpression>>();
 
-			switch (dmlKind) {
-			case WRITE_FROM_QUERY_RESULT: {
-				load = new WriteResultOperator(dataSource, varRef,
-						varRefsForLoading);
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				break;
-			}
-			case INSERT: {
-				ILogicalOperator insertOp = new InsertDeleteOperator(
-						dataSource, varRef, varRefsForLoading,
-						InsertDeleteOperator.Kind.INSERT);
-				insertOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				load = new SinkOperator();
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(insertOp));
-				isTransactionalWrite = true;
-				break;
-			}
-			case DELETE: {
-				ILogicalOperator deleteOp = new InsertDeleteOperator(
-						dataSource, varRef, varRefsForLoading,
-						InsertDeleteOperator.Kind.DELETE);
-				deleteOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				load = new SinkOperator();
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(deleteOp));
-				isTransactionalWrite = true;
-				break;
-			}
-			case BEGIN_FEED: {
-				ILogicalOperator insertOp = new InsertDeleteOperator(
-						dataSource, varRef, varRefsForLoading,
-						InsertDeleteOperator.Kind.INSERT);
-				insertOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(assign));
-				load = new SinkOperator();
-				load.getInputs().add(
-						new MutableObject<ILogicalOperator>(insertOp));
-				isTransactionalWrite = false;
-				break;
-			}
-			}
-			topOp = load;
-		}
+            List<String> partitionKeys = DatasetUtils.getPartitioningKeys(targetDatasource.getDataset());
+            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)))));
+                f.substituteVar(METADATA_DUMMY_VAR, resVar);
+                exprs.add(new MutableObject<ILogicalExpression>(f));
+                LogicalVariable v = context.newVar();
+                vars.add(v);
+                varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            }
+            AssignOperator assign = new AssignOperator(vars, exprs);
+            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
 
-		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-		AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
-				isTransactionalWrite, compiledDeclarations);
-		ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
-				plan, metadataProvider);
-		return planAndMetadata;
-	}
+            Mutable<ILogicalExpression> varRef = new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                    resVar));
+            ILogicalOperator leafOperator = null;
 
-	private FileSplit getDefaultOutputFileLocation() throws MetadataException {
-		if (AsterixProperties.INSTANCE.getOutputDir() == null) {
-			throw new MetadataException(
-					"Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
-		}
-		String filePath = AsterixProperties.INSTANCE.getOutputDir()
-				+ System.getProperty("file.separator") + OUTPUT_FILE_PREFIX
-				+ outputFileID.incrementAndGet();
-		return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(),
-				new FileReference(new File(filePath)));
-	}
+            switch (stmt.getKind()) {
+                case WRITE_FROM_QUERY_RESULT: {
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v = context.newVar(fc.getVarExpr());
+                    leafOperator = new WriteResultOperator(targetDatasource, varRef, varRefsForLoading);
+                    leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    break;
+                }
+                case INSERT: {
+                    ILogicalOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
+                            InsertDeleteOperator.Kind.INSERT);
+                    insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    leafOperator = new SinkOperator();
+                    leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+                    isTransactionalWrite = true;
+                    break;
+                }
+                case DELETE: {
+                    ILogicalOperator deleteOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
+                            InsertDeleteOperator.Kind.DELETE);
+                    deleteOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    leafOperator = new SinkOperator();
+                    leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(deleteOp));
+                    isTransactionalWrite = true;
+                    break;
+                }
+                case BEGIN_FEED: {
+                    ILogicalOperator insertOp = new InsertDeleteOperator(targetDatasource, varRef, varRefsForLoading,
+                            InsertDeleteOperator.Kind.INSERT);
+                    insertOp.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+                    leafOperator = new SinkOperator();
+                    leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+                    isTransactionalWrite = false;
+                    break;
+                }
+            }
+            topOp = leafOperator;
+        }
 
-		Expression inExpr = fc.getInExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				inExpr, tupSource);
-		ILogicalOperator returnedOp;
+        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+        return plan;
+    }
 
-		if (fc.getPosVarExpr() == null) {
-			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.getInputs().add(eo.second);
+    private AqlDataSource validateDatasetInfo(AqlMetadataProvider metadataProvider, String dataverseName,
+            String datasetName) throws AlgebricksException {
+        Dataset dataset = metadataProvider.findDataset(dataverseName, datasetName);
+        if (dataset == null) {
+            throw new AlgebricksException("Cannot find dataset " + datasetName + " in dataverse " + dataverseName);
+        }
 
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+        AqlSourceId sourceId = new AqlSourceId(dataverseName, datasetName);
+        String itemTypeName = dataset.getItemTypeName();
+        IAType itemType = metadataProvider.findType(dataverseName, itemTypeName);
+        AqlDataSource dataSource = new AqlDataSource(sourceId, dataset, itemType);
+        if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+            throw new AlgebricksException("Cannot write output to an external dataset.");
+        }
+        return dataSource;
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v;
-		ILogicalOperator returnedOp;
+    }
 
-		switch (lc.getBindingExpr().getKind()) {
-		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.getInputs().add(tupSource);
-			break;
-		}
-		default: {
-			v = context.newVar(lc.getVarExpr());
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					lc.getBindingExpr(), tupSource);
-			returnedOp = new AssignOperator(v,
-					new MutableObject<ILogicalExpression>(eo.first));
-			returnedOp.getInputs().add(eo.second);
-			break;
-		}
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+    private FileSplit getDefaultOutputFileLocation() throws MetadataException {
+        if (AsterixProperties.INSTANCE.getOutputDir() == null) {
+            throw new MetadataException(
+                    "Output location for query result not specified at the time of deployment, must specify explicitly using 'write output to ..' statement");
+        }
+        String filePath = AsterixProperties.INSTANCE.getOutputDir() + System.getProperty("file.separator")
+                + OUTPUT_FILE_PREFIX + outputFileID.incrementAndGet();
+        return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(), new FileReference(new File(filePath)));
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
-			FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> flworPlan = tupSource;
-		boolean isTop = context.isTopFlwor();
-		if (isTop) {
-			context.setTopFlwor(false);
-		}
-		for (Clause c : flwor.getClauseList()) {
-			Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
-					flworPlan);
-			flworPlan = new MutableObject<ILogicalOperator>(pC.first);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar(fc.getVarExpr());
 
-		Expression r = flwor.getReturnExpr();
-		boolean noFlworClause = flwor.noForClause();
+        Expression inExpr = fc.getInExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
+        ILogicalOperator returnedOp;
 
-		if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
-			VariableExpr v = (VariableExpr) r;
-			LogicalVariable var = context.getVar(v.getVar().getId());
+        if (fc.getPosVarExpr() == null) {
+            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.getInputs().add(eo.second);
 
-			return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-		} else {
-			Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
-					flworPlan.getValue());
-			Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
-					baseOp);
-			ILogicalOperator rOp = rRes.first;
-			ILogicalOperator resOp;
-			if (expressionNeedsNoNesting(r)) {
-				baseOp.setValue(flworPlan.getValue());
-				resOp = rOp;
-			} else {
-				SubplanOperator s = new SubplanOperator(rOp);
-				s.getInputs().add(flworPlan);
-				resOp = s;
-				baseOp.setValue(new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(s)));
-			}
-			Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
-					resOp);
-			return produceFlwrResult(noFlworClause, isTop, resOpRef,
-					rRes.second);
-		}
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v;
+        ILogicalOperator returnedOp;
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
-			FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				fa.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(p.first));
-		ILogicalExpression faExpr = new ConstantExpression(
-				new AsterixConstantValue(new AString(fa.getIdent().getValue())));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(faExpr));
-		AssignOperator a = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(fldAccess));
-		a.getInputs().add(p.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+        switch (lc.getBindingExpr().getKind()) {
+            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.getInputs().add(tupSource);
+                break;
+            }
+            default: {
+                v = context.newVar(lc.getVarExpr());
+                Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
+                        tupSource);
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
+                returnedOp.getInputs().add(eo.second);
+                break;
+            }
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> flworPlan = tupSource;
+        boolean isTop = context.isTopFlwor();
+        if (isTop) {
+            context.setTopFlwor(false);
+        }
+        for (Clause c : flwor.getClauseList()) {
+            Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
+            flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+        }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
-			IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				ia.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression f;
-		int i = ia.getIndex();
-		if (i == IndexAccessor.ANY) {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		} else {
-			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)))));
-		}
-		AssignOperator a = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(f));
-		a.getInputs().add(p.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
-	}
+        Expression r = flwor.getReturnExpr();
+        boolean noFlworClause = flwor.noForClause();
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(
-			CallExpr fcall, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		LogicalVariable v = context.newVar();
-		AsterixFunction fid = fcall.getIdent();
-		List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> topOp = tupSource;
+        if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+            VariableExpr v = (VariableExpr) r;
+            LogicalVariable var = context.getVar(v.getVar().getId());
 
-		for (Expression expr : fcall.getExprList()) {
-			switch (expr.getKind()) {
-			case VARIABLE_EXPRESSION: {
-				LogicalVariable var = context.getVar(((VariableExpr) expr)
-						.getVar().getId());
-				args.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(var)));
-				break;
-			}
-			case LITERAL_EXPRESSION: {
-				LiteralExpr val = (LiteralExpr) expr;
-				args.add(new MutableObject<ILogicalExpression>(
-						new ConstantExpression(
-								new AsterixConstantValue(ConstantHelper
-										.objectFromLiteral(val.getValue())))));
-				break;
-			}
-			default: {
-				Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-						expr, topOp);
-				AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
-						.getValue();
-				args.add(new MutableObject<ILogicalExpression>(eo.first));
-				if (o1 != null
-						&& !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
-								o1, topOp))) {
-					topOp = eo.second;
-				}
-				break;
-			}
-			}
-		}
+            return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
 
-		FunctionIdentifier fi = new FunctionIdentifier(
-				AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
-		AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
-		FunctionIdentifier builtinAquafi = afi == null ? null : afi
-				.getFunctionIdentifier();
+        } else {
+            Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
+            Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
+            ILogicalOperator rOp = rRes.first;
+            ILogicalOperator resOp;
+            if (expressionNeedsNoNesting(r)) {
+                baseOp.setValue(flworPlan.getValue());
+                resOp = rOp;
+            } else {
+                SubplanOperator s = new SubplanOperator(rOp);
+                s.getInputs().add(flworPlan);
+                resOp = s;
+                baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+            }
+            Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
+            return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
+        }
+    }
 
-		if (builtinAquafi != null) {
-			fi = builtinAquafi;
-		} else {
-			fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-					fid.getFunctionName());
-			FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
-					.getBuiltinFunctionIdentifier(fi);
-			if (builtinAsterixFi != null) {
-				fi = builtinAsterixFi;
-			}
-		}
-		AbstractFunctionCallExpression f;
-		if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
-			f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
-					args);
-		} else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
-			UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-			ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
-					.returnsUniqueValues(fi));
-			f = ufce;
-		} else {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-		}
-		AssignOperator op = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(f));
-		if (topOp != null) {
-			op.getInputs().add(topOp);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
+                .getValue())));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
+        a.getInputs().add(p.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(op, v);
-	}
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
-			FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
-		// TODO Auto-generated method stub
-		throw new NotImplementedException();
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(IndexAccessor ia,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression f;
+        int i = ia.getIndex();
+        if (i == IndexAccessor.ANY) {
+            f = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        } else {
+            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)))));
+        }
+        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> visitGroupbyClause(
-			GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		GroupByOperator gOp = new GroupByOperator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addGbyExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addDecorExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		gOp.getInputs().add(topOp);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar();
+        FunctionSignature signature = fcall.getFunctionSignature();
+        List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		for (VariableExpr var : gc.getWithVarList()) {
-			LogicalVariable aggVar = context.newVar();
-			LogicalVariable oldVar = context.getVar(var);
-			List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			flArgs.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(oldVar)));
-			AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
-					.makeAggregateFunctionExpression(
-							AsterixBuiltinFunctions.LISTIFY, flArgs);
-			AggregateOperator agg = new AggregateOperator(
-					mkSingletonArrayList(aggVar),
-					(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-							fListify)));
+        for (Expression expr : fcall.getExprList()) {
+            switch (expr.getKind()) {
+                case VARIABLE_EXPRESSION: {
+                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
+                    args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    break;
+                }
+                case LITERAL_EXPRESSION: {
+                    LiteralExpr val = (LiteralExpr) expr;
+                    args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                            ConstantHelper.objectFromLiteral(val.getValue())))));
+                    break;
+                }
+                default: {
+                    Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+                    AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
+                    args.add(new MutableObject<ILogicalExpression>(eo.first));
+                    if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
+                        topOp = eo.second;
+                    }
+                    break;
+                }
+            }
+        }
 
-			agg.getInputs().add(
-					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
-			// the one bound by the aggregation op.
-			context.setVar(var, aggVar);
-		}
+        AbstractFunctionCallExpression f;
+        if ((f = lookupUserDefinedFunction(signature, args)) == null) {
+            f = lookupBuiltinFunction(signature.getName(), signature.getArity(), args);
+        }
 
-		gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
-				gc.hasHashGroupByHint());
-		return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
-	}
+        if (f == null) {
+            throw new AsterixException(" Unknown function " + signature.getName() + "@" + signature.getArity());
+        }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		// In the most general case, IfThenElse is translated in the following
-		// way.
-		//
-		// We assign the result of the condition to one variable varCond.
-		// We create one subplan which contains the plan for the "then" branch,
-		// on top of which there is a selection whose condition is varCond.
-		// Similarly, we create one subplan for the "else" branch, in which the
-		// selection is not(varCond).
-		// Finally, we concatenate the results. (??)
+        // Put hints into function call expr.
+        if (fcall.hasHints()) {
+            for (IExpressionAnnotation hint : fcall.getHints()) {
+                f.getAnnotations().put(hint, hint);
+            }
+        }
 
-		Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
-				.accept(this, tupSource);
-		ILogicalOperator opCond = pCond.first;
-		LogicalVariable varCond = pCond.second;
+        AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+        if (topOp != null) {
+            op.getInputs().add(topOp);
+        }
 
-		SubplanOperator sp = new SubplanOperator();
-		Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
-				new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(sp)));
+        return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+    }
 
-		Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
-				.accept(this, nestedSource);
-		SelectOperator sel1 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
+    private AbstractFunctionCallExpression lookupUserDefinedFunction(FunctionSignature signature,
+            List<Mutable<ILogicalExpression>> args) throws MetadataException {
+        if (signature.getNamespace() == null) {
+            return null;
+        }
+        Function function = MetadataManager.INSTANCE.getFunction(metadataProvider.getMetadataTxnContext(), signature);
+        if (function == null) {
+            return null;
+        }
+        AbstractFunctionCallExpression f = null;
+        if (function.getLanguage().equalsIgnoreCase(Function.LANGUAGE_AQL)) {
+            IFunctionInfo finfo = new AsterixFunctionInfo(signature);
+            return new ScalarFunctionCallExpression(finfo, args);
+        } else {
+            throw new MetadataException(" User defined functions written in " + function.getLanguage()
+                    + " are not supported");
+        }
+    }
 
-		Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr()
-				.accept(this, nestedSource);
-		AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		SelectOperator sel2 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(notVarCond));
-		sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+    private AbstractFunctionCallExpression lookupBuiltinFunction(String functionName, int arity,
+            List<Mutable<ILogicalExpression>> args) {
+        AbstractFunctionCallExpression f = null;
 
-		ILogicalPlan p1 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel1));
-		sp.getNestedPlans().add(p1);
-		ILogicalPlan p2 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel2));
-		sp.getNestedPlans().add(p2);
+        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, functionName, arity);
+        AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+        FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
 
-		Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
-				opCond);
-		sp.getInputs().add(opCondRef);
+        if (builtinAquafi != null) {
+            fi = builtinAquafi;
+        } else {
+            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, functionName, arity);
+            afi = AsterixBuiltinFunctions.lookupFunction(fi);
+            if (afi == null) {
+                return null;
+            }
+        }
+        if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+            f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
+        } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+            UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(fi), args);
+            ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
+            f = ufce;
+        } else {
+            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
+        }
+        return f;
+    }
 
-		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)));
-		AssignOperator a = new AssignOperator(resV,
-				new MutableObject<ILogicalExpression>(concatNonNull));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+            Mutable<ILogicalOperator> tupSource) {
+        // TODO Auto-generated method stub
+        throw new NotImplementedException();
+    }
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        GroupByOperator gOp = new GroupByOperator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addGbyExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addDecorExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        gOp.getInputs().add(topOp);
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
-			LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
-		LogicalVariable var = context.newVar();
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(new ConstantExpression(
-						new AsterixConstantValue(ConstantHelper
-								.objectFromLiteral(l.getValue())))));
-		if (tupSource != null) {
-			a.getInputs().add(tupSource);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+        for (VariableExpr var : gc.getWithVarList()) {
+            LogicalVariable aggVar = context.newVar();
+            LogicalVariable oldVar = context.getVar(var);
+            List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
+            flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
+            AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                    AsterixBuiltinFunctions.LISTIFY, flArgs);
+            AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
+                    (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+            agg.getInputs().add(
+                    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
+            // the one bound by the aggregation op.
+            context.setVar(var, aggVar);
+        }
 
-		if (nOps > 0
-				&& (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
-			return visitAndOrOperator(op, tupSource);
-		}
+        gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+        return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+    }
 
-		ArrayList<Expression> exprs = op.getExprList();
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        // In the most general case, IfThenElse is translated in the following
+        // way.
+        //
+        // We assign the result of the condition to one variable varCond.
+        // We create one subplan which contains the plan for the "then" branch,
+        // on top of which there is a selection whose condition is varCond.
+        // Similarly, we create one subplan for the "else" branch, in which the
+        // selection is not(varCond).
+        // Finally, we concatenate the results. (??)
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+        ILogicalOperator opCond = pCond.first;
+        LogicalVariable varCond = pCond.second;
 
-		ILogicalExpression currExpr = null;
-		for (int i = 0; i <= nOps; i++) {
+        SubplanOperator sp = new SubplanOperator();
+        Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                new MutableObject<ILogicalOperator>(sp)));
 
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			ILogicalExpression e = p.first;
-			// now look at the operator
-			if (i < nOps) {
-				if (OperatorExpr.opIsComparison(ops.get(i))) {
-					AbstractFunctionCallExpression c = createComparisonExpression(ops
-							.get(i));
+        Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
+        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                varCond)));
+        sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
-					// chain the operators
-					if (i == 0) {
-						c.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = c;
-						if (op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.LEFT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						c.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = c;
-						if (i == 1 && op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.RIGHT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					}
-				} else {
-					AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
-							.get(i));
+        Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
+        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(varCond)));
+        SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
+        sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
 
-					if (i == 0) {
-						f.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = f;
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						f.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = f;
-					}
-				}
-			} else { // don't forget the last expression...
-				((AbstractFunctionCallExpression) currExpr).getArguments().add(
-						new MutableObject<ILogicalExpression>(e));
-				if (i == 1 && op.isBroadcastOperand(i)) {
-					BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-					bcast.setObject(BroadcastSide.RIGHT);
-					((AbstractFunctionCallExpression) currExpr)
-							.getAnnotations()
-							.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-									bcast);
-				}
-			}
-		}
+        ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
+        sp.getNestedPlans().add(p1);
+        ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
+        sp.getNestedPlans().add(p2);
 
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(currExpr));
+        Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
+        sp.getInputs().add(opCondRef);
 
-		a.getInputs().add(topOp);
+        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)));
+        AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
-			OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+        LogicalVariable var = context.newVar();
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
+                new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+        if (tupSource != null) {
+            a.getInputs().add(tupSource);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-		OrderOperator ord = new OrderOperator();
-		Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression e : oc.getOrderbyList()) {
-			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)));
-			topOp = p.second;
-		}
-		ord.getInputs().add(topOp);
-		if (oc.getNumTuples() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
-					oc.getNumTuples());
-		}
-		if (oc.getNumFrames() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
-					oc.getNumFrames());
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
-			QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> topOp = tupSource;
+        if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+            return visitAndOrOperator(op, tupSource);
+        }
 
-		ILogicalOperator firstOp = null;
-		Mutable<ILogicalOperator> lastOp = null;
+        ArrayList<Expression> exprs = op.getExprList();
 
-		for (QuantifiedPair qt : qe.getQuantifiedList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					qt.getExpr(), topOp);
-			topOp = eo1.second;
-			LogicalVariable uVar = context.newVar(qt.getVarExpr());
-			ILogicalOperator u = new UnnestOperator(uVar,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo1.first)));
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-			if (firstOp == null) {
-				firstOp = u;
-			}
-			if (lastOp != null) {
-				u.getInputs().add(lastOp);
-			}
-			lastOp = new MutableObject<ILogicalOperator>(u);
-		}
+        ILogicalExpression currExpr = null;
+        for (int i = 0; i <= nOps; i++) {
 
-		// We make all the unnest correspond. to quantif. vars. sit on top
-		// in the hope of enabling joins & other optimiz.
-		firstOp.getInputs().add(topOp);
-		topOp = lastOp;
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            ILogicalExpression e = p.first;
+            // now look at the operator
+            if (i < nOps) {
+                if (OperatorExpr.opIsComparison(ops.get(i))) {
+                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
 
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-				qe.getSatisfiesExpr(), topOp);
+                    // chain the operators
+                    if (i == 0) {
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = c;
+                        if (op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.LEFT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = c;
+                        if (i == 1 && op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.RIGHT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    }
+                } else {
+                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
+
+                    if (i == 0) {
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = f;
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = f;
+                    }
+                }
+            } else { // don't forget the last expression...
+                ((AbstractFunctionCallExpression) currExpr).getArguments()
+                        .add(new MutableObject<ILogicalExpression>(e));
+                if (i == 1 && op.isBroadcastOperand(i)) {
+                    BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                    bcast.setObject(BroadcastSide.RIGHT);
+                    ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
+                            BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                }
+            }
+        }
+
+        // Add hints as annotations.
+        if (op.hasHints() && currExpr instanceof AbstractFunctionCallExpression) {
+            AbstractFunctionCallExpression currFuncExpr = (AbstractFunctionCallExpression) currExpr;
+            for (IExpressionAnnotation hint : op.getHints()) {
+                currFuncExpr.getAnnotations().put(hint, hint);
+            }
+        }
+
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
+
+        a.getInputs().add(topOp);
+
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+
+        OrderOperator ord = new OrderOperator();
+        Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression e : oc.getOrderbyList()) {
+            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)));
+            topOp = p.second;
+        }
+        ord.getInputs().add(topOp);
+        if (oc.getNumTuples() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
+        }
+        if (oc.getNumFrames() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> topOp = tupSource;
+
+        ILogicalOperator firstOp = null;
+        Mutable<ILogicalOperator> lastOp = null;
+
+        for (QuantifiedPair qt : qe.getQuantifiedList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
+            topOp = eo1.second;
+            LogicalVariable uVar = context.newVar(qt.getVarExpr());
+            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
+                    makeUnnestExpression(eo1.first)));
+
+            if (firstOp == null) {
+                firstOp = u;
+            }
+            if (lastOp != null) {
+                u.getInputs().add(lastOp);
+            }
+            lastOp = new MutableObject<ILogicalOperator>(u);
+        }
+
+        // We make all the unnest correspond. to quantif. vars. sit on top
+        // in the hope of enabling joins & other optimiz.
+        firstOp.getInputs().add(topOp);
+        topOp = lastOp;
+
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
 
-		AggregateFunctionCallExpression fAgg;
-		SelectOperator s;
-		if (qe.getQuantifier() == Quantifier.SOME) {
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					eo2.first));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.NON_EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		} else { // EVERY
-			List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					new ScalarFunctionCallExpression(FunctionUtils
-							.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-							satExprList)));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		}
-		LogicalVariable qeVar = context.newVar();
-		AggregateOperator a = new AggregateOperator(
-				mkSingletonArrayList(qeVar),
-				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-						fAgg)));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(s));
-		return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
-	}
+        AggregateFunctionCallExpression fAgg;
+        SelectOperator s;
+        if (qe.getQuantifier() == Quantifier.SOME) {
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        } else { // EVERY
+            List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        }
+        LogicalVariable qeVar = context.newVar();
+        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+        return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		return q.getBody().accept(this, tupSource);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        return q.getBody().accept(this, tupSource);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
-			RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (FieldBinding fb : rc.getFbList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					fb.getLeftExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo1.first));
-			topOp = eo1.second;
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-					fb.getRightExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo2.first));
-			topOp = eo2.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (FieldBinding fb : rc.getFbList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
+            topOp = eo1.second;
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
+            topOp = eo2.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
-			ListConstructor lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
-				: AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression expr : lc.getExprList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					expr, topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			topOp = eo.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+                : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression expr : lc.getExprList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            topOp = eo.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Expression expr = u.getExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				expr, tupSource);
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a;
-		if (u.getSign() == Sign.POSITIVE) {
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
-					eo.first));
-		} else {
-			AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
-			m.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
-		}
-		a.getInputs().add(eo.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Expression expr = u.getExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a;
+        if (u.getSign() == Sign.POSITIVE) {
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
+        } else {
+            AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+            m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+        }
+        a.getInputs().add(eo.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
-			VariableExpr v, Mutable<ILogicalOperator> tupSource) {
-		// 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)));
-		a.getInputs().add(tupSource);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+        // 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)));
+        a.getInputs().add(tupSource);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
-			WhereClause w, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				w.getWhereExpr(), tupSource);
-		SelectOperator s = new SelectOperator(
-				new MutableObject<ILogicalExpression>(p.first));
-		s.getInputs().add(p.second);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
+        SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
+        s.getInputs().add(p.second);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(s, null);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
-			LimitClause lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getLimitExpr(), tupSource);
-		LimitOperator opLim;
-		Expression offset = lc.getOffset();
-		if (offset != null) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
-					offset, p1.second);
-			opLim = new LimitOperator(p1.first, p2.first);
-			opLim.getInputs().add(p2.second);
-		} else {
-			opLim = new LimitOperator(p1.first);
-			opLim.getInputs().add(p1.second);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        LimitOperator opLim;
+        Expression offset = lc.getOffset();
+        if (offset != null) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
+            opLim = new LimitOperator(p1.first, p2.first);
+            opLim.getInputs().add(p2.second);
+        } else {
+            opLim = new LimitOperator(p1.first);
+            opLim.getInputs().add(p1.second);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getDieExpr(), tupSource);
-		DieOperator opDie = new DieOperator(p1.first);
-		opDie.getInputs().add(p1.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
+        DieOperator opDie = new DieOperator(p1.first);
+        opDie.getInputs().add(p1.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
-			DistinctClause dc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> input = null;
-		for (Expression expr : dc.getDistinctByExpr()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					expr, tupSource);
-			exprList.add(new MutableObject<ILogicalExpression>(p.first));
-			input = p.second;
-		}
-		DistinctOperator opDistinct = new DistinctOperator(exprList);
-		opDistinct.getInputs().add(input);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> input = null;
+        for (Expression expr : dc.getDistinctByExpr()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
+            exprList.add(new MutableObject<ILogicalExpression>(p.first));
+            input = p.second;
+        }
+        DistinctOperator opDistinct = new DistinctOperator(exprList);
+        opDistinct.getInputs().add(input);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
-			UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> ts = tupSource;
-		ILogicalOperator lastOp = null;
-		LogicalVariable lastVar = null;
-		boolean first = true;
-		for (Expression e : unionExpr.getExprs()) {
-			if (first) {
-				first = false;
-			} else {
-				ts = new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator());
-			}
-			Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
-			if (lastOp == null) {
-				lastOp = p1.first;
-				lastVar = p1.second;
-			} else {
-				LogicalVariable unnestVar1 = context.newVar();
-				UnnestOperator unnest1 = new UnnestOperator(
-						unnestVar1,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										lastVar))));
-				unnest1.getInputs().add(
-						new MutableObject<ILogicalOperator>(lastOp));
-				LogicalVariable unnestVar2 = context.newVar();
-				UnnestOperator unnest2 = new UnnestOperator(
-						unnestVar2,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										p1.second))));
-				unnest2.getInputs().add(
-						new MutableObject<ILogicalOperator>(p1.first));
-				List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
-						1);
-				LogicalVariable resultVar = context.newVar();
-				Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
-						unnestVar1, unnestVar2, resultVar);
-				varMap.add(triple);
-				UnionAllOperator unionOp = new UnionAllOperator(varMap);
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest1));
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest2));
-				lastVar = resultVar;
-				lastOp = unionOp;
-			}
-		}
-		LogicalVariable aggVar = context.newVar();
-		ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
-				1);
-		aggregVars.add(aggVar);
-		List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		afcExprs.add(new MutableObject<ILogicalExpression>(
-				new VariableReferenceExpression(lastVar)));
-		AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY, afcExprs);
-		ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
-		AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
-		agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-		return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> ts = tupSource;
+        ILogicalOperator lastOp = null;
+        LogicalVariable lastVar = null;
+        boolean first = true;
+        for (Expression e : unionExpr.getExprs()) {
+            if (first) {
+                first = false;
+            } else {
+                ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+            }
+            Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+            if (lastOp == null) {
+                lastOp = p1.first;
+                lastVar = p1.second;
+            } else {
+                LogicalVariable unnestVar1 = context.newVar();
+                UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(lastVar))));
+                unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+                LogicalVariable unnestVar2 = context.newVar();
+                UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(p1.second))));
+                unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
+                List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+                        1);
+                LogicalVariable resultVar = context.newVar();
+                Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+                        unnestVar1, unnestVar2, resultVar);
+                varMap.add(triple);
+                UnionAllOperator unionOp = new UnionAllOperator(varMap);
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
+                lastVar = resultVar;
+                lastOp = unionOp;
+            }
+        }
+        LogicalVariable aggVar = context.newVar();
+        ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
+        aggregVars.add(aggVar);
+        List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
+        AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, afcExprs);
+        ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+        AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+        agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+        return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+    }
 
-	private AbstractFunctionCallExpression createComparisonExpression(
-			OperatorType t) {
-		FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-		IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
-		return new ScalarFunctionCallExpression(finfo);
-	}
+    private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
+        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+        return new ScalarFunctionCallExpression(finfo);
+    }
 
-	private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-		switch (t) {
-		case EQ: {
-			return AlgebricksBuiltinFunctions.EQ;
-		}
-		case NEQ: {
-			return AlgebricksBuiltinFunctions.NEQ;
-		}
-		case GT: {
-			return AlgebricksBuiltinFunctions.GT;
-		}
-		case GE: {
-			return AlgebricksBuiltinFunctions.GE;
-		}
-		case LT: {
-			return AlgebricksBuiltinFunctions.LT;
-		}
-		case LE: {
-			return AlgebricksBuiltinFunctions.LE;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-	}
+    private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+        switch (t) {
+            case EQ: {
+                return AlgebricksBuiltinFunctions.EQ;
+            }
+            case NEQ: {
+                return AlgebricksBuiltinFunctions.NEQ;
+            }
+            case GT: {
+                return AlgebricksBuiltinFunctions.GT;
+            }
+            case GE: {
+                return AlgebricksBuiltinFunctions.GE;
+            }
+            case LT: {
+                return AlgebricksBuiltinFunctions.LT;
+            }
+            case LE: {
+                return AlgebricksBuiltinFunctions.LE;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
 
-	private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
-			OperatorType t) throws AsterixException {
+    private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
+            throws AsterixException {
 
-		FunctionIdentifier fid = null;
-		switch (t) {
-		case PLUS: {
-			fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
-			break;
-		}
-		case MINUS: {
-			fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
-			break;
-		}
-		case MUL: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
-			break;
-		}
-		case DIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
-			break;
-		}
-		case MOD: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MOD;
-			break;
-		}
-		case IDIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
-			break;
-		}
-		case CARET: {
-			fid = AsterixBuiltinFunctions.CARET;
-			break;
-		}
-		case AND: {
-			fid = AlgebricksBuiltinFunctions.AND;
-			break;
-		}
-		case OR: {
-			fid = AlgebricksBuiltinFunctions.OR;
-			break;
-		}
-		case FUZZY_EQ: {
-			fid = AsterixBuiltinFunctions.FUZZY_EQ;
-			break;
-		}
+        FunctionIdentifier fid = null;
+        switch (t) {
+            case PLUS: {
+                fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+                break;
+            }
+            case MINUS: {
+                fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+                break;
+            }
+            case MUL: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+                break;
+            }
+            case DIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+                break;
+            }
+            case MOD: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+                break;
+            }
+            case IDIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+                break;
+            }
+            case CARET: {
+                fid = AsterixBuiltinFunctions.CARET;
+                break;
+            }
+            case AND: {
+                fid = AlgebricksBuiltinFunctions.AND;
+                break;
+            }
+            case OR: {
+                fid = AlgebricksBuiltinFunctions.OR;
+                break;
+            }
+            case FUZZY_EQ: {
+                fid = AsterixBuiltinFunctions.FUZZY_EQ;
+                break;
+            }
 
-		default: {
-			throw new NotImplementedException("Operator " + t
-					+ " is not yet implemented");
-		}
-		}
-		return new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-	}
+            default: {
+                throw new NotImplementedException("Operator " + t + " is not yet implemented");
+            }
+        }
+        return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+    }
 
-	private static boolean hasOnlyChild(ILogicalOperator parent,
-			Mutable<ILogicalOperator> childCandidate) {
-		List<Mutable<ILogicalOperator>> inp = parent.getInputs();
-		if (inp == null || inp.size() != 1) {
-			return false;
-		}
-		return inp.get(0) == childCandidate;
-	}
+    private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
+        List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+        if (inp == null || inp.size() != 1) {
+            return false;
+        }
+        return inp.get(0) == childCandidate;
+    }
 
-	private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
-			Expression expr, Mutable<ILogicalOperator> topOp)
-			throws AsterixException {
-		switch (expr.getKind()) {
-		case VARIABLE_EXPRESSION: {
-			VariableReferenceExpression ve = new VariableReferenceExpression(
-					context.getVar(((VariableExpr) expr).getVar().getId()));
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
-					topOp);
-		}
-		case LITERAL_EXPRESSION: {
-			LiteralExpr val = (LiteralExpr) expr;
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-					new ConstantExpression(new AsterixConstantValue(
-							ConstantHelper.objectFromLiteral(val.getValue()))),
-					topOp);
-		}
-		default: {
-			// Mutable<ILogicalOperator> src = new
-			// Mutable<ILogicalOperator>();
-			// Mutable<ILogicalOperator> src = topOp;
-			if (expressionNeedsNoNesting(expr)) {
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						topOp);
-				ILogicalExpression exp = ((AssignOperator) p.first)
-						.getExpressions().get(0).getValue();
-				return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-						exp, p.first.getInputs().get(0));
-			} else {
-				Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+    private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
+            Mutable<ILogicalOperator> topOp) throws AsterixException {
+        switch (expr.getKind()) {
+            case VARIABLE_EXPRESSION: {
+                VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
+                        .getVar().getId()));
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
+            }
+            case LITERAL_EXPRESSION: {
+                LiteralExpr val = (LiteralExpr) expr;
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
+                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
+            }
+            default: {
+                // Mutable<ILogicalOperator> src = new
+                // Mutable<ILogicalOperator>();
+                // Mutable<ILogicalOperator> src = topOp;
+                if (expressionNeedsNoNesting(expr)) {
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
+                    ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
+                    return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
+                } else {
+                    Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
 
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						src);
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
 
-				if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-					// src.setOperator(topOp.getOperator());
-					Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
-							p.first);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second), top2);
-				} else {
-					SubplanOperator s = new SubplanOperator();
-					s.getInputs().add(topOp);
-					src.setValue(new NestedTupleSourceOperator(
-							new MutableObject<ILogicalOperator>(s)));
-					Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
-							p.first);
-					s.setRootOp(planRoot);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second),
-							new MutableObject<ILogicalOperator>(s));
-				}
-			}
-		}
-		}
+                    if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                        // src.setOperator(topOp.getOperator());
+                        Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), top2);
+                    } else {
+                        SubplanOperator s = new SubplanOperator();
+                        s.getInputs().add(topOp);
+                        src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+                        Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
+                        s.setRootOp(planRoot);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), new MutableObject<ILogicalOperator>(s));
+                    }
+                }
+            }
+        }
 
-	}
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
-			boolean noForClause, boolean isTop,
-			Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
-		if (isTop) {
-			ProjectOperator pr = new ProjectOperator(resVar);
-			pr.getInputs().add(resOpRef);
-			return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+    private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
+            Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+        if (isTop) {
+            ProjectOperator pr = new ProjectOperator(resVar);
+            pr.getInputs().add(resOpRef);
+            return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
 
-		} else if (noForClause) {
-			return new Pair<ILogicalOperator, LogicalVariable>(
-					resOpRef.getValue(), resVar);
-		} else {
-			return aggListify(resVar, resOpRef, false);
-		}
-	}
+        } else if (noForClause) {
+            return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
+        } else {
+            return aggListify(resVar, resOpRef, false);
+        }
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> aggListify(
-			LogicalVariable var, Mutable<ILogicalOperator> opRef,
-			boolean bProject) {
-		AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY,
-						new ArrayList<Mutable<ILogicalExpression>>());
-		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)));
-		agg.getInputs().add(opRef);
-		ILogicalOperator res;
-		if (bProject) {
-			ProjectOperator pr = new ProjectOperator(varListified);
-			pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
-			res = pr;
-		} else {
-			res = agg;
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
-	}
+    private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
+            boolean bProject) {
+        AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
+        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)));
+        agg.getInputs().add(opRef);
+        ILogicalOperator res;
+        if (bProject) {
+            ProjectOperator pr = new ProjectOperator(varListified);
+            pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+            res = pr;
+        } else {
+            res = agg;
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+    private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-		ArrayList<Expression> exprs = op.getExprList();
+        ArrayList<Expression> exprs = op.getExprList();
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		OperatorType opLogical = ops.get(0);
-		AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+        OperatorType opLogical = ops.get(0);
+        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
 
-		for (int i = 0; i <= nOps; i++) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			// now look at the operator
-			if (i < nOps) {
-				if (ops.get(i) != opLogical) {
-					throw new TranslationException("Unexpected operator "
-							+ ops.get(i) + " in an OperatorExpr starting with "
-							+ opLogical);
-				}
-			}
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		}
+        for (int i = 0; i <= nOps; i++) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            // now look at the operator
+            if (i < nOps) {
+                if (ops.get(i) != opLogical) {
+                    throw new TranslationException("Unexpected operator " + ops.get(i)
+                            + " in an OperatorExpr starting with " + opLogical);
+                }
+            }
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        }
 
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(f));
-		a.getInputs().add(topOp);
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
+        a.getInputs().add(topOp);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
 
-	}
+    }
 
-	private static boolean expressionNeedsNoNesting(Expression expr) {
-		Kind k = expr.getKind();
-		return k == Kind.LITERAL_EXPRESSION
-				|| k == Kind.LIST_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
-				|| k == Kind.OP_EXPRESSION
-				|| k == Kind.FIELD_ACCESSOR_EXPRESSION
-				|| k == Kind.INDEX_ACCESSOR_EXPRESSION
-				|| k == Kind.UNARY_EXPRESSION;
-	}
+    private static boolean expressionNeedsNoNesting(Expression expr) {
+        Kind k = expr.getKind();
+        return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+                || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
+                || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
+                || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
+    }
 
-	private <T> ArrayList<T> mkSingletonArrayList(T item) {
-		ArrayList<T> array = new ArrayList<T>(1);
-		array.add(item);
-		return array;
-	}
+    private <T> ArrayList<T> mkSingletonArrayList(T item) {
+        ArrayList<T> array = new ArrayList<T>(1);
+        array.add(item);
+        return array;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
-			RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
-			TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
-			NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
-			LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
-			DropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
-			ControlFeedStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
-			CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
-			OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
-			UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
-		switch (expr.getExpressionTag()) {
-		case VARIABLE: {
-			return new UnnestingFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-					new MutableObject<ILogicalExpression>(expr));
-		}
-		case FUNCTION_CALL: {
-			AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-			if (fce.getKind() == FunctionKind.UNNEST) {
-				return expr;
-			} else {
-				return new UnnestingFunctionCallExpression(
-						FunctionUtils
-								.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-						new MutableObject<ILogicalExpression>(expr));
-			}
-		}
-		default: {
-			return expr;
-		}
-		}
-	}
+    private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+        switch (expr.getExpressionTag()) {
+            case VARIABLE: {
+                return new UnnestingFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                        new MutableObject<ILogicalExpression>(expr));
+            }
+            case FUNCTION_CALL: {
+                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+                if (fce.getKind() == FunctionKind.UNNEST) {
+                    return expr;
+                } else {
+                    return new UnnestingFunctionCallExpression(
+                            FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                            new MutableObject<ILogicalExpression>(expr));
+                }
+            }
+            default: {
+                return expr;
+            }
+        }
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
-			InsertStatement insert, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
-			DeleteStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
-			UpdateStatement update, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
-			UpdateClause del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
-			DataverseDecl dv, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
-			DatasetDecl dd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
-			SetStatement ss, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
-			WriteStatement ws, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
-			CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
-			IndexDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
-			NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
-			DataverseDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
-			TypeDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visit(
-			CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
-			FunctionDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
-			BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 }
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 4fc1fc8..be6e2af 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
@@ -79,10 +79,8 @@
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
-import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
-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;
@@ -92,11 +90,11 @@
 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.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.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 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;
@@ -105,7 +103,6 @@
 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.ILogicalPlan;
-import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlanAndMetadata;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
@@ -153,1576 +150,1315 @@
  * source for the current subtree.
  */
 
-public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator
-		implements
-		IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+public class AqlPlusExpressionToPlanTranslator extends AbstractAqlTranslator implements
+        IAqlPlusExpressionVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
 
-	private static final Logger LOGGER = Logger
-			.getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
+    private static final Logger LOGGER = Logger.getLogger(AqlPlusExpressionToPlanTranslator.class.getName());
 
-	private class MetaScopeLogicalVariable {
-		private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
+    private class MetaScopeLogicalVariable {
+        private HashMap<Identifier, LogicalVariable> map = new HashMap<Identifier, LogicalVariable>();
 
-		public VariableReferenceExpression getVariableReferenceExpression(
-				Identifier id) throws AsterixException {
-			LogicalVariable var = map.get(id);
-			LOGGER.fine("get:" + id + ":" + var);
-			if (var == null) {
-				throw new AsterixException("Identifier " + id
-						+ " not found in AQL+ meta-scope.");
-			}
-			return new VariableReferenceExpression(var);
-		}
+        public VariableReferenceExpression getVariableReferenceExpression(Identifier id) throws AsterixException {
+            LogicalVariable var = map.get(id);
+            LOGGER.fine("get:" + id + ":" + var);
+            if (var == null) {
+                throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
+            }
+            return new VariableReferenceExpression(var);
+        }
 
-		public void put(Identifier id, LogicalVariable var) {
-			LOGGER.fine("put:" + id + ":" + var);
-			map.put(id, var);
-		}
-	}
+        public void put(Identifier id, LogicalVariable var) {
+            LOGGER.fine("put:" + id + ":" + var);
+            map.put(id, var);
+        }
+    }
 
-	private class MetaScopeILogicalOperator {
-		private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
+    private class MetaScopeILogicalOperator {
+        private HashMap<Identifier, ILogicalOperator> map = new HashMap<Identifier, ILogicalOperator>();
 
-		public ILogicalOperator get(Identifier id) throws AsterixException {
-			ILogicalOperator op = map.get(id);
-			if (op == null) {
-				throw new AsterixException("Identifier " + id
-						+ " not found in AQL+ meta-scope.");
-			}
-			return op;
-		}
+        public ILogicalOperator get(Identifier id) throws AsterixException {
+            ILogicalOperator op = map.get(id);
+            if (op == null) {
+                throw new AsterixException("Identifier " + id + " not found in AQL+ meta-scope.");
+            }
+            return op;
+        }
 
-		public void put(Identifier id, ILogicalOperator op) {
-			LOGGER.fine("put:" + id + ":" + op);
-			map.put(id, op);
-		}
-	}
+        public void put(Identifier id, ILogicalOperator op) {
+            LOGGER.fine("put:" + id + ":" + op);
+            map.put(id, op);
+        }
+    }
 
-	private final long txnId;
-	private final MetadataTransactionContext mdTxnCtx;
-	private TranslationContext context;
-	private String outputDatasetName;
-	private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
-	private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
-	private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
+    private final long txnId;
+   private TranslationContext context;
+    private String outputDatasetName;
+    private ICompiledDmlStatement stmt;
+    private AqlMetadataProvider metadataProvider;
 
-	public AqlPlusExpressionToPlanTranslator(long txnId,
-			MetadataTransactionContext mdTxnCtx, Counter currentVarCounter,
-			String outputDatasetName) {
-		this.txnId = txnId;
-		this.mdTxnCtx = mdTxnCtx;
-		this.context = new TranslationContext(currentVarCounter);
-		this.outputDatasetName = outputDatasetName;
-		this.context.setTopFlwor(false);
-	}
+    private MetaScopeLogicalVariable metaScopeExp = new MetaScopeLogicalVariable();
+    private MetaScopeILogicalOperator metaScopeOp = new MetaScopeILogicalOperator();
+    private static LogicalVariable METADATA_DUMMY_VAR = new LogicalVariable(-1);
 
-	public int getVarCounter() {
-		return context.getVarCounter();
-	}
+    
+    
+    public AqlPlusExpressionToPlanTranslator(long txnId, AqlMetadataProvider metadataProvider,
+            Counter currentVarCounter, String outputDatasetName, ICompiledDmlStatement stmt) {
+        this.txnId = txnId;
+        this.metadataProvider = metadataProvider;
+        this.context = new TranslationContext(currentVarCounter);
+        this.outputDatasetName = outputDatasetName;
+        this.stmt = stmt;
+        this.context.setTopFlwor(false);
+    }
 
-	public ILogicalPlanAndMetadata translate(Query expr)
-			throws AlgebricksException, AsterixException {
-		return translate(expr, null);
-	}
+    public int getVarCounter() {
+        return context.getVarCounter();
+    }
 
-	public ILogicalPlanAndMetadata translate(Query expr,
-			AqlCompiledMetadataDeclarations compiledDeclarations)
-			throws AlgebricksException, AsterixException {
-		if (expr == null) {
-			return null;
-		}
-		if (compiledDeclarations == null) {
-			compiledDeclarations = compileMetadata(mdTxnCtx,
-					expr.getPrologDeclList(), true);
-		}
-		if (!compiledDeclarations.isConnectedToDataverse())
-			compiledDeclarations.connectToDataverse(compiledDeclarations
-					.getDataverseName());
-		IDataFormat format = compiledDeclarations.getFormat();
-		if (format == null) {
-			throw new AlgebricksException("Data format has not been set.");
-		}
-		format.registerRuntimeFunctions();
-		Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-				new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator()));
+    public ILogicalPlan translate(Query expr) throws AlgebricksException, AsterixException {
+        return translate(expr, null);
+    }
 
-		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+    public ILogicalPlan translate(Query expr, AqlMetadataProvider metadata)
+            throws AlgebricksException, AsterixException {
+        IDataFormat format = metadata.getFormat();
+        if (format == null) {
+            throw new AlgebricksException("Data format has not been set.");
+        }
+        format.registerRuntimeFunctions();
+        Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, new MutableObject<ILogicalOperator>(
+                new EmptyTupleSourceOperator()));
 
-		boolean isTransactionalWrite = false;
-		ILogicalOperator topOp = p.first;
-		ProjectOperator project = (ProjectOperator) topOp;
-		LogicalVariable resVar = project.getVariables().get(0);
-		if (outputDatasetName == null) {
-			List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			writeExprList.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(resVar)));
-			FileSplitSinkId fssi = new FileSplitSinkId(
-					compiledDeclarations.getOutputFile());
-			FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
-			topOp = new WriteOperator(writeExprList, sink);
-			topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
-		} else {
-			Dataset dataset = compiledDeclarations
-					.findDataset(outputDatasetName);
-			if (dataset == null) {
-				throw new AlgebricksException("Cannot find dataset "
-						+ outputDatasetName);
-			}
-			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 (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));
-				LogicalVariable v = context.newVar();
-				vars.add(v);
-				varRefsForLoading.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(v)));
-			}
-			AssignOperator assign = new AssignOperator(vars, exprs);
-			assign.getInputs()
-					.add(new MutableObject<ILogicalOperator>(project));
-		}
+        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-		AqlMetadataProvider metadataProvider = new AqlMetadataProvider(txnId,
-				isTransactionalWrite, compiledDeclarations);
-		ILogicalPlanAndMetadata planAndMetadata = new AqlLogicalPlanAndMetadataImpl(
-				plan, metadataProvider);
-		return planAndMetadata;
-	}
+        boolean isTransactionalWrite = false;
+        ILogicalOperator topOp = p.first;
+        ProjectOperator project = (ProjectOperator) topOp;
+        LogicalVariable resVar = project.getVariables().get(0);
+        if (outputDatasetName == null) {
+            List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
+            FileSplitSinkId fssi = new FileSplitSinkId(metadata.getOutputFile());
+            FileSplitDataSink sink = new FileSplitDataSink(fssi, null);
+            topOp = new WriteOperator(writeExprList, sink);
+            topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
+        } else {
+            Dataset dataset = metadata.findDataset(stmt.getDataverseName(), outputDatasetName);
+            if (dataset == null) {
+                throw new AlgebricksException("Cannot find dataset " + outputDatasetName);
+            }
+            if (dataset.getDatasetType() == DatasetType.EXTERNAL) {
+                throw new AlgebricksException("Cannot write output to an external dataset.");
+            }
+            ARecordType itemType = (ARecordType) metadata.findType(dataset.getDataverseName(),
+                    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 (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));
+                LogicalVariable v = context.newVar();
+                vars.add(v);
+                varRefsForLoading.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(v)));
+            }
+            AssignOperator assign = new AssignOperator(vars, exprs);
+            assign.getInputs().add(new MutableObject<ILogicalOperator>(project));
+        }
 
-	public ILogicalPlan translate(List<Clause> clauses)
-			throws AlgebricksException, AsterixException {
+        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+        return plan;
+    }
 
-		if (clauses == null) {
-			return null;
-		}
+    public ILogicalPlan translate(List<Clause> clauses) throws AlgebricksException, AsterixException {
 
-		Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(
-				new EmptyTupleSourceOperator());
-		Pair<ILogicalOperator, LogicalVariable> p = null;
-		for (Clause c : clauses) {
-			p = c.accept(this, opRef);
-			opRef = new MutableObject<ILogicalOperator>(p.first);
-		}
+        if (clauses == null) {
+            return null;
+        }
 
-		ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
+        Mutable<ILogicalOperator> opRef = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+        Pair<ILogicalOperator, LogicalVariable> p = null;
+        for (Clause c : clauses) {
+            p = c.accept(this, opRef);
+            opRef = new MutableObject<ILogicalOperator>(p.first);
+        }
 
-		ILogicalOperator topOp = p.first;
+        ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<Mutable<ILogicalOperator>>();
 
-		globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
-		ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
-		return plan;
-	}
+        ILogicalOperator topOp = p.first;
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v = context.newVar(fc.getVarExpr());
+        globalPlanRoots.add(new MutableObject<ILogicalOperator>(topOp));
+        ILogicalPlan plan = new ALogicalPlanImpl(globalPlanRoots);
+        return plan;
+    }
 
-		Expression inExpr = fc.getInExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				inExpr, tupSource);
-		ILogicalOperator returnedOp;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitForClause(ForClause fc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar(fc.getVarExpr());
 
-		if (fc.getPosVarExpr() == null) {
-			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.getInputs().add(eo.second);
+        Expression inExpr = fc.getInExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(inExpr, tupSource);
+        ILogicalOperator returnedOp;
 
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+        if (fc.getPosVarExpr() == null) {
+            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.getInputs().add(eo.second);
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		LogicalVariable v;
-		ILogicalOperator returnedOp;
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-		switch (lc.getBindingExpr().getKind()) {
-		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.getInputs().add(tupSource);
-			break;
-		}
-		default: {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					lc.getBindingExpr(), tupSource);
-			v = context.newVar(lc.getVarExpr());
-			returnedOp = new AssignOperator(v,
-					new MutableObject<ILogicalExpression>(eo.first));
-			returnedOp.getInputs().add(eo.second);
-			break;
-		}
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLetClause(LetClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v;
+        ILogicalOperator returnedOp;
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(
-			FLWOGRExpression flwor, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> flworPlan = tupSource;
-		boolean isTop = context.isTopFlwor();
-		if (isTop) {
-			context.setTopFlwor(false);
-		}
-		for (Clause c : flwor.getClauseList()) {
-			Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this,
-					flworPlan);
-			flworPlan = new MutableObject<ILogicalOperator>(pC.first);
-		}
+        switch (lc.getBindingExpr().getKind()) {
+            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.getInputs().add(tupSource);
+                break;
+            }
+            default: {
+                Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(lc.getBindingExpr(),
+                        tupSource);
+                v = context.newVar(lc.getVarExpr());
+                returnedOp = new AssignOperator(v, new MutableObject<ILogicalExpression>(eo.first));
+                returnedOp.getInputs().add(eo.second);
+                break;
+            }
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(returnedOp, v);
+    }
 
-		Expression r = flwor.getReturnExpr();
-		boolean noFlworClause = flwor.noForClause();
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFlworExpression(FLWOGRExpression flwor,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> flworPlan = tupSource;
+        boolean isTop = context.isTopFlwor();
+        if (isTop) {
+            context.setTopFlwor(false);
+        }
+        for (Clause c : flwor.getClauseList()) {
+            Pair<ILogicalOperator, LogicalVariable> pC = c.accept(this, flworPlan);
+            flworPlan = new MutableObject<ILogicalOperator>(pC.first);
+        }
 
-		if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
-			VariableExpr v = (VariableExpr) r;
-			LogicalVariable var = context.getVar(v.getVar().getId());
+        Expression r = flwor.getReturnExpr();
+        boolean noFlworClause = flwor.noForClause();
 
-			return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
+        if (r.getKind() == Kind.VARIABLE_EXPRESSION) {
+            VariableExpr v = (VariableExpr) r;
+            LogicalVariable var = context.getVar(v.getVar().getId());
 
-		} else {
-			Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(
-					flworPlan.getValue());
-			Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this,
-					baseOp);
-			ILogicalOperator rOp = rRes.first;
-			ILogicalOperator resOp;
-			if (expressionNeedsNoNesting(r)) {
-				baseOp.setValue(flworPlan.getValue());
-				resOp = rOp;
-			} else {
-				SubplanOperator s = new SubplanOperator(rOp);
-				s.getInputs().add(flworPlan);
-				resOp = s;
-				baseOp.setValue(new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(s)));
-			}
-			Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(
-					resOp);
-			return produceFlwrResult(noFlworClause, isTop, resOpRef,
-					rRes.second);
-		}
-	}
+            return produceFlwrResult(noFlworClause, isTop, flworPlan, var);
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(
-			FieldAccessor fa, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				fa.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(p.first));
-		ILogicalExpression faExpr = new ConstantExpression(
-				new AsterixConstantValue(new AString(fa.getIdent().getValue())));
-		fldAccess.getArguments().add(
-				new MutableObject<ILogicalExpression>(faExpr));
-		AssignOperator a = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(fldAccess));
-		a.getInputs().add(p.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+        } else {
+            Mutable<ILogicalOperator> baseOp = new MutableObject<ILogicalOperator>(flworPlan.getValue());
+            Pair<ILogicalOperator, LogicalVariable> rRes = r.accept(this, baseOp);
+            ILogicalOperator rOp = rRes.first;
+            ILogicalOperator resOp;
+            if (expressionNeedsNoNesting(r)) {
+                baseOp.setValue(flworPlan.getValue());
+                resOp = rOp;
+            } else {
+                SubplanOperator s = new SubplanOperator(rOp);
+                s.getInputs().add(flworPlan);
+                resOp = s;
+                baseOp.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+            }
+            Mutable<ILogicalOperator> resOpRef = new MutableObject<ILogicalOperator>(resOp);
+            return produceFlwrResult(noFlworClause, isTop, resOpRef, rRes.second);
+        }
+    }
 
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFieldAccessor(FieldAccessor fa,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(fa.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression fldAccess = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_NAME));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        ILogicalExpression faExpr = new ConstantExpression(new AsterixConstantValue(new AString(fa.getIdent()
+                .getValue())));
+        fldAccess.getArguments().add(new MutableObject<ILogicalExpression>(faExpr));
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(fldAccess));
+        a.getInputs().add(p.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(
-			IndexAccessor ia, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				ia.getExpr(), tupSource);
-		LogicalVariable v = context.newVar();
-		AbstractFunctionCallExpression f;
-		int i = ia.getIndex();
-		if (i == IndexAccessor.ANY) {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		} else {
-			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)))));
-		}
-		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)
-			throws AsterixException {
-		LogicalVariable v = context.newVar();
-		AsterixFunction fid = fcall.getIdent();
-		List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> topOp = tupSource;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexAccessor(IndexAccessor ia,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(ia.getExpr(), tupSource);
+        LogicalVariable v = context.newVar();
+        AbstractFunctionCallExpression f;
+        int i = ia.getIndex();
+        if (i == IndexAccessor.ANY) {
+            f = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.ANY_COLLECTION_MEMBER));
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        } else {
+            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)))));
+        }
+        AssignOperator a = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+        a.getInputs().add(p.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v);
+    }
 
-		for (Expression expr : fcall.getExprList()) {
-			switch (expr.getKind()) {
-			case VARIABLE_EXPRESSION: {
-				LogicalVariable var = context.getVar(((VariableExpr) expr)
-						.getVar().getId());
-				args.add(new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(var)));
-				break;
-			}
-			case LITERAL_EXPRESSION: {
-				LiteralExpr val = (LiteralExpr) expr;
-				args.add(new MutableObject<ILogicalExpression>(
-						new ConstantExpression(
-								new AsterixConstantValue(ConstantHelper
-										.objectFromLiteral(val.getValue())))));
-				break;
-			}
-			default: {
-				Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-						expr, topOp);
-				AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second
-						.getValue();
-				args.add(new MutableObject<ILogicalExpression>(eo.first));
-				if (o1 != null
-						&& !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(
-								o1, topOp))) {
-					topOp = eo.second;
-				}
-				break;
-			}
-			}
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCallExpr(CallExpr fcall, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        LogicalVariable v = context.newVar();
+        FunctionSignature signature = fcall.getFunctionSignature();
+        List<Mutable<ILogicalExpression>> args = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		FunctionIdentifier fi = new FunctionIdentifier(
-				AlgebricksBuiltinFunctions.ALGEBRICKS_NS, fid.getFunctionName());
-		AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
-		FunctionIdentifier builtinAquafi = afi == null ? null : afi
-				.getFunctionIdentifier();
+        for (Expression expr : fcall.getExprList()) {
+            switch (expr.getKind()) {
+                case VARIABLE_EXPRESSION: {
+                    LogicalVariable var = context.getVar(((VariableExpr) expr).getVar().getId());
+                    args.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    break;
+                }
+                case LITERAL_EXPRESSION: {
+                    LiteralExpr val = (LiteralExpr) expr;
+                    args.add(new MutableObject<ILogicalExpression>(new ConstantExpression(new AsterixConstantValue(
+                            ConstantHelper.objectFromLiteral(val.getValue())))));
+                    break;
+                }
+                default: {
+                    Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+                    AbstractLogicalOperator o1 = (AbstractLogicalOperator) eo.second.getValue();
+                    args.add(new MutableObject<ILogicalExpression>(eo.first));
+                    if (o1 != null && !(o1.getOperatorTag() == LogicalOperatorTag.ASSIGN && hasOnlyChild(o1, topOp))) {
+                        topOp = eo.second;
+                    }
+                    break;
+                }
+            }
+        }
 
-		if (builtinAquafi != null) {
-			fi = builtinAquafi;
-		} else {
-			fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-					fid.getFunctionName());
-			FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions
-					.getBuiltinFunctionIdentifier(fi);
-			if (builtinAsterixFi != null) {
-				fi = builtinAsterixFi;
-			}
-		}
-		AbstractFunctionCallExpression f;
-		if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
-			f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi,
-					args);
-		} else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
-			UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-			ufce.setReturnsUniqueValues(AsterixBuiltinFunctions
-					.returnsUniqueValues(fi));
-			f = ufce;
-		} else {
-			f = new ScalarFunctionCallExpression(
-					FunctionUtils.getFunctionInfo(fi), args);
-		}
-		AssignOperator op = new AssignOperator(v,
-				new MutableObject<ILogicalExpression>(f));
-		if (topOp != null) {
-			op.getInputs().add(topOp);
-		}
+        FunctionIdentifier fi = new FunctionIdentifier(AlgebricksBuiltinFunctions.ALGEBRICKS_NS, signature.getName());
+        AsterixFunctionInfo afi = AsterixBuiltinFunctions.lookupFunction(fi);
+        FunctionIdentifier builtinAquafi = afi == null ? null : afi.getFunctionIdentifier();
 
-		return new Pair<ILogicalOperator, LogicalVariable>(op, v);
-	}
+        if (builtinAquafi != null) {
+            fi = builtinAquafi;
+        } else {
+            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, signature.getName());
+            FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
+            if (builtinAsterixFi != null) {
+                fi = builtinAsterixFi;
+            }
+        }
+        AbstractFunctionCallExpression f;
+        if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
+            f = AsterixBuiltinFunctions.makeAggregateFunctionExpression(fi, args);
+        } else if (AsterixBuiltinFunctions.isBuiltinUnnestingFunction(fi)) {
+            UnnestingFunctionCallExpression ufce = new UnnestingFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(fi), args);
+            ufce.setReturnsUniqueValues(AsterixBuiltinFunctions.returnsUniqueValues(fi));
+            f = ufce;
+        } else {
+            f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fi), args);
+        }
+        AssignOperator op = new AssignOperator(v, new MutableObject<ILogicalExpression>(f));
+        if (topOp != null) {
+            op.getInputs().add(topOp);
+        }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(
-			FunctionDecl fd, Mutable<ILogicalOperator> tupSource) {
-		// TODO Auto-generated method stub
-		throw new NotImplementedException();
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(op, v);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(
-			GroupbyClause gc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		GroupByOperator gOp = new GroupByOperator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addGbyExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
-			LogicalVariable v;
-			VariableExpr vexpr = ve.getVar();
-			if (vexpr != null) {
-				v = context.newVar(vexpr);
-			} else {
-				v = context.newVar();
-			}
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					ve.getExpr(), topOp);
-			gOp.addDecorExpression(v, eo.first);
-			topOp = eo.second;
-		}
-		gOp.getInputs().add(topOp);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDecl(FunctionDecl fd,
+            Mutable<ILogicalOperator> tupSource) {
+        // TODO Auto-generated method stub
+        throw new NotImplementedException();
+    }
 
-		for (VariableExpr var : gc.getWithVarList()) {
-			LogicalVariable aggVar = context.newVar();
-			LogicalVariable oldVar = context.getVar(var);
-			List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			flArgs.add(new MutableObject<ILogicalExpression>(
-					new VariableReferenceExpression(oldVar)));
-			AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions
-					.makeAggregateFunctionExpression(
-							AsterixBuiltinFunctions.LISTIFY, flArgs);
-			AggregateOperator agg = new AggregateOperator(
-					mkSingletonArrayList(aggVar),
-					(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-							fListify)));
-			agg.getInputs().add(
-					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
-			// the one bound by the aggregation op.
-			context.setVar(var, aggVar);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitGroupbyClause(GroupbyClause gc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        GroupByOperator gOp = new GroupByOperator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (GbyVariableExpressionPair ve : gc.getGbyPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addGbyExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        for (GbyVariableExpressionPair ve : gc.getDecorPairList()) {
+            LogicalVariable v;
+            VariableExpr vexpr = ve.getVar();
+            if (vexpr != null) {
+                v = context.newVar(vexpr);
+            } else {
+                v = context.newVar();
+            }
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(ve.getExpr(), topOp);
+            gOp.addDecorExpression(v, eo.first);
+            topOp = eo.second;
+        }
+        gOp.getInputs().add(topOp);
 
-		gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY,
-				gc.hasHashGroupByHint());
-		return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
-	}
+        for (VariableExpr var : gc.getWithVarList()) {
+            LogicalVariable aggVar = context.newVar();
+            LogicalVariable oldVar = context.getVar(var);
+            List<Mutable<ILogicalExpression>> flArgs = new ArrayList<Mutable<ILogicalExpression>>(1);
+            flArgs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(oldVar)));
+            AggregateFunctionCallExpression fListify = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                    AsterixBuiltinFunctions.LISTIFY, flArgs);
+            AggregateOperator agg = new AggregateOperator(mkSingletonArrayList(aggVar),
+                    (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fListify)));
+            agg.getInputs().add(
+                    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
+            // the one bound by the aggregation op.
+            context.setVar(var, aggVar);
+        }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		// In the most general case, IfThenElse is translated in the following
-		// way.
-		//
-		// We assign the result of the condition to one variable varCond.
-		// We create one subplan which contains the plan for the "then" branch,
-		// on top of which there is a selection whose condition is varCond.
-		// Similarly, we create one subplan for the "else" branch, in which the
-		// selection is not(varCond).
-		// Finally, we concatenate the results. (??)
+        gOp.getAnnotations().put(OperatorAnnotations.USE_HASH_GROUP_BY, gc.hasHashGroupByHint());
+        return new Pair<ILogicalOperator, LogicalVariable>(gOp, null);
+    }
 
-		Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr()
-				.accept(this, tupSource);
-		ILogicalOperator opCond = pCond.first;
-		LogicalVariable varCond = pCond.second;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIfExpr(IfExpr ifexpr, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        // In the most general case, IfThenElse is translated in the following
+        // way.
+        //
+        // We assign the result of the condition to one variable varCond.
+        // We create one subplan which contains the plan for the "then" branch,
+        // on top of which there is a selection whose condition is varCond.
+        // Similarly, we create one subplan for the "else" branch, in which the
+        // selection is not(varCond).
+        // Finally, we concatenate the results. (??)
 
-		SubplanOperator sp = new SubplanOperator();
-		Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(
-				new NestedTupleSourceOperator(
-						new MutableObject<ILogicalOperator>(sp)));
+        Pair<ILogicalOperator, LogicalVariable> pCond = ifexpr.getCondExpr().accept(this, tupSource);
+        ILogicalOperator opCond = pCond.first;
+        LogicalVariable varCond = pCond.second;
 
-		Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr()
-				.accept(this, nestedSource);
-		SelectOperator sel1 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
+        SubplanOperator sp = new SubplanOperator();
+        Mutable<ILogicalOperator> nestedSource = new MutableObject<ILogicalOperator>(new NestedTupleSourceOperator(
+                new MutableObject<ILogicalOperator>(sp)));
 
-		Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr()
-				.accept(this, nestedSource);
-		AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-				new MutableObject<ILogicalExpression>(
-						new VariableReferenceExpression(varCond)));
-		SelectOperator sel2 = new SelectOperator(
-				new MutableObject<ILogicalExpression>(notVarCond));
-		sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
+        Pair<ILogicalOperator, LogicalVariable> pThen = ifexpr.getThenExpr().accept(this, nestedSource);
+        SelectOperator sel1 = new SelectOperator(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(
+                varCond)));
+        sel1.getInputs().add(new MutableObject<ILogicalOperator>(pThen.first));
 
-		ILogicalPlan p1 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel1));
-		sp.getNestedPlans().add(p1);
-		ILogicalPlan p2 = new ALogicalPlanImpl(
-				new MutableObject<ILogicalOperator>(sel2));
-		sp.getNestedPlans().add(p2);
+        Pair<ILogicalOperator, LogicalVariable> pElse = ifexpr.getElseExpr().accept(this, nestedSource);
+        AbstractFunctionCallExpression notVarCond = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), new MutableObject<ILogicalExpression>(
+                        new VariableReferenceExpression(varCond)));
+        SelectOperator sel2 = new SelectOperator(new MutableObject<ILogicalExpression>(notVarCond));
+        sel2.getInputs().add(new MutableObject<ILogicalOperator>(pElse.first));
 
-		Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(
-				opCond);
-		sp.getInputs().add(opCondRef);
+        ILogicalPlan p1 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel1));
+        sp.getNestedPlans().add(p1);
+        ILogicalPlan p2 = new ALogicalPlanImpl(new MutableObject<ILogicalOperator>(sel2));
+        sp.getNestedPlans().add(p2);
 
-		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)));
-		AssignOperator a = new AssignOperator(resV,
-				new MutableObject<ILogicalExpression>(concatNonNull));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
+        Mutable<ILogicalOperator> opCondRef = new MutableObject<ILogicalOperator>(opCond);
+        sp.getInputs().add(opCondRef);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
-	}
+        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)));
+        AssignOperator a = new AssignOperator(resV, new MutableObject<ILogicalExpression>(concatNonNull));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(sp));
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(
-			LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
-		LogicalVariable var = context.newVar();
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(new ConstantExpression(
-						new AsterixConstantValue(ConstantHelper
-								.objectFromLiteral(l.getValue())))));
-		if (tupSource != null) {
-			a.getInputs().add(tupSource);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(a, resV);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLiteralExpr(LiteralExpr l, Mutable<ILogicalOperator> tupSource) {
+        LogicalVariable var = context.newVar();
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(new ConstantExpression(
+                new AsterixConstantValue(ConstantHelper.objectFromLiteral(l.getValue())))));
+        if (tupSource != null) {
+            a.getInputs().add(tupSource);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-		if (nOps > 0
-				&& (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
-			return visitAndOrOperator(op, tupSource);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOperatorExpr(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-		ArrayList<Expression> exprs = op.getExprList();
+        if (nOps > 0 && (ops.get(0) == OperatorType.AND || ops.get(0) == OperatorType.OR)) {
+            return visitAndOrOperator(op, tupSource);
+        }
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        ArrayList<Expression> exprs = op.getExprList();
 
-		ILogicalExpression currExpr = null;
-		for (int i = 0; i <= nOps; i++) {
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			ILogicalExpression e = p.first;
-			// now look at the operator
-			if (i < nOps) {
-				if (OperatorExpr.opIsComparison(ops.get(i))) {
-					AbstractFunctionCallExpression c = createComparisonExpression(ops
-							.get(i));
+        ILogicalExpression currExpr = null;
+        for (int i = 0; i <= nOps; i++) {
 
-					// chain the operators
-					if (i == 0) {
-						c.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = c;
-						if (op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.LEFT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						c.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = c;
-						if (i == 1 && op.isBroadcastOperand(i)) {
-							BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-							bcast.setObject(BroadcastSide.RIGHT);
-							c.getAnnotations()
-									.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-											bcast);
-						}
-					}
-				} else {
-					AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops
-							.get(i));
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            ILogicalExpression e = p.first;
+            // now look at the operator
+            if (i < nOps) {
+                if (OperatorExpr.opIsComparison(ops.get(i))) {
+                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i));
 
-					if (i == 0) {
-						f.getArguments().add(
-								new MutableObject<ILogicalExpression>(e));
-						currExpr = f;
-					} else {
-						((AbstractFunctionCallExpression) currExpr)
-								.getArguments()
-								.add(new MutableObject<ILogicalExpression>(e));
-						f.getArguments()
-								.add(new MutableObject<ILogicalExpression>(
-										currExpr));
-						currExpr = f;
-					}
-				}
-			} else { // don't forget the last expression...
-				((AbstractFunctionCallExpression) currExpr).getArguments().add(
-						new MutableObject<ILogicalExpression>(e));
-				if (i == 1 && op.isBroadcastOperand(i)) {
-					BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-					bcast.setObject(BroadcastSide.RIGHT);
-					((AbstractFunctionCallExpression) currExpr)
-							.getAnnotations()
-							.put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY,
-									bcast);
-				}
-			}
-		}
+                    // chain the operators
+                    if (i == 0) {
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = c;
+                        if (op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.LEFT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        c.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = c;
+                        if (i == 1 && op.isBroadcastOperand(i)) {
+                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                            bcast.setObject(BroadcastSide.RIGHT);
+                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                        }
+                    }
+                } else {
+                    AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(ops.get(i));
 
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(currExpr));
+                    if (i == 0) {
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(e));
+                        currExpr = f;
+                    } else {
+                        ((AbstractFunctionCallExpression) currExpr).getArguments().add(
+                                new MutableObject<ILogicalExpression>(e));
+                        f.getArguments().add(new MutableObject<ILogicalExpression>(currExpr));
+                        currExpr = f;
+                    }
+                }
+            } else { // don't forget the last expression...
+                ((AbstractFunctionCallExpression) currExpr).getArguments()
+                        .add(new MutableObject<ILogicalExpression>(e));
+                if (i == 1 && op.isBroadcastOperand(i)) {
+                    BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                    bcast.setObject(BroadcastSide.RIGHT);
+                    ((AbstractFunctionCallExpression) currExpr).getAnnotations().put(
+                            BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
+                }
+            }
+        }
 
-		a.getInputs().add(topOp);
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(currExpr));
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
-	}
+        a.getInputs().add(topOp);
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(
-			OrderbyClause oc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+    }
 
-		OrderOperator ord = new OrderOperator();
-		Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression e : oc.getOrderbyList()) {
-			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)));
-			topOp = p.second;
-		}
-		ord.getInputs().add(topOp);
-		if (oc.getNumTuples() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.CARDINALITY,
-					oc.getNumTuples());
-		}
-		if (oc.getNumFrames() > 0) {
-			ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES,
-					oc.getNumFrames());
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderbyClause(OrderbyClause oc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(
-			QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> topOp = tupSource;
+        OrderOperator ord = new OrderOperator();
+        Iterator<OrderModifier> modifIter = oc.getModifierList().iterator();
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression e : oc.getOrderbyList()) {
+            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)));
+            topOp = p.second;
+        }
+        ord.getInputs().add(topOp);
+        if (oc.getNumTuples() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.CARDINALITY, oc.getNumTuples());
+        }
+        if (oc.getNumFrames() > 0) {
+            ord.getAnnotations().put(OperatorAnnotations.MAX_NUMBER_FRAMES, oc.getNumFrames());
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(ord, null);
+    }
 
-		ILogicalOperator firstOp = null;
-		Mutable<ILogicalOperator> lastOp = null;
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuantifiedExpression(QuantifiedExpression qe,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		for (QuantifiedPair qt : qe.getQuantifiedList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					qt.getExpr(), topOp);
-			topOp = eo1.second;
-			LogicalVariable uVar = context.newVar(qt.getVarExpr());
-			ILogicalOperator u = new UnnestOperator(uVar,
-					new MutableObject<ILogicalExpression>(
-							makeUnnestExpression(eo1.first)));
+        ILogicalOperator firstOp = null;
+        Mutable<ILogicalOperator> lastOp = null;
 
-			if (firstOp == null) {
-				firstOp = u;
-			}
-			if (lastOp != null) {
-				u.getInputs().add(lastOp);
-			}
-			lastOp = new MutableObject<ILogicalOperator>(u);
-		}
+        for (QuantifiedPair qt : qe.getQuantifiedList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(qt.getExpr(), topOp);
+            topOp = eo1.second;
+            LogicalVariable uVar = context.newVar(qt.getVarExpr());
+            ILogicalOperator u = new UnnestOperator(uVar, new MutableObject<ILogicalExpression>(
+                    makeUnnestExpression(eo1.first)));
 
-		// We make all the unnest correspond. to quantif. vars. sit on top
-		// in the hope of enabling joins & other optimiz.
-		firstOp.getInputs().add(topOp);
-		topOp = lastOp;
+            if (firstOp == null) {
+                firstOp = u;
+            }
+            if (lastOp != null) {
+                u.getInputs().add(lastOp);
+            }
+            lastOp = new MutableObject<ILogicalOperator>(u);
+        }
 
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-				qe.getSatisfiesExpr(), topOp);
+        // We make all the unnest correspond. to quantif. vars. sit on top
+        // in the hope of enabling joins & other optimiz.
+        firstOp.getInputs().add(topOp);
+        topOp = lastOp;
 
-		AggregateFunctionCallExpression fAgg;
-		SelectOperator s;
-		if (qe.getQuantifier() == Quantifier.SOME) {
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					eo2.first));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.NON_EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		} else { // EVERY
-			List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(
-					1);
-			satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
-			s = new SelectOperator(new MutableObject<ILogicalExpression>(
-					new ScalarFunctionCallExpression(FunctionUtils
-							.getFunctionInfo(AlgebricksBuiltinFunctions.NOT),
-							satExprList)));
-			s.getInputs().add(eo2.second);
-			fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
-					AsterixBuiltinFunctions.EMPTY_STREAM,
-					new ArrayList<Mutable<ILogicalExpression>>());
-		}
-		LogicalVariable qeVar = context.newVar();
-		AggregateOperator a = new AggregateOperator(
-				mkSingletonArrayList(qeVar),
-				(List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(
-						fAgg)));
-		a.getInputs().add(new MutableObject<ILogicalOperator>(s));
-		return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
-	}
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(qe.getSatisfiesExpr(), topOp);
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		return q.getBody().accept(this, tupSource);
-	}
+        AggregateFunctionCallExpression fAgg;
+        SelectOperator s;
+        if (qe.getQuantifier() == Quantifier.SOME) {
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(eo2.first));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.NON_EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        } else { // EVERY
+            List<Mutable<ILogicalExpression>> satExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
+            satExprList.add(new MutableObject<ILogicalExpression>(eo2.first));
+            s = new SelectOperator(new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AlgebricksBuiltinFunctions.NOT), satExprList)));
+            s.getInputs().add(eo2.second);
+            fAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(AsterixBuiltinFunctions.EMPTY_STREAM,
+                    new ArrayList<Mutable<ILogicalExpression>>());
+        }
+        LogicalVariable qeVar = context.newVar();
+        AggregateOperator a = new AggregateOperator(mkSingletonArrayList(qeVar),
+                (List) mkSingletonArrayList(new MutableObject<ILogicalExpression>(fAgg)));
+        a.getInputs().add(new MutableObject<ILogicalOperator>(s));
+        return new Pair<ILogicalOperator, LogicalVariable>(a, qeVar);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(
-			RecordConstructor rc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils
-						.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (FieldBinding fb : rc.getFbList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(
-					fb.getLeftExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo1.first));
-			topOp = eo1.second;
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(
-					fb.getRightExpr(), topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo2.first));
-			topOp = eo2.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitQuery(Query q, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        return q.getBody().accept(this, tupSource);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(
-			ListConstructor lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
-				: AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
-		AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a = new AssignOperator(v1,
-				new MutableObject<ILogicalExpression>(f));
-		Mutable<ILogicalOperator> topOp = tupSource;
-		for (Expression expr : lc.getExprList()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-					expr, topOp);
-			f.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			topOp = eo.second;
-		}
-		a.getInputs().add(topOp);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordConstructor(RecordConstructor rc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(
+                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (FieldBinding fb : rc.getFbList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = aqlExprToAlgExpression(fb.getLeftExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo1.first));
+            topOp = eo1.second;
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo2 = aqlExprToAlgExpression(fb.getRightExpr(), topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo2.first));
+            topOp = eo2.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitListConstructor(ListConstructor lc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        FunctionIdentifier fid = (lc.getType() == Type.ORDERED_LIST_CONSTRUCTOR) ? AsterixBuiltinFunctions.ORDERED_LIST_CONSTRUCTOR
+                : AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR;
+        AbstractFunctionCallExpression f = new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(f));
+        Mutable<ILogicalOperator> topOp = tupSource;
+        for (Expression expr : lc.getExprList()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, topOp);
+            f.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            topOp = eo.second;
+        }
+        a.getInputs().add(topOp);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Expression expr = u.getExpr();
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(
-				expr, tupSource);
-		LogicalVariable v1 = context.newVar();
-		AssignOperator a;
-		if (u.getSign() == Sign.POSITIVE) {
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(
-					eo.first));
-		} else {
-			AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
-			m.getArguments().add(
-					new MutableObject<ILogicalExpression>(eo.first));
-			a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
-		}
-		a.getInputs().add(eo.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnaryExpr(UnaryExpr u, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Expression expr = u.getExpr();
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo = aqlExprToAlgExpression(expr, tupSource);
+        LogicalVariable v1 = context.newVar();
+        AssignOperator a;
+        if (u.getSign() == Sign.POSITIVE) {
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(eo.first));
+        } else {
+            AbstractFunctionCallExpression m = new ScalarFunctionCallExpression(
+                    FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.NUMERIC_UNARY_MINUS));
+            m.getArguments().add(new MutableObject<ILogicalExpression>(eo.first));
+            a = new AssignOperator(v1, new MutableObject<ILogicalExpression>(m));
+        }
+        a.getInputs().add(eo.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, v1);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(
-			VariableExpr v, Mutable<ILogicalOperator> tupSource) {
-		// 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)));
-		a.getInputs().add(tupSource);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitVariableExpr(VariableExpr v, Mutable<ILogicalOperator> tupSource) {
+        // 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)));
+        a.getInputs().add(tupSource);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(
-			WhereClause w, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-				w.getWhereExpr(), tupSource);
-		SelectOperator s = new SelectOperator(
-				new MutableObject<ILogicalExpression>(p.first));
-		s.getInputs().add(p.second);
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWhereClause(WhereClause w, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(w.getWhereExpr(), tupSource);
+        SelectOperator s = new SelectOperator(new MutableObject<ILogicalExpression>(p.first));
+        s.getInputs().add(p.second);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(s, null);
-	}
+        return new Pair<ILogicalOperator, LogicalVariable>(s, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(
-			LimitClause lc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getLimitExpr(), tupSource);
-		LimitOperator opLim;
-		Expression offset = lc.getOffset();
-		if (offset != null) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(
-					offset, p1.second);
-			opLim = new LimitOperator(p1.first, p2.first);
-			opLim.getInputs().add(p2.second);
-		} else {
-			opLim = new LimitOperator(p1.first);
-			opLim.getInputs().add(p1.second);
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLimitClause(LimitClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        LimitOperator opLim;
+        Expression offset = lc.getOffset();
+        if (offset != null) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = aqlExprToAlgExpression(offset, p1.second);
+            opLim = new LimitOperator(p1.first, p2.first);
+            opLim.getInputs().add(p2.second);
+        } else {
+            opLim = new LimitOperator(p1.first);
+            opLim.getInputs().add(p1.second);
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(opLim, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc,
-			Mutable<ILogicalOperator> tupSource) throws AsterixException {
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(
-				lc.getDieExpr(), tupSource);
-		DieOperator opDie = new DieOperator(p1.first);
-		opDie.getInputs().add(p1.second);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDieClause(DieClause lc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = aqlExprToAlgExpression(lc.getDieExpr(), tupSource);
+        DieOperator opDie = new DieOperator(p1.first);
+        opDie.getInputs().add(p1.second);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDie, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(
-			DistinctClause dc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
-		Mutable<ILogicalOperator> input = null;
-		for (Expression expr : dc.getDistinctByExpr()) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					expr, tupSource);
-			exprList.add(new MutableObject<ILogicalExpression>(p.first));
-			input = p.second;
-		}
-		DistinctOperator opDistinct = new DistinctOperator(exprList);
-		opDistinct.getInputs().add(input);
-		return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDistinctClause(DistinctClause dc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        List<Mutable<ILogicalExpression>> exprList = new ArrayList<Mutable<ILogicalExpression>>();
+        Mutable<ILogicalOperator> input = null;
+        for (Expression expr : dc.getDistinctByExpr()) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(expr, tupSource);
+            exprList.add(new MutableObject<ILogicalExpression>(p.first));
+            input = p.second;
+        }
+        DistinctOperator opDistinct = new DistinctOperator(exprList);
+        opDistinct.getInputs().add(input);
+        return new Pair<ILogicalOperator, LogicalVariable>(opDistinct, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(
-			UnionExpr unionExpr, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		Mutable<ILogicalOperator> ts = tupSource;
-		ILogicalOperator lastOp = null;
-		LogicalVariable lastVar = null;
-		boolean first = true;
-		for (Expression e : unionExpr.getExprs()) {
-			if (first) {
-				first = false;
-			} else {
-				ts = new MutableObject<ILogicalOperator>(
-						new EmptyTupleSourceOperator());
-			}
-			Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
-			if (lastOp == null) {
-				lastOp = p1.first;
-				lastVar = p1.second;
-			} else {
-				LogicalVariable unnestVar1 = context.newVar();
-				UnnestOperator unnest1 = new UnnestOperator(
-						unnestVar1,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										lastVar))));
-				unnest1.getInputs().add(
-						new MutableObject<ILogicalOperator>(lastOp));
-				LogicalVariable unnestVar2 = context.newVar();
-				UnnestOperator unnest2 = new UnnestOperator(
-						unnestVar2,
-						new MutableObject<ILogicalExpression>(
-								makeUnnestExpression(new VariableReferenceExpression(
-										p1.second))));
-				unnest2.getInputs().add(
-						new MutableObject<ILogicalOperator>(p1.first));
-				List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
-						1);
-				LogicalVariable resultVar = context.newVar();
-				Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
-						unnestVar1, unnestVar2, resultVar);
-				varMap.add(triple);
-				UnionAllOperator unionOp = new UnionAllOperator(varMap);
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest1));
-				unionOp.getInputs().add(
-						new MutableObject<ILogicalOperator>(unnest2));
-				lastVar = resultVar;
-				lastOp = unionOp;
-			}
-		}
-		LogicalVariable aggVar = context.newVar();
-		ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(
-				1);
-		aggregVars.add(aggVar);
-		List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		afcExprs.add(new MutableObject<ILogicalExpression>(
-				new VariableReferenceExpression(lastVar)));
-		AggregateFunctionCallExpression afc = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY, afcExprs);
-		ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(
-				1);
-		aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
-		AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
-		agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
-		return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnionExpr(UnionExpr unionExpr,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        Mutable<ILogicalOperator> ts = tupSource;
+        ILogicalOperator lastOp = null;
+        LogicalVariable lastVar = null;
+        boolean first = true;
+        for (Expression e : unionExpr.getExprs()) {
+            if (first) {
+                first = false;
+            } else {
+                ts = new MutableObject<ILogicalOperator>(new EmptyTupleSourceOperator());
+            }
+            Pair<ILogicalOperator, LogicalVariable> p1 = e.accept(this, ts);
+            if (lastOp == null) {
+                lastOp = p1.first;
+                lastVar = p1.second;
+            } else {
+                LogicalVariable unnestVar1 = context.newVar();
+                UnnestOperator unnest1 = new UnnestOperator(unnestVar1, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(lastVar))));
+                unnest1.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+                LogicalVariable unnestVar2 = context.newVar();
+                UnnestOperator unnest2 = new UnnestOperator(unnestVar2, new MutableObject<ILogicalExpression>(
+                        makeUnnestExpression(new VariableReferenceExpression(p1.second))));
+                unnest2.getInputs().add(new MutableObject<ILogicalOperator>(p1.first));
+                List<Triple<LogicalVariable, LogicalVariable, LogicalVariable>> varMap = new ArrayList<Triple<LogicalVariable, LogicalVariable, LogicalVariable>>(
+                        1);
+                LogicalVariable resultVar = context.newVar();
+                Triple<LogicalVariable, LogicalVariable, LogicalVariable> triple = new Triple<LogicalVariable, LogicalVariable, LogicalVariable>(
+                        unnestVar1, unnestVar2, resultVar);
+                varMap.add(triple);
+                UnionAllOperator unionOp = new UnionAllOperator(varMap);
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest1));
+                unionOp.getInputs().add(new MutableObject<ILogicalOperator>(unnest2));
+                lastVar = resultVar;
+                lastOp = unionOp;
+            }
+        }
+        LogicalVariable aggVar = context.newVar();
+        ArrayList<LogicalVariable> aggregVars = new ArrayList<LogicalVariable>(1);
+        aggregVars.add(aggVar);
+        List<Mutable<ILogicalExpression>> afcExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        afcExprs.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(lastVar)));
+        AggregateFunctionCallExpression afc = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, afcExprs);
+        ArrayList<Mutable<ILogicalExpression>> aggregExprs = new ArrayList<Mutable<ILogicalExpression>>(1);
+        aggregExprs.add(new MutableObject<ILogicalExpression>(afc));
+        AggregateOperator agg = new AggregateOperator(aggregVars, aggregExprs);
+        agg.getInputs().add(new MutableObject<ILogicalOperator>(lastOp));
+        return new Pair<ILogicalOperator, LogicalVariable>(agg, aggVar);
+    }
 
-	private AbstractFunctionCallExpression createComparisonExpression(
-			OperatorType t) {
-		FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-		IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
-		return new ScalarFunctionCallExpression(finfo);
-	}
+    private AbstractFunctionCallExpression createComparisonExpression(OperatorType t) {
+        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
+        IFunctionInfo finfo = FunctionUtils.getFunctionInfo(fi);
+        return new ScalarFunctionCallExpression(finfo);
+    }
 
-	private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-		switch (t) {
-		case EQ: {
-			return AlgebricksBuiltinFunctions.EQ;
-		}
-		case NEQ: {
-			return AlgebricksBuiltinFunctions.NEQ;
-		}
-		case GT: {
-			return AlgebricksBuiltinFunctions.GT;
-		}
-		case GE: {
-			return AlgebricksBuiltinFunctions.GE;
-		}
-		case LT: {
-			return AlgebricksBuiltinFunctions.LT;
-		}
-		case LE: {
-			return AlgebricksBuiltinFunctions.LE;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-	}
+    private FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
+        switch (t) {
+            case EQ: {
+                return AlgebricksBuiltinFunctions.EQ;
+            }
+            case NEQ: {
+                return AlgebricksBuiltinFunctions.NEQ;
+            }
+            case GT: {
+                return AlgebricksBuiltinFunctions.GT;
+            }
+            case GE: {
+                return AlgebricksBuiltinFunctions.GE;
+            }
+            case LT: {
+                return AlgebricksBuiltinFunctions.LT;
+            }
+            case LE: {
+                return AlgebricksBuiltinFunctions.LE;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
 
-	private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(
-			OperatorType t) throws AsterixException {
+    private AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t)
+            throws AsterixException {
 
-		FunctionIdentifier fid = null;
-		switch (t) {
-		case PLUS: {
-			fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
-			break;
-		}
-		case MINUS: {
-			fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
-			break;
-		}
-		case MUL: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
-			break;
-		}
-		case DIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
-			break;
-		}
-		case MOD: {
-			fid = AsterixBuiltinFunctions.NUMERIC_MOD;
-			break;
-		}
-		case IDIV: {
-			fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
-			break;
-		}
-		case CARET: {
-			fid = AsterixBuiltinFunctions.CARET;
-			break;
-		}
-		case AND: {
-			fid = AlgebricksBuiltinFunctions.AND;
-			break;
-		}
-		case OR: {
-			fid = AlgebricksBuiltinFunctions.OR;
-			break;
-		}
-		case FUZZY_EQ: {
-			fid = AsterixBuiltinFunctions.FUZZY_EQ;
-			break;
-		}
+        FunctionIdentifier fid = null;
+        switch (t) {
+            case PLUS: {
+                fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
+                break;
+            }
+            case MINUS: {
+                fid = AsterixBuiltinFunctions.NUMERIC_SUBTRACT;
+                break;
+            }
+            case MUL: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MULTIPLY;
+                break;
+            }
+            case DIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_DIVIDE;
+                break;
+            }
+            case MOD: {
+                fid = AsterixBuiltinFunctions.NUMERIC_MOD;
+                break;
+            }
+            case IDIV: {
+                fid = AsterixBuiltinFunctions.NUMERIC_IDIV;
+                break;
+            }
+            case CARET: {
+                fid = AsterixBuiltinFunctions.CARET;
+                break;
+            }
+            case AND: {
+                fid = AlgebricksBuiltinFunctions.AND;
+                break;
+            }
+            case OR: {
+                fid = AlgebricksBuiltinFunctions.OR;
+                break;
+            }
+            case FUZZY_EQ: {
+                fid = AsterixBuiltinFunctions.FUZZY_EQ;
+                break;
+            }
 
-		default: {
-			throw new NotImplementedException("Operator " + t
-					+ " is not yet implemented");
-		}
-		}
-		return new ScalarFunctionCallExpression(
-				FunctionUtils.getFunctionInfo(fid));
-	}
+            default: {
+                throw new NotImplementedException("Operator " + t + " is not yet implemented");
+            }
+        }
+        return new ScalarFunctionCallExpression(FunctionUtils.getFunctionInfo(fid));
+    }
 
-	private static boolean hasOnlyChild(ILogicalOperator parent,
-			Mutable<ILogicalOperator> childCandidate) {
-		List<Mutable<ILogicalOperator>> inp = parent.getInputs();
-		if (inp == null || inp.size() != 1) {
-			return false;
-		}
-		return inp.get(0) == childCandidate;
-	}
+    private static boolean hasOnlyChild(ILogicalOperator parent, Mutable<ILogicalOperator> childCandidate) {
+        List<Mutable<ILogicalOperator>> inp = parent.getInputs();
+        if (inp == null || inp.size() != 1) {
+            return false;
+        }
+        return inp.get(0) == childCandidate;
+    }
 
-	private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(
-			Expression expr, Mutable<ILogicalOperator> topOp)
-			throws AsterixException {
-		switch (expr.getKind()) {
-		case VARIABLE_EXPRESSION: {
-			VariableReferenceExpression ve = new VariableReferenceExpression(
-					context.getVar(((VariableExpr) expr).getVar().getId()));
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve,
-					topOp);
-		}
-		case METAVARIABLE_EXPRESSION: {
-			ILogicalExpression le = metaScopeExp
-					.getVariableReferenceExpression(((VariableExpr) expr)
-							.getVar());
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le,
-					topOp);
-		}
-		case LITERAL_EXPRESSION: {
-			LiteralExpr val = (LiteralExpr) expr;
-			return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-					new ConstantExpression(new AsterixConstantValue(
-							ConstantHelper.objectFromLiteral(val.getValue()))),
-					topOp);
-		}
-		default: {
-			// Mutable<ILogicalExpression> src = new
-			// Mutable<ILogicalExpression>();
-			// Mutable<ILogicalExpression> src = topOp;
-			if (expressionNeedsNoNesting(expr)) {
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						topOp);
-				ILogicalExpression exp = ((AssignOperator) p.first)
-						.getExpressions().get(0).getValue();
-				return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-						exp, p.first.getInputs().get(0));
-			} else {
-				Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
+    private Pair<ILogicalExpression, Mutable<ILogicalOperator>> aqlExprToAlgExpression(Expression expr,
+            Mutable<ILogicalOperator> topOp) throws AsterixException {
+        switch (expr.getKind()) {
+            case VARIABLE_EXPRESSION: {
+                VariableReferenceExpression ve = new VariableReferenceExpression(context.getVar(((VariableExpr) expr)
+                        .getVar().getId()));
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(ve, topOp);
+            }
+            case METAVARIABLE_EXPRESSION: {
+                ILogicalExpression le = metaScopeExp.getVariableReferenceExpression(((VariableExpr) expr).getVar());
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(le, topOp);
+            }
+            case LITERAL_EXPRESSION: {
+                LiteralExpr val = (LiteralExpr) expr;
+                return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new ConstantExpression(
+                        new AsterixConstantValue(ConstantHelper.objectFromLiteral(val.getValue()))), topOp);
+            }
+            default: {
+                // Mutable<ILogicalExpression> src = new
+                // Mutable<ILogicalExpression>();
+                // Mutable<ILogicalExpression> src = topOp;
+                if (expressionNeedsNoNesting(expr)) {
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOp);
+                    ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
+                    return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(exp, p.first.getInputs().get(0));
+                } else {
+                    Mutable<ILogicalOperator> src = new MutableObject<ILogicalOperator>();
 
-				Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this,
-						src);
+                    Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, src);
 
-				if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
-					// src.setOperator(topOp.getOperator());
-					Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(
-							p.first);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second), top2);
-				} else {
-					SubplanOperator s = new SubplanOperator();
-					s.getInputs().add(topOp);
-					src.setValue(new NestedTupleSourceOperator(
-							new MutableObject<ILogicalOperator>(s)));
-					Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(
-							p.first);
-					s.setRootOp(planRoot);
-					return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(
-							new VariableReferenceExpression(p.second),
-							new MutableObject<ILogicalOperator>(s));
-				}
-			}
-		}
-		}
+                    if (((AbstractLogicalOperator) p.first).getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+                        // src.setOperator(topOp.getOperator());
+                        Mutable<ILogicalOperator> top2 = new MutableObject<ILogicalOperator>(p.first);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), top2);
+                    } else {
+                        SubplanOperator s = new SubplanOperator();
+                        s.getInputs().add(topOp);
+                        src.setValue(new NestedTupleSourceOperator(new MutableObject<ILogicalOperator>(s)));
+                        Mutable<ILogicalOperator> planRoot = new MutableObject<ILogicalOperator>(p.first);
+                        s.setRootOp(planRoot);
+                        return new Pair<ILogicalExpression, Mutable<ILogicalOperator>>(new VariableReferenceExpression(
+                                p.second), new MutableObject<ILogicalOperator>(s));
+                    }
+                }
+            }
+        }
 
-	}
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(
-			boolean noForClause, boolean isTop,
-			Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
-		if (isTop) {
-			ProjectOperator pr = new ProjectOperator(resVar);
-			pr.getInputs().add(resOpRef);
-			return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
+    private Pair<ILogicalOperator, LogicalVariable> produceFlwrResult(boolean noForClause, boolean isTop,
+            Mutable<ILogicalOperator> resOpRef, LogicalVariable resVar) {
+        if (isTop) {
+            ProjectOperator pr = new ProjectOperator(resVar);
+            pr.getInputs().add(resOpRef);
+            return new Pair<ILogicalOperator, LogicalVariable>(pr, resVar);
 
-		} else if (noForClause) {
-			return new Pair<ILogicalOperator, LogicalVariable>(
-					resOpRef.getValue(), resVar);
-		} else {
-			return aggListify(resVar, resOpRef, false);
-		}
-	}
+        } else if (noForClause) {
+            return new Pair<ILogicalOperator, LogicalVariable>(resOpRef.getValue(), resVar);
+        } else {
+            return aggListify(resVar, resOpRef, false);
+        }
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> aggListify(
-			LogicalVariable var, Mutable<ILogicalOperator> opRef,
-			boolean bProject) {
-		AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions
-				.makeAggregateFunctionExpression(
-						AsterixBuiltinFunctions.LISTIFY,
-						new ArrayList<Mutable<ILogicalExpression>>());
-		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)));
-		agg.getInputs().add(opRef);
-		ILogicalOperator res;
-		if (bProject) {
-			ProjectOperator pr = new ProjectOperator(varListified);
-			pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
-			res = pr;
-		} else {
-			res = agg;
-		}
-		return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
-	}
+    private Pair<ILogicalOperator, LogicalVariable> aggListify(LogicalVariable var, Mutable<ILogicalOperator> opRef,
+            boolean bProject) {
+        AggregateFunctionCallExpression funAgg = AsterixBuiltinFunctions.makeAggregateFunctionExpression(
+                AsterixBuiltinFunctions.LISTIFY, new ArrayList<Mutable<ILogicalExpression>>());
+        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)));
+        agg.getInputs().add(opRef);
+        ILogicalOperator res;
+        if (bProject) {
+            ProjectOperator pr = new ProjectOperator(varListified);
+            pr.getInputs().add(new MutableObject<ILogicalOperator>(agg));
+            res = pr;
+        } else {
+            res = agg;
+        }
+        return new Pair<ILogicalOperator, LogicalVariable>(res, varListified);
+    }
 
-	private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(
-			OperatorExpr op, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		ArrayList<OperatorType> ops = op.getOpList();
-		int nOps = ops.size();
+    private Pair<ILogicalOperator, LogicalVariable> visitAndOrOperator(OperatorExpr op,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        ArrayList<OperatorType> ops = op.getOpList();
+        int nOps = ops.size();
 
-		ArrayList<Expression> exprs = op.getExprList();
+        ArrayList<Expression> exprs = op.getExprList();
 
-		Mutable<ILogicalOperator> topOp = tupSource;
+        Mutable<ILogicalOperator> topOp = tupSource;
 
-		OperatorType opLogical = ops.get(0);
-		AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
+        OperatorType opLogical = ops.get(0);
+        AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opLogical);
 
-		for (int i = 0; i <= nOps; i++) {
-			Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(
-					exprs.get(i), topOp);
-			topOp = p.second;
-			// now look at the operator
-			if (i < nOps) {
-				if (ops.get(i) != opLogical) {
-					throw new TranslationException("Unexpected operator "
-							+ ops.get(i) + " in an OperatorExpr starting with "
-							+ opLogical);
-				}
-			}
-			f.getArguments()
-					.add(new MutableObject<ILogicalExpression>(p.first));
-		}
+        for (int i = 0; i <= nOps; i++) {
+            Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = aqlExprToAlgExpression(exprs.get(i), topOp);
+            topOp = p.second;
+            // now look at the operator
+            if (i < nOps) {
+                if (ops.get(i) != opLogical) {
+                    throw new TranslationException("Unexpected operator " + ops.get(i)
+                            + " in an OperatorExpr starting with " + opLogical);
+                }
+            }
+            f.getArguments().add(new MutableObject<ILogicalExpression>(p.first));
+        }
 
-		LogicalVariable assignedVar = context.newVar();
-		AssignOperator a = new AssignOperator(assignedVar,
-				new MutableObject<ILogicalExpression>(f));
-		a.getInputs().add(topOp);
+        LogicalVariable assignedVar = context.newVar();
+        AssignOperator a = new AssignOperator(assignedVar, new MutableObject<ILogicalExpression>(f));
+        a.getInputs().add(topOp);
 
-		return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, assignedVar);
 
-	}
+    }
 
-	private static boolean expressionNeedsNoNesting(Expression expr) {
-		Kind k = expr.getKind();
-		return k == Kind.LITERAL_EXPRESSION
-				|| k == Kind.LIST_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.RECORD_CONSTRUCTOR_EXPRESSION
-				|| k == Kind.VARIABLE_EXPRESSION || k == Kind.CALL_EXPRESSION
-				|| k == Kind.OP_EXPRESSION
-				|| k == Kind.FIELD_ACCESSOR_EXPRESSION
-				|| k == Kind.INDEX_ACCESSOR_EXPRESSION
-				|| k == Kind.UNARY_EXPRESSION;
-	}
+    private static boolean expressionNeedsNoNesting(Expression expr) {
+        Kind k = expr.getKind();
+        return k == Kind.LITERAL_EXPRESSION || k == Kind.LIST_CONSTRUCTOR_EXPRESSION
+                || k == Kind.RECORD_CONSTRUCTOR_EXPRESSION || k == Kind.VARIABLE_EXPRESSION
+                || k == Kind.CALL_EXPRESSION || k == Kind.OP_EXPRESSION || k == Kind.FIELD_ACCESSOR_EXPRESSION
+                || k == Kind.INDEX_ACCESSOR_EXPRESSION || k == Kind.UNARY_EXPRESSION;
+    }
 
-	private <T> ArrayList<T> mkSingletonArrayList(T item) {
-		ArrayList<T> array = new ArrayList<T>(1);
-		array.add(item);
-		return array;
-	}
+    private <T> ArrayList<T> mkSingletonArrayList(T item) {
+        ArrayList<T> array = new ArrayList<T>(1);
+        array.add(item);
+        return array;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDecl(TypeDecl td, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(
-			RecordTypeDefinition tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitRecordTypeDefiniton(RecordTypeDefinition tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(
-			TypeReferenceExpression tre, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeReferenceExpression(TypeReferenceExpression tre,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(
-			NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodegroupDecl(NodegroupDecl ngd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(
-			LoadFromFileStatement stmtLoad, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromFileStatement(LoadFromFileStatement stmtLoad,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(
-			DropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDropStatement(DropStatement del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(
-			CreateIndexStatement cis, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateIndexStatement(CreateIndexStatement cis,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(
-			OrderedListTypeDefinition olte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitOrderedListTypeDefiniton(OrderedListTypeDefinition olte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(
-			UnorderedListTypeDefinition ulte, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUnorderedListTypeDefiniton(UnorderedListTypeDefinition ulte,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(
-			MetaVariableClause mc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc
-				.getVar()), null);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableClause(MetaVariableClause mc,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        return new Pair<ILogicalOperator, LogicalVariable>(metaScopeOp.get(mc.getVar()), null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(
-			JoinClause jc, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		// Pair<ILogicalOperator, LogicalVariable> leftSide =
-		// jc.getLeftExpr().accept(this, tupSource);
-		Mutable<ILogicalOperator> opRef = tupSource;
-		Pair<ILogicalOperator, LogicalVariable> leftSide = null;
-		for (Clause c : jc.getLeftClauses()) {
-			leftSide = c.accept(this, opRef);
-			opRef = new MutableObject<ILogicalOperator>(leftSide.first);
-		}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitJoinClause(JoinClause jc, Mutable<ILogicalOperator> tupSource)
+            throws AsterixException {
+        // Pair<ILogicalOperator, LogicalVariable> leftSide =
+        // jc.getLeftExpr().accept(this, tupSource);
+        Mutable<ILogicalOperator> opRef = tupSource;
+        Pair<ILogicalOperator, LogicalVariable> leftSide = null;
+        for (Clause c : jc.getLeftClauses()) {
+            leftSide = c.accept(this, opRef);
+            opRef = new MutableObject<ILogicalOperator>(leftSide.first);
+        }
 
-		// Pair<ILogicalOperator, LogicalVariable> rightSide =
-		// jc.getRightExpr().accept(this, tupSource);
-		opRef = tupSource;
-		Pair<ILogicalOperator, LogicalVariable> rightSide = null;
-		for (Clause c : jc.getRightClauses()) {
-			rightSide = c.accept(this, opRef);
-			opRef = new MutableObject<ILogicalOperator>(rightSide.first);
-		}
+        // Pair<ILogicalOperator, LogicalVariable> rightSide =
+        // jc.getRightExpr().accept(this, tupSource);
+        opRef = tupSource;
+        Pair<ILogicalOperator, LogicalVariable> rightSide = null;
+        for (Clause c : jc.getRightClauses()) {
+            rightSide = c.accept(this, opRef);
+            opRef = new MutableObject<ILogicalOperator>(rightSide.first);
+        }
 
-		Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(
-				jc.getWhereExpr(), tupSource);
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> whereCond = aqlExprToAlgExpression(jc.getWhereExpr(),
+                tupSource);
 
-		AbstractBinaryJoinOperator join;
-		switch (jc.getKind()) {
-		case INNER: {
-			join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(
-					whereCond.first));
-			break;
-		}
-		case LEFT_OUTER: {
-			join = new LeftOuterJoinOperator(
-					new MutableObject<ILogicalExpression>(whereCond.first));
-			break;
-		}
-		default: {
-			throw new IllegalStateException();
-		}
-		}
-		join.getInputs().add(
-				new MutableObject<ILogicalOperator>(leftSide.first));
-		join.getInputs().add(
-				new MutableObject<ILogicalOperator>(rightSide.first));
-		return new Pair<ILogicalOperator, LogicalVariable>(join, null);
-	}
+        AbstractBinaryJoinOperator join;
+        switch (jc.getKind()) {
+            case INNER: {
+                join = new InnerJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
+                break;
+            }
+            case LEFT_OUTER: {
+                join = new LeftOuterJoinOperator(new MutableObject<ILogicalExpression>(whereCond.first));
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+        join.getInputs().add(new MutableObject<ILogicalOperator>(leftSide.first));
+        join.getInputs().add(new MutableObject<ILogicalOperator>(rightSide.first));
+        return new Pair<ILogicalOperator, LogicalVariable>(join, null);
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(
-			MetaVariableExpr me, Mutable<ILogicalOperator> tupSource)
-			throws AsterixException {
-		LogicalVariable var = context.newVar();
-		AssignOperator a = new AssignOperator(var,
-				new MutableObject<ILogicalExpression>(metaScopeExp
-						.getVariableReferenceExpression(me.getVar())));
-		a.getInputs().add(tupSource);
-		return new Pair<ILogicalOperator, LogicalVariable>(a, var);
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitMetaVariableExpr(MetaVariableExpr me,
+            Mutable<ILogicalOperator> tupSource) throws AsterixException {
+        LogicalVariable var = context.newVar();
+        AssignOperator a = new AssignOperator(var, new MutableObject<ILogicalExpression>(
+                metaScopeExp.getVariableReferenceExpression(me.getVar())));
+        a.getInputs().add(tupSource);
+        return new Pair<ILogicalOperator, LogicalVariable>(a, var);
+    }
 
-	public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
-		metaScopeOp.put(id, op);
-	}
+    public void addOperatorToMetaScope(Identifier id, ILogicalOperator op) {
+        metaScopeOp.put(id, op);
+    }
 
-	public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
-		metaScopeExp.put(id, var);
-	}
+    public void addVariableToMetaScope(Identifier id, LogicalVariable var) {
+        metaScopeExp.put(id, var);
+    }
 
-	private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
-		switch (expr.getExpressionTag()) {
-		case VARIABLE: {
-			return new UnnestingFunctionCallExpression(
-					FunctionUtils
-							.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-					new MutableObject<ILogicalExpression>(expr));
-		}
-		case FUNCTION_CALL: {
-			AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-			if (fce.getKind() == FunctionKind.UNNEST) {
-				return expr;
-			} else {
-				return new UnnestingFunctionCallExpression(
-						FunctionUtils
-								.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
-						new MutableObject<ILogicalExpression>(expr));
-			}
-		}
-		default: {
-			return expr;
-		}
-		}
-	}
+    private ILogicalExpression makeUnnestExpression(ILogicalExpression expr) {
+        switch (expr.getExpressionTag()) {
+            case VARIABLE: {
+                return new UnnestingFunctionCallExpression(
+                        FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                        new MutableObject<ILogicalExpression>(expr));
+            }
+            case FUNCTION_CALL: {
+                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+                if (fce.getKind() == FunctionKind.UNNEST) {
+                    return expr;
+                } else {
+                    return new UnnestingFunctionCallExpression(
+                            FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.SCAN_COLLECTION),
+                            new MutableObject<ILogicalExpression>(expr));
+                }
+            }
+            default: {
+                return expr;
+            }
+        }
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(
-			InsertStatement insert, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitInsertStatement(InsertStatement insert,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(
-			DeleteStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDeleteStatement(DeleteStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(
-			UpdateStatement update, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateStatement(UpdateStatement update,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(
-			UpdateClause del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitUpdateClause(UpdateClause del, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(
-			DataverseDecl dv, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDecl(DataverseDecl dv, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(
-			DatasetDecl dd, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDatasetDecl(DatasetDecl dd, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(
-			SetStatement ss, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitSetStatement(SetStatement ss, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(
-			WriteStatement ws, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitWriteStatement(WriteStatement ws, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
-			WriteFromQueryResultStatement stmtLoad,
-			Mutable<ILogicalOperator> arg) throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitLoadFromQueryResultStatement(
+            WriteFromQueryResultStatement stmtLoad, Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(
-			CreateDataverseStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitCreateDataverseStatement(CreateDataverseStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(
-			IndexDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitIndexDropStatement(IndexDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(
-			NodeGroupDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitNodeGroupDropStatement(NodeGroupDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(
-			DataverseDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitDataverseDropStatement(DataverseDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(
-			TypeDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitTypeDropStatement(TypeDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(
-			ControlFeedStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitControlFeedStatement(ControlFeedStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visit(
-			CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(CreateFunctionStatement cfs, Mutable<ILogicalOperator> arg)
+            throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(
-			FunctionDropStatement del, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitFunctionDropStatement(FunctionDropStatement del,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 
-	@Override
-	public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(
-			BeginFeedStatement bf, Mutable<ILogicalOperator> arg)
-			throws AsterixException {
-		// TODO Auto-generated method stub
-		return null;
-	}
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visitBeginFeedStatement(BeginFeedStatement bf,
+            Mutable<ILogicalOperator> arg) throws AsterixException {
+        // TODO Auto-generated method stub
+        return null;
+    }
 }
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
new file mode 100644
index 0000000..625fed1
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/CompiledStatements.java
@@ -0,0 +1,575 @@
+/*
+ * Copyright 2009-2012 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.translator;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.aql.base.Clause;
+import edu.uci.ics.asterix.aql.base.Expression;
+import edu.uci.ics.asterix.aql.base.Statement.Kind;
+import edu.uci.ics.asterix.aql.expression.CallExpr;
+import edu.uci.ics.asterix.aql.expression.ControlFeedStatement.OperationType;
+import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
+import edu.uci.ics.asterix.aql.expression.FieldAccessor;
+import edu.uci.ics.asterix.aql.expression.FieldBinding;
+import edu.uci.ics.asterix.aql.expression.ForClause;
+import edu.uci.ics.asterix.aql.expression.Identifier;
+import edu.uci.ics.asterix.aql.expression.LiteralExpr;
+import edu.uci.ics.asterix.aql.expression.Query;
+import edu.uci.ics.asterix.aql.expression.RecordConstructor;
+import edu.uci.ics.asterix.aql.expression.VariableExpr;
+import edu.uci.ics.asterix.aql.expression.WhereClause;
+import edu.uci.ics.asterix.aql.literal.StringLiteral;
+import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
+import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+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.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+/**
+ * An AQL statement instance is translated into an instance of type CompileX
+ * that has additional fields for use by the AqlTranslator.
+ */
+public class CompiledStatements {
+
+	public static interface ICompiledStatement {
+
+		public Kind getKind();
+	}
+
+	public static class CompiledWriteFromQueryResultStatement implements
+			ICompiledDmlStatement {
+
+		private String dataverseName;
+		private String datasetName;
+		private Query query;
+		private int varCounter;
+
+		public CompiledWriteFromQueryResultStatement(String dataverseName,
+				String datasetName, Query query, int varCounter) {
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.query = query;
+			this.varCounter = varCounter;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		public int getVarCounter() {
+			return varCounter;
+		}
+
+		public Query getQuery() {
+			return query;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.WRITE_FROM_QUERY_RESULT;
+		}
+
+	}
+
+	public static class CompiledDatasetDropStatement implements
+			ICompiledStatement {
+		private final String dataverseName;
+		private final String datasetName;
+
+		public CompiledDatasetDropStatement(String dataverseName,
+				String datasetName) {
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.DATASET_DROP;
+		}
+	}
+
+	// added by yasser
+	public static class CompiledCreateDataverseStatement implements
+			ICompiledStatement {
+		private String dataverseName;
+		private String format;
+
+		public CompiledCreateDataverseStatement(String dataverseName,
+				String format) {
+			this.dataverseName = dataverseName;
+			this.format = format;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public String getFormat() {
+			return format;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.CREATE_DATAVERSE;
+		}
+	}
+
+	public static class CompiledNodeGroupDropStatement implements
+			ICompiledStatement {
+		private String nodeGroupName;
+
+		public CompiledNodeGroupDropStatement(String nodeGroupName) {
+			this.nodeGroupName = nodeGroupName;
+		}
+
+		public String getNodeGroupName() {
+			return nodeGroupName;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.NODEGROUP_DROP;
+		}
+	}
+
+	public static class CompiledIndexDropStatement implements
+			ICompiledStatement {
+		private String dataverseName;
+		private String datasetName;
+		private String indexName;
+
+		public CompiledIndexDropStatement(String dataverseName,
+				String datasetName, String indexName) {
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.indexName = indexName;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		public String getIndexName() {
+			return indexName;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.INDEX_DROP;
+		}
+	}
+
+	public static class CompiledDataverseDropStatement implements
+			ICompiledStatement {
+		private String dataverseName;
+		private boolean ifExists;
+
+		public CompiledDataverseDropStatement(String dataverseName,
+				boolean ifExists) {
+			this.dataverseName = dataverseName;
+			this.ifExists = ifExists;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public boolean getIfExists() {
+			return ifExists;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.DATAVERSE_DROP;
+		}
+	}
+
+	public static class CompiledTypeDropStatement implements ICompiledStatement {
+		private String typeName;
+
+		public CompiledTypeDropStatement(String nodeGroupName) {
+			this.typeName = nodeGroupName;
+		}
+
+		public String getTypeName() {
+			return typeName;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.TYPE_DROP;
+		}
+	}
+
+	public static interface ICompiledDmlStatement extends ICompiledStatement {
+
+		public String getDataverseName();
+
+		public String getDatasetName();
+	}
+
+	public static class CompiledCreateIndexStatement implements
+			ICompiledDmlStatement {
+		private final String indexName;
+		private final String dataverseName;
+		private final String datasetName;
+		private final List<String> keyFields;
+		private final IndexType indexType;
+
+		// Specific to NGram index.
+		private final int gramLength;
+
+		public CompiledCreateIndexStatement(String indexName,
+				String dataverseName, String datasetName,
+				List<String> keyFields, int gramLength, IndexType indexType) {
+			this.indexName = indexName;
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.keyFields = keyFields;
+			this.gramLength = gramLength;
+			this.indexType = indexType;
+		}
+
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public String getIndexName() {
+			return indexName;
+		}
+
+		public List<String> getKeyFields() {
+			return keyFields;
+		}
+
+		public IndexType getIndexType() {
+			return indexType;
+		}
+
+		public int getGramLength() {
+			return gramLength;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.CREATE_INDEX;
+		}
+	}
+
+	public static class CompiledLoadFromFileStatement implements
+			ICompiledDmlStatement {
+		private String dataverseName;
+		private String datasetName;
+		private boolean alreadySorted;
+		private String adapter;
+		private Map<String, String> properties;
+
+		public CompiledLoadFromFileStatement(String dataverseName,
+				String datasetName, String adapter,
+				Map<String, String> properties, boolean alreadySorted) {
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.alreadySorted = alreadySorted;
+			this.adapter = adapter;
+			this.properties = properties;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		public boolean alreadySorted() {
+			return alreadySorted;
+		}
+
+		public String getAdapter() {
+			return adapter;
+		}
+
+		public Map<String, String> getProperties() {
+			return properties;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.LOAD_FROM_FILE;
+		}
+	}
+
+	public static class CompiledInsertStatement implements
+			ICompiledDmlStatement {
+		private final String dataverseName;
+		private final String datasetName;
+		private final Query query;
+		private final int varCounter;
+
+		public CompiledInsertStatement(String dataverseName,
+				String datasetName, Query query, int varCounter) {
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.query = query;
+			this.varCounter = varCounter;
+		}
+
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		public int getVarCounter() {
+			return varCounter;
+		}
+
+		public Query getQuery() {
+			return query;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.INSERT;
+		}
+	}
+
+	public static class CompiledBeginFeedStatement implements
+			ICompiledDmlStatement {
+		private String dataverseName;
+		private String datasetName;
+		private Query query;
+		private int varCounter;
+
+		public CompiledBeginFeedStatement(String dataverseName,
+				String datasetName, Query query, int varCounter) {
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.query = query;
+			this.varCounter = varCounter;
+		}
+
+		@Override
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		@Override
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		public int getVarCounter() {
+			return varCounter;
+		}
+
+		public Query getQuery() {
+			return query;
+		}
+
+		public void setQuery(Query query) {
+			this.query = query;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.BEGIN_FEED;
+		}
+	}
+
+	public static class CompiledControlFeedStatement implements
+			ICompiledDmlStatement {
+		private String dataverseName;
+		private String datasetName;
+		private OperationType operationType;
+		private Query query;
+		private int varCounter;
+		private Map<String, String> alteredParams;
+
+		public CompiledControlFeedStatement(OperationType operationType,
+				String dataverseName, String datasetName,
+				Map<String, String> alteredParams) {
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.operationType = operationType;
+			this.alteredParams = alteredParams;
+		}
+
+		@Override
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		@Override
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		public OperationType getOperationType() {
+			return operationType;
+		}
+
+		public int getVarCounter() {
+			return varCounter;
+		}
+
+		public Query getQuery() {
+			return query;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.CONTROL_FEED;
+		}
+
+		public Map<String, String> getProperties() {
+			return alteredParams;
+		}
+
+		public void setProperties(Map<String, String> properties) {
+			this.alteredParams = properties;
+		}
+	}
+
+	public static class CompiledDeleteStatement implements
+			ICompiledDmlStatement {
+		private VariableExpr var;
+		private String dataverseName;
+		private String datasetName;
+		private Expression condition;
+		private Clause dieClause;
+		private int varCounter;
+		private AqlMetadataProvider metadataProvider;
+
+		public CompiledDeleteStatement(VariableExpr var, String dataverseName,
+				String datasetName, Expression condition, Clause dieClause,
+				int varCounter, AqlMetadataProvider metadataProvider) {
+			this.var = var;
+			this.dataverseName = dataverseName;
+			this.datasetName = datasetName;
+			this.condition = condition;
+			this.dieClause = dieClause;
+			this.varCounter = varCounter;
+			this.metadataProvider = metadataProvider;
+		}
+
+		@Override
+		public String getDatasetName() {
+			return datasetName;
+		}
+
+		@Override
+		public String getDataverseName() {
+			return dataverseName;
+		}
+
+		public int getVarCounter() {
+			return varCounter;
+		}
+
+		public Expression getCondition() {
+			return condition;
+		}
+
+		public Clause getDieClause() {
+			return dieClause;
+		}
+
+		public Query getQuery() throws AlgebricksException {
+
+			List<Expression> arguments = new ArrayList<Expression>();
+			String arg = dataverseName == null ? datasetName : dataverseName
+					+ "." + datasetName;
+			LiteralExpr argumentLiteral = new LiteralExpr(
+					new StringLiteral(arg));
+			arguments.add(argumentLiteral);
+
+			CallExpr callExpression = new CallExpr(new FunctionSignature(
+					FunctionConstants.ASTERIX_NS, "dataset", 1), arguments);
+			List<Clause> clauseList = new ArrayList<Clause>();
+			Clause forClause = new ForClause(var, callExpression);
+			clauseList.add(forClause);
+			Clause whereClause = null;
+			if (condition != null) {
+				whereClause = new WhereClause(condition);
+				clauseList.add(whereClause);
+			}
+			if (dieClause != null) {
+				clauseList.add(dieClause);
+			}
+
+			Dataset dataset = metadataProvider.findDataset(dataverseName,
+					datasetName);
+			if (dataset == null) {
+				throw new AlgebricksException("Unknown dataset " + datasetName);
+			}
+			String itemTypeName = dataset.getItemTypeName();
+			IAType itemType = metadataProvider.findType(
+					dataset.getDataverseName(), itemTypeName);
+			ARecordType recType = (ARecordType) itemType;
+			String[] fieldNames = recType.getFieldNames();
+			List<FieldBinding> fieldBindings = new ArrayList<FieldBinding>();
+			for (int i = 0; i < fieldNames.length; i++) {
+				FieldAccessor fa = new FieldAccessor(var, new Identifier(
+						fieldNames[i]));
+				FieldBinding fb = new FieldBinding(new LiteralExpr(
+						new StringLiteral(fieldNames[i])), fa);
+				fieldBindings.add(fb);
+			}
+			RecordConstructor rc = new RecordConstructor(fieldBindings);
+
+			FLWOGRExpression flowgr = new FLWOGRExpression(clauseList, rc);
+			Query query = new Query();
+			query.setBody(flowgr);
+			return query;
+		}
+
+		@Override
+		public Kind getKind() {
+			return Kind.DELETE;
+		}
+
+	}
+
+}
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
deleted file mode 100644
index c8b1079..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/DmlTranslator.java
+++ /dev/null
@@ -1,492 +0,0 @@
-package edu.uci.ics.asterix.translator;
-
-import java.rmi.RemoteException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.aql.base.Clause;
-import edu.uci.ics.asterix.aql.base.Expression;
-import edu.uci.ics.asterix.aql.base.Statement;
-import edu.uci.ics.asterix.aql.base.Statement.Kind;
-import edu.uci.ics.asterix.aql.expression.BeginFeedStatement;
-import edu.uci.ics.asterix.aql.expression.CallExpr;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement;
-import edu.uci.ics.asterix.aql.expression.ControlFeedStatement.OperationType;
-import edu.uci.ics.asterix.aql.expression.CreateIndexStatement;
-import edu.uci.ics.asterix.aql.expression.DeleteStatement;
-import edu.uci.ics.asterix.aql.expression.FLWOGRExpression;
-import edu.uci.ics.asterix.aql.expression.FieldAccessor;
-import edu.uci.ics.asterix.aql.expression.FieldBinding;
-import edu.uci.ics.asterix.aql.expression.ForClause;
-import edu.uci.ics.asterix.aql.expression.Identifier;
-import edu.uci.ics.asterix.aql.expression.InsertStatement;
-import edu.uci.ics.asterix.aql.expression.LiteralExpr;
-import edu.uci.ics.asterix.aql.expression.LoadFromFileStatement;
-import edu.uci.ics.asterix.aql.expression.Query;
-import edu.uci.ics.asterix.aql.expression.RecordConstructor;
-import edu.uci.ics.asterix.aql.expression.VariableExpr;
-import edu.uci.ics.asterix.aql.expression.WhereClause;
-import edu.uci.ics.asterix.aql.expression.WriteFromQueryResultStatement;
-import edu.uci.ics.asterix.aql.literal.StringLiteral;
-import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.metadata.IDatasetDetails;
-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.AqlCompiledMetadataDeclarations;
-import edu.uci.ics.asterix.metadata.entities.Dataset;
-import edu.uci.ics.asterix.metadata.entities.FeedDatasetDetails;
-import edu.uci.ics.asterix.metadata.entities.Index;
-import edu.uci.ics.asterix.om.functions.AsterixFunction;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.asterix.transaction.management.exception.ACIDException;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-
-public class DmlTranslator extends AbstractAqlTranslator {
-
-    private final MetadataTransactionContext mdTxnCtx;
-    private final List<Statement> aqlStatements;
-    private AqlCompiledMetadataDeclarations compiledDeclarations;
-    private List<ICompiledDmlStatement> compiledDmlStatements;
-
-    public DmlTranslator(MetadataTransactionContext mdTxnCtx, List<Statement> aqlStatements) {
-        this.mdTxnCtx = mdTxnCtx;
-        this.aqlStatements = aqlStatements;
-    }
-
-    public void translate() throws AlgebricksException, RemoteException, ACIDException, MetadataException {
-        compiledDeclarations = compileMetadata(mdTxnCtx, aqlStatements, true);
-        compiledDmlStatements = compileDmlStatements();
-    }
-
-    public AqlCompiledMetadataDeclarations getCompiledDeclarations() {
-        return compiledDeclarations;
-    }
-
-    public List<ICompiledDmlStatement> getCompiledDmlStatements() {
-        return compiledDmlStatements;
-    }
-
-    private List<ICompiledDmlStatement> compileDmlStatements() throws AlgebricksException, MetadataException {
-        List<ICompiledDmlStatement> dmlStatements = new ArrayList<ICompiledDmlStatement>();
-        for (Statement stmt : aqlStatements) {
-            validateOperation(compiledDeclarations, stmt);
-            switch (stmt.getKind()) {
-                case LOAD_FROM_FILE: {
-                    LoadFromFileStatement loadStmt = (LoadFromFileStatement) stmt;
-                    CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(loadStmt.getDatasetName()
-                            .getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
-                            loadStmt.dataIsAlreadySorted());
-                    dmlStatements.add(cls);
-                    // Also load the dataset's secondary indexes.
-                    List<Index> datasetIndexes = MetadataManager.INSTANCE.getDatasetIndexes(mdTxnCtx,
-                            compiledDeclarations.getDataverseName(), loadStmt.getDatasetName().getValue());
-                    for (Index index : datasetIndexes) {
-                        if (!index.isSecondaryIndex()) {
-                            continue;
-                        }
-                        // Create CompiledCreateIndexStatement from metadata entity 'index'.
-                        CompiledCreateIndexStatement cis = new CompiledCreateIndexStatement(index.getIndexName(),
-                                index.getDatasetName(), index.getKeyFieldNames(), index.getGramLength(),
-                                index.getIndexType());
-                        dmlStatements.add(cis);
-                    }
-                    break;
-                }
-                case WRITE_FROM_QUERY_RESULT: {
-                    WriteFromQueryResultStatement st1 = (WriteFromQueryResultStatement) stmt;
-                    CompiledWriteFromQueryResultStatement clfrqs = new CompiledWriteFromQueryResultStatement(st1
-                            .getDatasetName().getValue(), st1.getQuery(), st1.getVarCounter());
-                    dmlStatements.add(clfrqs);
-                    break;
-                }
-                case CREATE_INDEX: {
-                    CreateIndexStatement cis = (CreateIndexStatement) stmt;
-                    // Assumptions: We first processed the DDL, which added the secondary index to the metadata.
-                    // If the index's dataset is being loaded in this 'session', then let the load add 
-                    // the CompiledCreateIndexStatement to dmlStatements, and don't add it again here.
-                    // It's better to have the load handle this because:
-                    // 1. There may be more secondary indexes to load, which were possibly created in an earlier session.
-                    // 2. If the create index stmt came before the load stmt, then we would first create an empty index only to load it again later. 
-                    // This may cause problems because the index would be considered loaded (even though it was loaded empty). 
-                    for (Statement s : aqlStatements) {
-                        if (s.getKind() != Kind.LOAD_FROM_FILE) {
-                            continue;
-                        }
-                        LoadFromFileStatement loadStmt = (LoadFromFileStatement) s;
-                        if (loadStmt.getDatasetName().equals(cis.getDatasetName())) {
-                            cis.setNeedToCreate(false);
-                        }
-                    }
-                    if (cis.getNeedToCreate()) {
-                        CompiledCreateIndexStatement ccis = new CompiledCreateIndexStatement(cis.getIndexName()
-                                .getValue(), cis.getDatasetName().getValue(), cis.getFieldExprs(), cis.getGramLength(),
-                                cis.getIndexType());
-                        dmlStatements.add(ccis);
-                    }
-                    break;
-                }
-                case INSERT: {
-                    InsertStatement is = (InsertStatement) stmt;
-                    CompiledInsertStatement clfrqs = new CompiledInsertStatement(is.getDatasetName().getValue(),
-                            is.getQuery(), is.getVarCounter());
-                    dmlStatements.add(clfrqs);
-                    break;
-                }
-                case DELETE: {
-                    DeleteStatement ds = (DeleteStatement) stmt;
-                    CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(ds.getVariableExpr(),
-                            ds.getDatasetName(), ds.getCondition(), ds.getDieClause(), ds.getVarCounter(),
-                            compiledDeclarations);
-                    dmlStatements.add(clfrqs);
-                    break;
-                }
-
-                case BEGIN_FEED: {
-                    BeginFeedStatement bfs = (BeginFeedStatement) stmt;
-                    CompiledBeginFeedStatement cbfs = new CompiledBeginFeedStatement(bfs.getDatasetName(),
-                            bfs.getQuery(), bfs.getVarCounter());
-                    dmlStatements.add(cbfs);
-                    Dataset dataset;
-                    try {
-                        dataset = MetadataManager.INSTANCE.getDataset(mdTxnCtx,
-                                compiledDeclarations.getDataverseName(), bfs.getDatasetName().getValue());
-                    } catch (MetadataException me) {
-                        throw new AlgebricksException(me);
-                    }
-                    IDatasetDetails datasetDetails = dataset.getDatasetDetails();
-                    if (datasetDetails.getDatasetType() != DatasetType.FEED) {
-                        throw new IllegalArgumentException("Dataset " + bfs.getDatasetName().getValue()
-                                + " is not a feed dataset");
-                    }
-                    bfs.initialize((FeedDatasetDetails) datasetDetails);
-                    cbfs.setQuery(bfs.getQuery());
-                    break;
-                }
-
-                case CONTROL_FEED: {
-                    ControlFeedStatement cfs = (ControlFeedStatement) stmt;
-                    CompiledControlFeedStatement clcfs = new CompiledControlFeedStatement(cfs.getOperationType(),
-                            cfs.getDatasetName(), cfs.getAlterAdapterConfParams());
-                    dmlStatements.add(clcfs);
-                    break;
-
-                }
-            }
-        }
-        return dmlStatements;
-    }
-
-    public static interface ICompiledDmlStatement {
-
-        public abstract Kind getKind();
-    }
-
-    public static class CompiledCreateIndexStatement implements ICompiledDmlStatement {
-        private final String indexName;
-        private final String datasetName;
-        private final List<String> keyFields;
-        private final IndexType indexType;
-
-        // Specific to NGram index.
-        private final int gramLength;
-
-        public CompiledCreateIndexStatement(String indexName, String datasetName, List<String> keyFields,
-                int gramLength, IndexType indexType) {
-            this.indexName = indexName;
-            this.datasetName = datasetName;
-            this.keyFields = keyFields;
-            this.gramLength = gramLength;
-            this.indexType = indexType;
-        }
-
-        public String getDatasetName() {
-            return datasetName;
-        }
-
-        public String getIndexName() {
-            return indexName;
-        }
-
-        public List<String> getKeyFields() {
-            return keyFields;
-        }
-
-        public IndexType getIndexType() {
-            return indexType;
-        }
-
-        public int getGramLength() {
-            return gramLength;
-        }
-
-        @Override
-        public Kind getKind() {
-            return Kind.CREATE_INDEX;
-        }
-    }
-
-    public static class CompiledLoadFromFileStatement implements ICompiledDmlStatement {
-        private String datasetName;
-        private boolean alreadySorted;
-        private String adapter;
-        private Map<String, String> properties;
-
-        public CompiledLoadFromFileStatement(String datasetName, String adapter, Map<String, String> properties,
-                boolean alreadySorted) {
-            this.datasetName = datasetName;
-            this.alreadySorted = alreadySorted;
-            this.adapter = adapter;
-            this.properties = properties;
-        }
-
-        public String getDatasetName() {
-            return datasetName;
-        }
-
-        public boolean alreadySorted() {
-            return alreadySorted;
-        }
-
-        public String getAdapter() {
-            return adapter;
-        }
-
-        public Map<String, String> getProperties() {
-            return properties;
-        }
-
-        @Override
-        public Kind getKind() {
-            return Kind.LOAD_FROM_FILE;
-        }
-    }
-
-    public static class CompiledWriteFromQueryResultStatement implements ICompiledDmlStatement {
-
-        private String datasetName;
-        private Query query;
-        private int varCounter;
-
-        public CompiledWriteFromQueryResultStatement(String datasetName, Query query, int varCounter) {
-            this.datasetName = datasetName;
-            this.query = query;
-            this.varCounter = varCounter;
-        }
-
-        public String getDatasetName() {
-            return datasetName;
-        }
-
-        public int getVarCounter() {
-            return varCounter;
-        }
-
-        public Query getQuery() {
-            return query;
-        }
-
-        @Override
-        public Kind getKind() {
-            return Kind.WRITE_FROM_QUERY_RESULT;
-        }
-
-    }
-
-    public static class CompiledInsertStatement implements ICompiledDmlStatement {
-        private String datasetName;
-        private Query query;
-        private int varCounter;
-
-        public CompiledInsertStatement(String datasetName, Query query, int varCounter) {
-            this.datasetName = datasetName;
-            this.query = query;
-            this.varCounter = varCounter;
-        }
-
-        public String getDatasetName() {
-            return datasetName;
-        }
-
-        public int getVarCounter() {
-            return varCounter;
-        }
-
-        public Query getQuery() {
-            return query;
-        }
-
-        @Override
-        public Kind getKind() {
-            return Kind.INSERT;
-        }
-    }
-
-    public static class CompiledBeginFeedStatement implements ICompiledDmlStatement {
-        private Identifier datasetName;
-        private Query query;
-        private int varCounter;
-
-        public CompiledBeginFeedStatement(Identifier datasetName, Query query, int varCounter) {
-            this.datasetName = datasetName;
-            this.query = query;
-            this.varCounter = varCounter;
-        }
-
-        public Identifier getDatasetName() {
-            return datasetName;
-        }
-
-        public int getVarCounter() {
-            return varCounter;
-        }
-
-        public Query getQuery() {
-            return query;
-        }
-
-        public void setQuery(Query query) {
-            this.query = query;
-        }
-
-        @Override
-        public Kind getKind() {
-            return Kind.BEGIN_FEED;
-        }
-    }
-
-    public static class CompiledControlFeedStatement implements ICompiledDmlStatement {
-        private Identifier datasetName;
-        private OperationType operationType;
-        private Query query;
-        private int varCounter;
-        private Map<String, String> alteredParams;
-
-        public CompiledControlFeedStatement(OperationType operationType, Identifier datasetName,
-                Map<String, String> alteredParams) {
-            this.datasetName = datasetName;
-            this.operationType = operationType;
-            this.alteredParams = alteredParams;
-        }
-
-        public Identifier getDatasetName() {
-            return datasetName;
-        }
-
-        public OperationType getOperationType() {
-            return operationType;
-        }
-
-        public int getVarCounter() {
-            return varCounter;
-        }
-
-        public Query getQuery() {
-            return query;
-        }
-
-        @Override
-        public Kind getKind() {
-            return Kind.CONTROL_FEED;
-        }
-
-        public Map<String, String> getProperties() {
-            return alteredParams;
-        }
-
-        public void setProperties(Map<String, String> properties) {
-            this.alteredParams = properties;
-        }
-    }
-
-    public static class CompiledDeleteStatement implements ICompiledDmlStatement {
-        private VariableExpr var;
-        private Identifier dataset;
-        private Expression condition;
-        private Clause dieClause;
-        private int varCounter;
-        private AqlCompiledMetadataDeclarations compiledDeclarations;
-
-        public CompiledDeleteStatement(VariableExpr var, Identifier dataset, Expression condition, Clause dieClause,
-                int varCounter, AqlCompiledMetadataDeclarations compiledDeclarations) {
-            this.var = var;
-            this.dataset = dataset;
-            this.condition = condition;
-            this.dieClause = dieClause;
-            this.varCounter = varCounter;
-            this.compiledDeclarations = compiledDeclarations;
-        }
-
-        public Identifier getDataset() {
-            return dataset;
-        }
-
-        public String getDatasetName() {
-            return dataset.getValue();
-        }
-
-        public int getVarCounter() {
-            return varCounter;
-        }
-
-        public Expression getCondition() {
-            return condition;
-        }
-
-        public Clause getDieClause() {
-            return dieClause;
-        }
-
-        public Query getQuery() throws AlgebricksException {
-            String datasetName = dataset.getValue();
-
-            List<Expression> arguments = new ArrayList<Expression>();
-            LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(datasetName));
-            arguments.add(argumentLiteral);
-
-            CallExpr callExpression = new CallExpr(new AsterixFunction("dataset", 1), arguments);
-            List<Clause> clauseList = new ArrayList<Clause>();
-            Clause forClause = new ForClause(var, callExpression);
-            clauseList.add(forClause);
-            Clause whereClause = null;
-            if (condition != null) {
-                whereClause = new WhereClause(condition);
-                clauseList.add(whereClause);
-            }
-            if (dieClause != null) {
-                clauseList.add(dieClause);
-            }
-
-            Dataset dataset = compiledDeclarations.findDataset(datasetName);
-            if (dataset == null) {
-                throw new AlgebricksException("Unknown dataset " + datasetName);
-            }
-            String itemTypeName = dataset.getItemTypeName();
-            IAType itemType = compiledDeclarations.findType(itemTypeName);
-            ARecordType recType = (ARecordType) itemType;
-            String[] fieldNames = recType.getFieldNames();
-            List<FieldBinding> fieldBindings = new ArrayList<FieldBinding>();
-            for (int i = 0; i < fieldNames.length; i++) {
-                FieldAccessor fa = new FieldAccessor(var, new Identifier(fieldNames[i]));
-                FieldBinding fb = new FieldBinding(new LiteralExpr(new StringLiteral(fieldNames[i])), fa);
-                fieldBindings.add(fb);
-            }
-            RecordConstructor rc = new RecordConstructor(fieldBindings);
-
-            FLWOGRExpression flowgr = new FLWOGRExpression(clauseList, rc);
-            Query query = new Query();
-            query.setBody(flowgr);
-            return query;
-        }
-
-        @Override
-        public Kind getKind() {
-            return Kind.DELETE;
-        }
-
-    }
-
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/MetadataDeclTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/MetadataDeclTranslator.java
deleted file mode 100644
index 77b8a04..0000000
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/MetadataDeclTranslator.java
+++ /dev/null
@@ -1,366 +0,0 @@
-package edu.uci.ics.asterix.translator;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import edu.uci.ics.asterix.aql.expression.OrderedListTypeDefinition;
-import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition;
-import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition.RecordKind;
-import edu.uci.ics.asterix.aql.expression.TypeDecl;
-import edu.uci.ics.asterix.aql.expression.TypeExpression;
-import edu.uci.ics.asterix.aql.expression.TypeReferenceExpression;
-import edu.uci.ics.asterix.aql.expression.UnorderedListTypeDefinition;
-import edu.uci.ics.asterix.common.annotations.IRecordFieldDataGen;
-import edu.uci.ics.asterix.common.annotations.RecordDataGenAnnotation;
-import edu.uci.ics.asterix.common.annotations.TypeDataGen;
-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.bootstrap.AsterixProperties;
-import edu.uci.ics.asterix.metadata.declared.AqlCompiledMetadataDeclarations;
-import edu.uci.ics.asterix.metadata.entities.AsterixBuiltinTypeMap;
-import edu.uci.ics.asterix.metadata.entities.Datatype;
-import edu.uci.ics.asterix.om.types.AOrderedListType;
-import edu.uci.ics.asterix.om.types.ARecordType;
-import edu.uci.ics.asterix.om.types.AUnionType;
-import edu.uci.ics.asterix.om.types.AUnorderedListType;
-import edu.uci.ics.asterix.om.types.AbstractCollectionType;
-import edu.uci.ics.asterix.om.types.BuiltinType;
-import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
-import edu.uci.ics.hyracks.algebricks.data.IAWriterFactory;
-import edu.uci.ics.hyracks.dataflow.std.file.FileSplit;
-
-public final class MetadataDeclTranslator {
-    private final MetadataTransactionContext mdTxnCtx;
-    private final String dataverseName;
-    private final List<TypeDecl> typeDeclarations;
-    private final FileSplit outputFile;
-    private final Map<String, String> config;
-    private final IAWriterFactory writerFactory;
-
-    public MetadataDeclTranslator(MetadataTransactionContext mdTxnCtx, String dataverseName, FileSplit outputFile,
-            IAWriterFactory writerFactory, Map<String, String> config, List<TypeDecl> typeDeclarations) {
-        this.mdTxnCtx = mdTxnCtx;
-        this.dataverseName = dataverseName;
-        this.outputFile = outputFile;
-        this.writerFactory = writerFactory;
-        this.config = config;
-        this.typeDeclarations = typeDeclarations;
-    }
-
-    // TODO: Should this not throw an AsterixException?
-    public AqlCompiledMetadataDeclarations computeMetadataDeclarations(boolean online) throws AlgebricksException,
-            MetadataException {
-        Map<String, TypeDataGen> typeDataGenMap = new HashMap<String, TypeDataGen>();
-        for (TypeDecl td : typeDeclarations) {
-            TypeDataGen tdg = td.getDatagenAnnotation();
-            if (tdg != null) {
-                typeDataGenMap.put(td.getIdent().getValue(), tdg);
-            }
-        }
-        Map<String, IAType> typeMap = computeTypes();
-        Map<String, String[]> stores = AsterixProperties.INSTANCE.getStores();
-        return new AqlCompiledMetadataDeclarations(mdTxnCtx, dataverseName, outputFile, config, stores, typeMap,
-                typeDataGenMap, writerFactory, online);
-    }
-
-    private Map<String, IAType> computeTypes() throws AlgebricksException, MetadataException {
-        Map<String, IAType> typeMap = new HashMap<String, IAType>();
-        Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
-        Map<String, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<String, List<AbstractCollectionType>>();
-        Map<String, List<String>> incompleteTopLevelTypeReferences = new HashMap<String, List<String>>();
-
-        firstPass(typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
-        secondPass(typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences);
-        return typeMap;
-    }
-
-    private void secondPass(Map<String, IAType> typeMap,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException, MetadataException {
-        // solve remaining top level references
-        for (String trefName : incompleteTopLevelTypeReferences.keySet()) {
-            IAType t = typeMap.get(trefName);
-            if (t == null) {
-                throw new AlgebricksException("Could not resolve type " + trefName);
-            }
-            for (String tname : incompleteTopLevelTypeReferences.get(trefName)) {
-                typeMap.put(tname, t);
-            }
-        }
-        // solve remaining field type references
-        for (String trefName : incompleteFieldTypes.keySet()) {
-            IAType t = typeMap.get(trefName);
-            if (t == null) {
-                // Try to get type from the metadata manager.
-                Datatype metadataDataType = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, dataverseName, trefName);
-                if (metadataDataType == null) {
-                    throw new AlgebricksException("Could not resolve type " + trefName);
-                }
-                t = metadataDataType.getDatatype();
-                typeMap.put(trefName, t);
-            }
-            Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes.get(trefName);
-            for (ARecordType recType : fieldsToFix.keySet()) {
-                List<Integer> positions = fieldsToFix.get(recType);
-                IAType[] fldTypes = recType.getFieldTypes();
-                for (Integer pos : positions) {
-                    if (fldTypes[pos] == null) {
-                        fldTypes[pos] = t;
-                    } else { // nullable
-                        AUnionType nullableUnion = (AUnionType) fldTypes[pos];
-                        nullableUnion.setTypeAtIndex(t, 1);
-                    }
-                }
-            }
-        }
-        // solve remaining item type references
-        for (String trefName : incompleteItemTypes.keySet()) {
-            IAType t = typeMap.get(trefName);
-            if (t == null) {
-                throw new AlgebricksException("Could not resolve type " + trefName);
-            }
-            for (AbstractCollectionType act : incompleteItemTypes.get(trefName)) {
-                act.setItemType(t);
-            }
-        }
-    }
-
-    private void firstPass(Map<String, IAType> typeMap,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, List<String>> incompleteTopLevelTypeReferences) throws AlgebricksException {
-        for (TypeDecl td : typeDeclarations) {
-            TypeExpression texpr = td.getTypeDef();
-            String tdname = td.getIdent().getValue();
-            if (AsterixBuiltinTypeMap.getBuiltinTypes().get(tdname) != null) {
-                throw new AlgebricksException("Cannot redefine builtin type " + tdname + " .");
-            }
-            switch (texpr.getTypeKind()) {
-                case TYPEREFERENCE: {
-                    TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
-                    IAType t = solveTypeReference(tre, typeMap);
-                    if (t != null) {
-                        typeMap.put(tdname, t);
-                    } else {
-                        addIncompleteTopLevelTypeReference(tdname, tre, incompleteTopLevelTypeReferences);
-                    }
-                    break;
-                }
-                case RECORD: {
-                    RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
-                    ARecordType recType = computeRecordType(tdname, rtd, typeMap, incompleteFieldTypes,
-                            incompleteItemTypes);
-                    typeMap.put(tdname, recType);
-                    break;
-                }
-                case ORDEREDLIST: {
-                    OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
-                    AOrderedListType olType = computeOrderedListType(tdname, oltd, typeMap, incompleteItemTypes,
-                            incompleteFieldTypes);
-                    typeMap.put(tdname, olType);
-                    break;
-                }
-                case UNORDEREDLIST: {
-                    UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
-                    AUnorderedListType ulType = computeUnorderedListType(tdname, ultd, typeMap, incompleteItemTypes,
-                            incompleteFieldTypes);
-                    typeMap.put(tdname, ulType);
-                    break;
-                }
-                default: {
-                    throw new IllegalStateException();
-                }
-            }
-        }
-    }
-
-    private AOrderedListType computeOrderedListType(String typeName, OrderedListTypeDefinition oltd,
-            Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-        TypeExpression tExpr = oltd.getItemTypeExpression();
-        AOrderedListType aolt = new AOrderedListType(null, typeName);
-        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, aolt);
-        return aolt;
-    }
-
-    private AUnorderedListType computeUnorderedListType(String typeName, UnorderedListTypeDefinition ultd,
-            Map<String, IAType> typeMap, Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-        TypeExpression tExpr = ultd.getItemTypeExpression();
-        AUnorderedListType ault = new AUnorderedListType(null, typeName);
-        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, ault);
-        return ault;
-    }
-
-    private void setCollectionItemType(TypeExpression tExpr, Map<String, IAType> typeMap,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, AbstractCollectionType act) {
-        switch (tExpr.getTypeKind()) {
-            case ORDEREDLIST: {
-                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
-                IAType t = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                act.setItemType(t);
-                break;
-            }
-            case UNORDEREDLIST: {
-                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
-                IAType t = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                act.setItemType(t);
-                break;
-            }
-            case RECORD: {
-                RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
-                IAType t = computeRecordType(null, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes);
-                act.setItemType(t);
-                break;
-            }
-            case TYPEREFERENCE: {
-                TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
-                IAType tref = solveTypeReference(tre, typeMap);
-                if (tref != null) {
-                    act.setItemType(tref);
-                } else {
-                    addIncompleteCollectionTypeReference(act, tre, incompleteItemTypes);
-                }
-                break;
-            }
-            default: {
-                throw new IllegalStateException();
-            }
-        }
-    }
-
-    private ARecordType computeRecordType(String typeName, RecordTypeDefinition rtd, Map<String, IAType> typeMap,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
-        List<String> names = rtd.getFieldNames();
-        int n = names.size();
-        String[] fldNames = new String[n];
-        IAType[] fldTypes = new IAType[n];
-        int i = 0;
-        for (String s : names) {
-            fldNames[i++] = s;
-        }
-        boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
-        ARecordType recType = new ARecordType(typeName, fldNames, fldTypes, isOpen);
-
-        List<IRecordFieldDataGen> fieldDataGen = rtd.getFieldDataGen();
-        if (fieldDataGen.size() == n) {
-            IRecordFieldDataGen[] rfdg = new IRecordFieldDataGen[n];
-            rfdg = fieldDataGen.toArray(rfdg);
-            recType.getAnnotations().add(new RecordDataGenAnnotation(rfdg, rtd.getUndeclaredFieldsDataGen()));
-        }
-
-        for (int j = 0; j < n; j++) {
-            TypeExpression texpr = rtd.getFieldTypes().get(j);
-            switch (texpr.getTypeKind()) {
-                case TYPEREFERENCE: {
-                    TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
-                    IAType tref = solveTypeReference(tre, typeMap);
-                    if (tref != null) {
-                        if (!rtd.getNullableFields().get(j)) { // not nullable
-                            fldTypes[j] = tref;
-                        } else { // nullable
-                            fldTypes[j] = makeUnionWithNull(null, tref);
-                        }
-                    } else {
-                        addIncompleteFieldTypeReference(recType, j, tre, incompleteFieldTypes);
-                        if (rtd.getNullableFields().get(j)) {
-                            fldTypes[j] = makeUnionWithNull(null, null);
-                        }
-                    }
-                    break;
-                }
-                case RECORD: {
-                    RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
-                    IAType t2 = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes, incompleteItemTypes);
-                    if (!rtd.getNullableFields().get(j)) { // not nullable
-                        fldTypes[j] = t2;
-                    } else { // nullable
-                        fldTypes[j] = makeUnionWithNull(null, t2);
-                    }
-                    break;
-                }
-                case ORDEREDLIST: {
-                    OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
-                    IAType t2 = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
-                    break;
-                }
-                case UNORDEREDLIST: {
-                    UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
-                    IAType t2 = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes);
-                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
-                    break;
-                }
-                default: {
-                    throw new IllegalStateException();
-                }
-            }
-
-        }
-
-        return recType;
-    }
-
-    private AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
-        ArrayList<IAType> unionList = new ArrayList<IAType>(2);
-        unionList.add(BuiltinType.ANULL);
-        unionList.add(type);
-        return new AUnionType(unionList, unionTypeName);
-    }
-
-    private void addIncompleteCollectionTypeReference(AbstractCollectionType collType, TypeReferenceExpression tre,
-            Map<String, List<AbstractCollectionType>> incompleteItemTypes) {
-        String typeName = tre.getIdent().getValue();
-        List<AbstractCollectionType> typeList = incompleteItemTypes.get(typeName);
-        if (typeList == null) {
-            typeList = new LinkedList<AbstractCollectionType>();
-            incompleteItemTypes.put(typeName, typeList);
-        }
-        typeList.add(collType);
-    }
-
-    private void addIncompleteFieldTypeReference(ARecordType recType, int fldPosition, TypeReferenceExpression tre,
-            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
-        String typeName = tre.getIdent().getValue();
-        Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes.get(typeName);
-        if (refMap == null) {
-            refMap = new HashMap<ARecordType, List<Integer>>();
-            incompleteFieldTypes.put(typeName, refMap);
-        }
-        List<Integer> typeList = refMap.get(recType);
-        if (typeList == null) {
-            typeList = new ArrayList<Integer>();
-            refMap.put(recType, typeList);
-        }
-        typeList.add(fldPosition);
-    }
-
-    private void addIncompleteTopLevelTypeReference(String tdeclName, TypeReferenceExpression tre,
-            Map<String, List<String>> incompleteTopLevelTypeReferences) {
-        String name = tre.getIdent().getValue();
-        List<String> refList = incompleteTopLevelTypeReferences.get(name);
-        if (refList == null) {
-            refList = new LinkedList<String>();
-            incompleteTopLevelTypeReferences.put(name, refList);
-        }
-        refList.add(tdeclName);
-    }
-
-    private IAType solveTypeReference(TypeReferenceExpression tre, Map<String, IAType> typeMap) {
-        String name = tre.getIdent().getValue();
-        IAType builtin = AsterixBuiltinTypeMap.getBuiltinTypes().get(name);
-        if (builtin != null) {
-            return builtin;
-        } else {
-            return typeMap.get(name);
-        }
-    }
-}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/TypeTranslator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/TypeTranslator.java
new file mode 100644
index 0000000..4047d9a
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/translator/TypeTranslator.java
@@ -0,0 +1,391 @@
+/*
+ * 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.translator;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import edu.uci.ics.asterix.aql.expression.OrderedListTypeDefinition;
+import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition;
+import edu.uci.ics.asterix.aql.expression.RecordTypeDefinition.RecordKind;
+import edu.uci.ics.asterix.aql.expression.TypeDecl;
+import edu.uci.ics.asterix.aql.expression.TypeExpression;
+import edu.uci.ics.asterix.aql.expression.TypeReferenceExpression;
+import edu.uci.ics.asterix.aql.expression.UnorderedListTypeDefinition;
+import edu.uci.ics.asterix.common.annotations.IRecordFieldDataGen;
+import edu.uci.ics.asterix.common.annotations.RecordDataGenAnnotation;
+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.entities.AsterixBuiltinTypeMap;
+import edu.uci.ics.asterix.metadata.entities.Datatype;
+import edu.uci.ics.asterix.om.types.AOrderedListType;
+import edu.uci.ics.asterix.om.types.ARecordType;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.types.TypeSignature;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+
+public class TypeTranslator {
+
+    public static Map<TypeSignature, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeDecl tDec,
+            String defaultDataverse) throws AlgebricksException, MetadataException {
+        Map<TypeSignature, IAType> typeMap = new HashMap<TypeSignature, IAType>();
+        Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
+        Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<TypeSignature, List<AbstractCollectionType>>();
+        Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences = new HashMap<TypeSignature, List<TypeSignature>>();
+        String typeDataverse = tDec.getDataverseName() == null ? defaultDataverse : tDec.getDataverseName().getValue();
+        firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences,
+                typeDataverse);
+        secondPass(mdTxnCtx, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences,
+                typeDataverse);
+
+        return typeMap;
+    }
+
+    public static Map<TypeSignature, IAType> computeTypes(MetadataTransactionContext mdTxnCtx, TypeDecl tDec,
+            String defaultDataverse, Map<TypeSignature, IAType> typeMap) throws AlgebricksException, MetadataException {
+        Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes = new HashMap<String, Map<ARecordType, List<Integer>>>();
+        Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes = new HashMap<TypeSignature, List<AbstractCollectionType>>();
+        Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences = new HashMap<TypeSignature, List<TypeSignature>>();
+        String typeDataverse = tDec.getDataverseName() == null ? defaultDataverse : tDec.getDataverseName().getValue();
+        firstPass(tDec, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences,
+                typeDataverse);
+        secondPass(mdTxnCtx, typeMap, incompleteFieldTypes, incompleteItemTypes, incompleteTopLevelTypeReferences,
+                typeDataverse);
+
+        return typeMap;
+    }
+
+    private static Map<String, BuiltinType> builtinTypeMap = AsterixBuiltinTypeMap.getBuiltinTypes();
+
+    private static void firstPass(TypeDecl td, Map<TypeSignature, IAType> typeMap,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse)
+            throws AlgebricksException {
+
+        TypeExpression texpr = td.getTypeDef();
+        String tdname = td.getIdent().getValue();
+        if (builtinTypeMap.get(tdname) != null) {
+            throw new AlgebricksException("Cannot redefine builtin type " + tdname + " .");
+        }
+        TypeSignature typeSignature = new TypeSignature(typeDataverse, tdname);
+        switch (texpr.getTypeKind()) {
+            case TYPEREFERENCE: {
+                TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+                IAType t = solveTypeReference(typeSignature, typeMap);
+                if (t != null) {
+                    typeMap.put(typeSignature, t);
+                } else {
+                    addIncompleteTopLevelTypeReference(tdname, tre, incompleteTopLevelTypeReferences, typeDataverse);
+                }
+                break;
+            }
+            case RECORD: {
+                RecordTypeDefinition rtd = (RecordTypeDefinition) texpr;
+                ARecordType recType = computeRecordType(typeSignature, rtd, typeMap, incompleteFieldTypes,
+                        incompleteItemTypes, typeDataverse);
+                typeMap.put(typeSignature, recType);
+                break;
+            }
+            case ORDEREDLIST: {
+                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+                AOrderedListType olType = computeOrderedListType(typeSignature, oltd, typeMap, incompleteItemTypes,
+                        incompleteFieldTypes, typeDataverse);
+                typeMap.put(typeSignature, olType);
+                break;
+            }
+            case UNORDEREDLIST: {
+                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+                AUnorderedListType ulType = computeUnorderedListType(typeSignature, ultd, typeMap, incompleteItemTypes,
+                        incompleteFieldTypes, typeDataverse);
+                typeMap.put(typeSignature, ulType);
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
+
+    private static void secondPass(MetadataTransactionContext mdTxnCtx, Map<TypeSignature, IAType> typeMap,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String typeDataverse)
+            throws AlgebricksException, MetadataException {
+        // solve remaining top level references
+
+        for (TypeSignature typeSignature : incompleteTopLevelTypeReferences.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
+                    typeSignature.getName());
+            if (dt == null) {
+                throw new AlgebricksException("Could not resolve type " + typeSignature);
+            } else
+                t = dt.getDatatype();
+            for (TypeSignature sign : incompleteTopLevelTypeReferences.get(typeSignature)) {
+                typeMap.put(sign, t);
+            }
+        }
+        // solve remaining field type references
+        for (String trefName : incompleteFieldTypes.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeDataverse, trefName);
+            if (dt == null) {
+                throw new AlgebricksException("Could not resolve type " + trefName);
+            } else
+                t = dt.getDatatype();
+            Map<ARecordType, List<Integer>> fieldsToFix = incompleteFieldTypes.get(trefName);
+            for (ARecordType recType : fieldsToFix.keySet()) {
+                List<Integer> positions = fieldsToFix.get(recType);
+                IAType[] fldTypes = recType.getFieldTypes();
+                for (Integer pos : positions) {
+                    if (fldTypes[pos] == null) {
+                        fldTypes[pos] = t;
+                    } else { // nullable
+                        AUnionType nullableUnion = (AUnionType) fldTypes[pos];
+                        nullableUnion.setTypeAtIndex(t, 1);
+                    }
+                }
+            }
+        }
+
+        // solve remaining item type references
+        for (TypeSignature typeSignature : incompleteItemTypes.keySet()) {
+            IAType t;// = typeMap.get(trefName);
+            Datatype dt = null;
+            if (MetadataManager.INSTANCE != null) {
+                dt = MetadataManager.INSTANCE.getDatatype(mdTxnCtx, typeSignature.getNamespace(),
+                        typeSignature.getName());
+                if (dt == null) {
+                    throw new AlgebricksException("Could not resolve type " + typeSignature);
+                }
+                t = dt.getDatatype();
+            } else {
+                t = typeMap.get(typeSignature);   
+            }
+            for (AbstractCollectionType act : incompleteItemTypes.get(typeSignature)) {
+                act.setItemType(t);
+            }
+        }
+    }
+
+    private static AOrderedListType computeOrderedListType(TypeSignature typeSignature, OrderedListTypeDefinition oltd,
+            Map<TypeSignature, IAType> typeMap, Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, String defaultDataverse) {
+        TypeExpression tExpr = oltd.getItemTypeExpression();
+        String typeName = typeSignature != null ? typeSignature.getName() : null;
+        AOrderedListType aolt = new AOrderedListType(null, typeName);
+        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, aolt, defaultDataverse);
+        return aolt;
+    }
+
+    private static AUnorderedListType computeUnorderedListType(TypeSignature typeSignature,
+            UnorderedListTypeDefinition ultd, Map<TypeSignature, IAType> typeMap,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, String defaulDataverse) {
+        TypeExpression tExpr = ultd.getItemTypeExpression();
+        String typeName = typeSignature != null ? typeSignature.getName() : null;
+        AUnorderedListType ault = new AUnorderedListType(null, typeName);
+        setCollectionItemType(tExpr, typeMap, incompleteItemTypes, incompleteFieldTypes, ault, defaulDataverse);
+        return ault;
+    }
+
+    private static void setCollectionItemType(TypeExpression tExpr, Map<TypeSignature, IAType> typeMap,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes, AbstractCollectionType act,
+            String defaultDataverse) {
+        switch (tExpr.getTypeKind()) {
+            case ORDEREDLIST: {
+                OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) tExpr;
+                IAType t = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+                        defaultDataverse);
+                act.setItemType(t);
+                break;
+            }
+            case UNORDEREDLIST: {
+                UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) tExpr;
+                IAType t = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+                        defaultDataverse);
+                act.setItemType(t);
+                break;
+            }
+            case RECORD: {
+                RecordTypeDefinition rtd = (RecordTypeDefinition) tExpr;
+                IAType t = computeRecordType(null, rtd, typeMap, incompleteFieldTypes, incompleteItemTypes,
+                        defaultDataverse);
+                act.setItemType(t);
+                break;
+            }
+            case TYPEREFERENCE: {
+                TypeReferenceExpression tre = (TypeReferenceExpression) tExpr;
+                TypeSignature signature = new TypeSignature(defaultDataverse, tre.getIdent().getValue());
+                IAType tref = solveTypeReference(signature, typeMap);
+                if (tref != null) {
+                    act.setItemType(tref);
+                } else {
+                    addIncompleteCollectionTypeReference(act, tre, incompleteItemTypes, defaultDataverse);
+                }
+                break;
+            }
+            default: {
+                throw new IllegalStateException();
+            }
+        }
+    }
+
+    private static void addIncompleteCollectionTypeReference(AbstractCollectionType collType,
+            TypeReferenceExpression tre, Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes,
+            String defaultDataverse) {
+        String typeName = tre.getIdent().getValue();
+        TypeSignature typeSignature = new TypeSignature(defaultDataverse, typeName);
+        List<AbstractCollectionType> typeList = incompleteItemTypes.get(typeName);
+        if (typeList == null) {
+            typeList = new LinkedList<AbstractCollectionType>();
+            incompleteItemTypes.put(typeSignature, typeList);
+        }
+        typeList.add(collType);
+    }
+
+    private static void addIncompleteFieldTypeReference(ARecordType recType, int fldPosition,
+            TypeReferenceExpression tre, Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes) {
+        String typeName = tre.getIdent().getValue();
+        Map<ARecordType, List<Integer>> refMap = incompleteFieldTypes.get(typeName);
+        if (refMap == null) {
+            refMap = new HashMap<ARecordType, List<Integer>>();
+            incompleteFieldTypes.put(typeName, refMap);
+        }
+        List<Integer> typeList = refMap.get(recType);
+        if (typeList == null) {
+            typeList = new ArrayList<Integer>();
+            refMap.put(recType, typeList);
+        }
+        typeList.add(fldPosition);
+    }
+
+    private static void addIncompleteTopLevelTypeReference(String tdeclName, TypeReferenceExpression tre,
+            Map<TypeSignature, List<TypeSignature>> incompleteTopLevelTypeReferences, String defaultDataverse) {
+        String name = tre.getIdent().getValue();
+        TypeSignature typeSignature = new TypeSignature(defaultDataverse, name);
+        List<TypeSignature> refList = incompleteTopLevelTypeReferences.get(name);
+        if (refList == null) {
+            refList = new LinkedList<TypeSignature>();
+            incompleteTopLevelTypeReferences.put(new TypeSignature(defaultDataverse, tre.getIdent().getValue()),
+                    refList);
+        }
+        refList.add(typeSignature);
+    }
+
+    private static IAType solveTypeReference(TypeSignature typeSignature, Map<TypeSignature, IAType> typeMap) {
+        IAType builtin = builtinTypeMap.get(typeSignature.getName());
+        if (builtin != null) {
+            return builtin;
+        } else {
+            return typeMap.get(typeSignature);
+        }
+    }
+
+    private static ARecordType computeRecordType(TypeSignature typeSignature, RecordTypeDefinition rtd,
+            Map<TypeSignature, IAType> typeMap, Map<String, Map<ARecordType, List<Integer>>> incompleteFieldTypes,
+            Map<TypeSignature, List<AbstractCollectionType>> incompleteItemTypes, String defaultDataverse) {
+        List<String> names = rtd.getFieldNames();
+        int n = names.size();
+        String[] fldNames = new String[n];
+        IAType[] fldTypes = new IAType[n];
+        int i = 0;
+        for (String s : names) {
+            fldNames[i++] = s;
+        }
+        boolean isOpen = rtd.getRecordKind() == RecordKind.OPEN;
+        ARecordType recType = new ARecordType(typeSignature == null ? null : typeSignature.getName(), fldNames,
+                fldTypes, isOpen);
+        
+        List<IRecordFieldDataGen> fieldDataGen = rtd.getFieldDataGen();
+        if (fieldDataGen.size() == n) {
+            IRecordFieldDataGen[] rfdg = new IRecordFieldDataGen[n];
+            rfdg = fieldDataGen.toArray(rfdg);
+            recType.getAnnotations().add(new RecordDataGenAnnotation(rfdg, rtd.getUndeclaredFieldsDataGen()));
+        }
+        
+        for (int j = 0; j < n; j++) {
+            TypeExpression texpr = rtd.getFieldTypes().get(j);
+            switch (texpr.getTypeKind()) {
+                case TYPEREFERENCE: {
+                    TypeReferenceExpression tre = (TypeReferenceExpression) texpr;
+                    TypeSignature signature = new TypeSignature(defaultDataverse, tre.getIdent().getValue());
+                    IAType tref = solveTypeReference(signature, typeMap);
+                    if (tref != null) {
+                        if (!rtd.getNullableFields().get(j)) { // not nullable
+                            fldTypes[j] = tref;
+                        } else { // nullable
+                            fldTypes[j] = makeUnionWithNull(null, tref);
+                        }
+                    } else {
+                        addIncompleteFieldTypeReference(recType, j, tre, incompleteFieldTypes);
+                        if (rtd.getNullableFields().get(j)) {
+                            fldTypes[j] = makeUnionWithNull(null, null);
+                        }
+                    }
+                    break;
+                }
+                case RECORD: {
+                    RecordTypeDefinition recTypeDef2 = (RecordTypeDefinition) texpr;
+                    IAType t2 = computeRecordType(null, recTypeDef2, typeMap, incompleteFieldTypes,
+                            incompleteItemTypes, defaultDataverse);
+                    if (!rtd.getNullableFields().get(j)) { // not nullable
+                        fldTypes[j] = t2;
+                    } else { // nullable
+                        fldTypes[j] = makeUnionWithNull(null, t2);
+                    }
+                    break;
+                }
+                case ORDEREDLIST: {
+                    OrderedListTypeDefinition oltd = (OrderedListTypeDefinition) texpr;
+                    IAType t2 = computeOrderedListType(null, oltd, typeMap, incompleteItemTypes, incompleteFieldTypes,
+                            defaultDataverse);
+                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
+                    break;
+                }
+                case UNORDEREDLIST: {
+                    UnorderedListTypeDefinition ultd = (UnorderedListTypeDefinition) texpr;
+                    IAType t2 = computeUnorderedListType(null, ultd, typeMap, incompleteItemTypes,
+                            incompleteFieldTypes, defaultDataverse);
+                    fldTypes[j] = (rtd.getNullableFields().get(j)) ? makeUnionWithNull(null, t2) : t2;
+                    break;
+                }
+                default: {
+                    throw new IllegalStateException();
+                }
+            }
+
+        }
+
+        return recType;
+    }
+
+    private static AUnionType makeUnionWithNull(String unionTypeName, IAType type) {
+        ArrayList<IAType> unionList = new ArrayList<IAType>(2);
+        unionList.add(BuiltinType.ANULL);
+        unionList.add(type);
+        return new AUnionType(unionList, unionTypeName);
+    }
+}
diff --git a/asterix-algebra/src/main/javacc/AQLPlus.jj b/asterix-algebra/src/main/javacc/AQLPlus.jj
index 3422652..5b97d04 100644
--- a/asterix-algebra/src/main/javacc/AQLPlus.jj
+++ b/asterix-algebra/src/main/javacc/AQLPlus.jj
@@ -26,7 +26,6 @@
 import edu.uci.ics.asterix.aql.literal.NullLiteral;
 import edu.uci.ics.asterix.aql.literal.StringLiteral;
 import edu.uci.ics.asterix.aql.literal.TrueLiteral;
-
 import edu.uci.ics.asterix.aql.parser.ScopeChecker;
 import edu.uci.ics.asterix.aql.base.*;
 import edu.uci.ics.asterix.aql.expression.*;
@@ -36,8 +35,15 @@
 import edu.uci.ics.asterix.aql.context.Scope;
 import edu.uci.ics.asterix.aql.context.RootScopeFactory;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
 import edu.uci.ics.asterix.om.functions.AsterixFunction;
+import edu.uci.ics.asterix.common.functions.FunctionSignature;
+import edu.uci.ics.asterix.metadata.bootstrap.MetadataConstants;
+
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IExpressionAnnotation;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IndexedNLJoinExpressionAnnotation;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+
+
 
 public class AQLPlusParser extends ScopeChecker {
 
@@ -57,6 +63,8 @@
     private static final String HASH_GROUP_BY_HINT = "hash";
     private static final String BROADCAST_JOIN_HINT = "bcast";
     private static final String INMEMORY_HINT = "inmem";
+    private static final String INDEXED_NESTED_LOOP_JOIN_HINT = "indexnl";
+   
     
    
     private static String getHint(Token t) {
@@ -75,17 +83,7 @@
 			File file = new File(args[0]);
 			Reader fis = new BufferedReader(new InputStreamReader(new FileInputStream(file), "UTF-8"));
 		    AQLPlusParser parser = new AQLPlusParser(fis);
-		    Statement st = parser.Statement();
-		    st.accept(new AQLPrintVisitor(), 0);
-		    
-//		    System.out.println("FunctionCalls not found:");
-//		    for(FunctionDescriptor fd: notFoundFunctionList)
-//		    {
-//			if(lookupFunctionSignature(fd.getValue(), fd.getArity())!=null)
-//				notFoundFunctionList.remove(fd);
-//		    }
-//			System.out.println(notFoundFunctionList.toString());
-
+		    List<Statement> st = parser.Statement();
 	}
 
     public void initScope() {
@@ -96,7 +94,7 @@
 PARSER_END(AQLPlusParser)
 
 
-Statement Statement() throws ParseException:
+List<Statement> Statement() throws ParseException:
 {
   Query query = null;
   // scopeStack.push(RootScopeFactory.createRootScope(this));
@@ -129,26 +127,20 @@
           | "set" {
                        decls.add(SetStatement());
                     }
+          |          
+            {
+             decls.add(Query()) ;
+           } ";"
                                                      
+                                                      
         )*
-        (query = Query())?
       )
 
       <EOF>
     )
     {
-      if (query == null) {
-        query = new Query(true);
-      }
-      query.setPrologDeclList(decls);
-    
-//        for(FunctionDecl fdc : fdList)
-//        {
-//        	FunctionDescriptor fd = (FunctionDescriptor) fdc.getIdent();
-//                notFoundFunctionList.remove(fd);
-//        }
-// }
-      return query;
+     
+     return decls;  
     }
 }
 
@@ -187,7 +179,7 @@
      <DATASET> <IDENTIFIER> { datasetName = new Identifier(token.image); }
      <LEFTPAREN> query = Query() <RIGHTPAREN>
      {
-        stmt = new WriteFromQueryResultStatement(datasetName, query, getVarCounter());
+        stmt = new WriteFromQueryResultStatement(null, datasetName, query, getVarCounter());
      } ))  
         
     ";"
@@ -201,10 +193,10 @@
   Identifier dvName = null;  
 }
 {
-  "dataverse" <IDENTIFIER> { dvName = new Identifier(token.image); }
+  "dataverse" <IDENTIFIER> { defaultDataverse = token.image;}
   ";"
   {
-    return new DataverseDecl(dvName);
+    return new DataverseDecl(new Identifier(defaultDataverse));
   }
 }
 
@@ -272,7 +264,7 @@
         
   ";"
   {
-     return new LoadFromFileStatement(datasetName, adapter, properties, alreadySorted);
+     return new LoadFromFileStatement(null, datasetName, adapter, properties, alreadySorted);
   }   
 }
 
@@ -289,7 +281,7 @@
   "as"
   ( typeExpr = TypeExpr() )
   {
-    return new TypeDecl(ident, typeExpr);
+    return new TypeDecl(null, ident, typeExpr);
   }
 }
 
@@ -393,8 +385,8 @@
 
 FunctionDecl FunctionDeclaration() throws ParseException:
 {
-  FunctionDecl func = new FunctionDecl();
-  AsterixFunction ident;
+  FunctionDecl funcDecl;
+  FunctionSignature signature;
   String functionName;
   int arity = 0;
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
@@ -427,12 +419,10 @@
     })*)? <RIGHTPAREN> "{" funcBody = Expression() "}"
 
     {
-      ident = new AsterixFunction(functionName, arity);
-      getCurrentScope().addFunctionDescriptor(ident, false);
-      func.setIdent(ident);
-      func.setFuncBody(funcBody);
-      func.setParamList(paramList);
-      return func;
+      signature = new FunctionSignature(defaultDataverse, functionName, arity);
+      getCurrentScope().addFunctionDescriptor(signature, false);
+      funcDecl = new FunctionDecl(signature, paramList, funcBody);
+      return funcDecl;
     }
 }
 
@@ -1044,16 +1034,24 @@
 
 Expression FunctionCallExpr() throws ParseException:
 {
-  CallExpr pf = new CallExpr();
-  List<Expression > argList = new ArrayList<Expression >();
+  CallExpr callExpr;
+  List<Expression> argList = new ArrayList<Expression>();
   Expression tmp;
   int arity = 0;
-  Token funcName;
+  String funcName;
+  String dataverse;
+  String hint=null;
+  String id1=null;
+  String id2=null;
 }
-{
-    ( <IDENTIFIER> | <DATASET> )
+{  
+    ( <IDENTIFIER> { dataverse = defaultDataverse; funcName = token.image;} 
+     ("." <IDENTIFIER> { dataverse = funcName; funcName = token.image;})? 
+     | 
+     <DATASET> {dataverse = MetadataConstants.METADATA_DATAVERSE_NAME; funcName = getToken(0).toString();}
+    )
     {
-		funcName = getToken(0);
+       hint=getHint(token);
     }
      <LEFTPAREN> (tmp = Expression()
      {
@@ -1062,16 +1060,16 @@
      } ("," tmp = Expression() { argList.add(tmp); arity++; })*)? <RIGHTPAREN>
 
      {
-       AsterixFunction fd = lookupFunctionSignature(funcName.toString(), arity);
-	     if(fd == null)
-	     {
-	        fd = new AsterixFunction(funcName.toString(), arity);
-//	     	notFoundFunctionList.add(fd);
-	     }
-//	     	throw new ParseException("can't find function "+ funcName.toString() + "@" + arity);
-       pf.setIdent(fd);
-       pf.setExprList(argList);
-       return pf;
+       FunctionSignature signature = lookupFunctionSignature(dataverse, funcName.toString(), arity);
+             if(signature == null)
+             {
+                signature = new FunctionSignature(dataverse, funcName.toString(), arity);
+             }
+       callExpr = new CallExpr(signature,argList);
+       if (hint != null && hint.startsWith(INDEXED_NESTED_LOOP_JOIN_HINT)) {
+          callExpr.addHint(IndexedNLJoinExpressionAnnotation.INSTANCE);
+        }
+       return callExpr;
      }
 }