merged master into dependency branch, resolving all conflicts and  made it compatible with hyracks master
diff --git a/.gitignore b/.gitignore
index 1108a44..d748fde 100644
--- a/.gitignore
+++ b/.gitignore
@@ -4,4 +4,7 @@
 .project
 ClusterControllerService
 asterix-app/rttest
-asterix-app/mdtest
\ No newline at end of file
+asterix-app/mdtest/
+asterix-app/opttest/
+build
+asterix_logs
\ No newline at end of file
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
index 06e6814..2fc9c89 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/algebra/operators/physical/InvertedIndexPOperator.java
@@ -3,7 +3,8 @@
 import java.util.ArrayList;
 import java.util.List;
 
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.dataflow.IStorageContext;
 import edu.uci.ics.asterix.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -196,20 +197,24 @@
             IBinaryTokenizerFactory queryTokenizerFactory = InvertedIndexAccessMethod.getBinaryTokenizerFactory(
                     searchModifierType, searchKeyType, secondaryIndex);
             IIndexDataflowHelperFactory dataflowHelperFactory;
+
+            AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
             if (!isPartitioned) {
                 dataflowHelperFactory = new LSMInvertedIndexDataflowHelperFactory(
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate());
             } else {
                 dataflowHelperFactory = new PartitionedLSMInvertedIndexDataflowHelperFactory(
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate());
             }
             LSMInvertedIndexSearchOperatorDescriptor invIndexSearchOp = new LSMInvertedIndexSearchOperatorDescriptor(
                     jobSpec, queryField, appContext.getStorageManagerInterface(), secondarySplitsAndConstraint.first,
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 f14fff8..783fbaa 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
@@ -20,6 +20,7 @@
 
 import edu.uci.ics.asterix.optimizer.rules.AsterixInlineVariablesRule;
 import edu.uci.ics.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
+import edu.uci.ics.asterix.optimizer.rules.CheckFilterExpressionTypeRule;
 import edu.uci.ics.asterix.optimizer.rules.ConstantFoldingRule;
 import edu.uci.ics.asterix.optimizer.rules.CountVarToCountOneRule;
 import edu.uci.ics.asterix.optimizer.rules.ExtractDistinctByExpressionsRule;
@@ -36,6 +37,7 @@
 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.NestedSubplanToJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.PullPositionalVariableFromUnnestRule;
 import edu.uci.ics.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import edu.uci.ics.asterix.optimizer.rules.PushAggregateIntoGroupbyRule;
@@ -102,6 +104,7 @@
         List<IAlgebraicRewriteRule> typeInfer = new LinkedList<IAlgebraicRewriteRule>();
         typeInfer.add(new InlineUnnestFunctionRule());
         typeInfer.add(new InferTypesRule());
+        typeInfer.add(new CheckFilterExpressionTypeRule());
         return typeInfer;
     }
 
@@ -192,6 +195,7 @@
         consolidation.add(new CountVarToCountOneRule());
         consolidation.add(new RemoveUnusedAssignAndAggregateRule());
         consolidation.add(new RemoveRedundantGroupByDecorVars());
+        consolidation.add(new NestedSubplanToJoinRule());
         return consolidation;
     }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
new file mode 100644
index 0000000..aa57ab9
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/CheckFilterExpressionTypeRule.java
@@ -0,0 +1,83 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.optimizer.rules;
+
+import org.apache.commons.lang3.mutable.Mutable;
+
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.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.ILogicalOperator;
+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.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule is to check if all the filter expression are of the boolean type or a possible (determined
+ * at the runtime) boolean type.
+ * If that is not the case, an exception should be thrown.
+ * 
+ * @author yingyib
+ */
+public class CheckFilterExpressionTypeRule 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 op = (AbstractLogicalOperator) opRef.getValue();
+        if (op.getOperatorTag() != LogicalOperatorTag.SELECT) {
+            return false;
+        }
+        SelectOperator select = (SelectOperator) op;
+        ILogicalExpression condition = select.getCondition().getValue();
+        IVariableTypeEnvironment env = select.computeOutputTypeEnvironment(context);
+        IAType condType = (IAType) env.getType(condition);
+        if (condType.getTypeTag() != ATypeTag.BOOLEAN && condType.getTypeTag() != ATypeTag.ANY
+                && !isPossibleBoolean(condType)) {
+            throw new AlgebricksException("The select condition " + condition.toString()
+                    + " should be of the boolean type.");
+        }
+        return false;
+    }
+
+    /**
+     * Check if the type is optional boolean or not
+     * 
+     * @param type
+     * @return true if it is; false otherwise.
+     */
+    private boolean isPossibleBoolean(IAType type) {
+        while (type.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) type)) {
+            type = ((AUnionType) type).getUnionList().get(NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+            if (type.getTypeTag() == ATypeTag.BOOLEAN || type.getTypeTag() == ATypeTag.ANY) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+}
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 2dce5f6..e494ef4 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
@@ -24,8 +24,12 @@
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 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.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.ILogicalOperator;
@@ -36,10 +40,10 @@
 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.FunctionIdentifier;
 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;
-import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
@@ -87,62 +91,157 @@
         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;
 
         InsertDeleteOperator insertDeleteOperator = (InsertDeleteOperator) op2;
-        AssignOperator oldAssignOperator = (AssignOperator) op3;
-
         AqlDataSource dataSource = (AqlDataSource) insertDeleteOperator.getDataSource();
         IAType[] schemaTypes = (IAType[]) dataSource.getSchemaTypes();
         ARecordType requiredRecordType = (ARecordType) schemaTypes[schemaTypes.length - 1];
+        ILogicalExpression expr = insertDeleteOperator.getPayloadExpression().getValue();
+        List<LogicalVariable> payloadVars = new ArrayList<LogicalVariable>();
+        expr.getUsedVariables(payloadVars);
+        LogicalVariable recordVar = payloadVars.get(0);
+        IVariableTypeEnvironment env = insertDeleteOperator.computeOutputTypeEnvironment(context);
+        IAType inputRecordType = (IAType) env.getVarType(recordVar);
 
-        List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(oldAssignOperator, usedVariables);
-        LogicalVariable inputRecordVar;
-        if (usedVariables.size() > 0) {
-            inputRecordVar = usedVariables.get(0);
-        } else {
-            VariableUtilities.getLiveVariables(oldAssignOperator, usedVariables);
-            inputRecordVar = usedVariables.get(0);
+        /** the input record type can be an union type -- for the case when it comes from a subplan or left-outer join */
+        boolean checkNull = false;
+        while (isOptional(inputRecordType)) {
+            /** while-loop for the case there is a nested multi-level union */
+            inputRecordType = ((AUnionType) inputRecordType).getUnionList().get(
+                    NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+            checkNull = true;
         }
-        IVariableTypeEnvironment env = oldAssignOperator.computeInputTypeEnvironment(context);
-        ARecordType inputRecordType = (ARecordType) env.getVarType(inputRecordVar);
 
-        boolean needCast = !requiredRecordType.equals(inputRecordType);
-        if (!needCast)
+        /** see whether the input record type needs to be casted */
+        boolean cast = !compatible(requiredRecordType, inputRecordType);
+
+        if (checkNull) {
+            recordVar = addWrapperFunction(requiredRecordType, recordVar, insertDeleteOp, context,
+                    AsterixBuiltinFunctions.NOT_NULL);
+        }
+        if (cast) {
+            addWrapperFunction(requiredRecordType, recordVar, insertDeleteOp, context,
+                    AsterixBuiltinFunctions.CAST_RECORD);
+        }
+        return cast || checkNull;
+    }
+
+    /**
+     * Inject a function to wrap a variable when necessary
+     * 
+     * @param requiredRecordType
+     *            the required record type
+     * @param recordVar
+     *            the record variable
+     * @param parent
+     *            the current parent operator to be rewritten
+     * @param context
+     *            the optimization context
+     * @param fd
+     *            the function to be injected
+     * @return true if cast is injected; false otherwise.
+     * @throws AlgebricksException
+     */
+    public LogicalVariable addWrapperFunction(ARecordType requiredRecordType, LogicalVariable recordVar,
+            ILogicalOperator parent, IOptimizationContext context, FunctionIdentifier fd) throws AlgebricksException {
+        List<Mutable<ILogicalOperator>> opRefs = parent.getInputs();
+        for (int index = 0; index < opRefs.size(); index++) {
+            Mutable<ILogicalOperator> opRef = opRefs.get(index);
+            ILogicalOperator op = opRef.getValue();
+
+            /** get produced vars */
+            List<LogicalVariable> producedVars = new ArrayList<LogicalVariable>();
+            VariableUtilities.getProducedVariables(op, producedVars);
+            IVariableTypeEnvironment env = op.computeOutputTypeEnvironment(context);
+            for (int i = 0; i < producedVars.size(); i++) {
+                LogicalVariable var = producedVars.get(i);
+                if (var.equals(recordVar)) {
+                    /** insert an assign operator to call the function on-top-of the variable */
+                    IAType actualType = (IAType) env.getVarType(var);
+                    AbstractFunctionCallExpression cast = new ScalarFunctionCallExpression(
+                            FunctionUtils.getFunctionInfo(fd));
+                    cast.getArguments()
+                            .add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(var)));
+                    /** enforce the required record type */
+                    TypeComputerUtilities.setRequiredAndInputTypes(cast, requiredRecordType, actualType);
+                    LogicalVariable newAssignVar = context.newVar();
+                    AssignOperator newAssignOperator = new AssignOperator(newAssignVar,
+                            new MutableObject<ILogicalExpression>(cast));
+                    newAssignOperator.getInputs().add(new MutableObject<ILogicalOperator>(op));
+                    opRef.setValue(newAssignOperator);
+                    context.computeAndSetTypeEnvironmentForOperator(newAssignOperator);
+                    newAssignOperator.computeOutputTypeEnvironment(context);
+                    VariableUtilities.substituteVariables(parent, recordVar, newAssignVar, context);
+                    return newAssignVar;
+                }
+            }
+            /** recursive descend to the operator who produced the recordVar */
+            LogicalVariable replacedVar = addWrapperFunction(requiredRecordType, recordVar, op, context, fd);
+            if (replacedVar != null) {
+                /** substitute the recordVar by the replacedVar for operators who uses recordVar */
+                VariableUtilities.substituteVariables(parent, recordVar, replacedVar, context);
+                return replacedVar;
+            }
+        }
+        return null;
+    }
+
+    /**
+     * Check whether the required record type and the input type is compatible
+     * 
+     * @param reqType
+     * @param inputType
+     * @return true if compatible; false otherwise
+     * @throws AlgebricksException
+     */
+    private boolean compatible(ARecordType reqType, IAType inputType) throws AlgebricksException {
+        if (inputType.getTypeTag() == ATypeTag.ANY) {
             return false;
+        }
+        if (inputType.getTypeTag() != ATypeTag.RECORD) {
+            throw new AlgebricksException("The input type " + inputType + " is not a valid record type!");
+        }
 
-        // insert
-        // project
-        // assign
-        // assign
-        AbstractFunctionCallExpression cast = new ScalarFunctionCallExpression(
-                FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_RECORD));
-        ARecordType[] types = new ARecordType[2];
-        types[0] = requiredRecordType;
-        types[1] = inputRecordType;
-        cast.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputRecordVar)));
-        cast.setOpaqueParameters(types);
-        LogicalVariable newAssignVar = context.newVar();
-        AssignOperator newAssignOperator = new AssignOperator(newAssignVar, new MutableObject<ILogicalExpression>(cast));
-        newAssignOperator.getInputs().add(new MutableObject<ILogicalOperator>(op3));
+        IAType[] reqTypes = reqType.getFieldTypes();
+        String[] reqFieldNames = reqType.getFieldNames();
+        IAType[] inputTypes = ((ARecordType) inputType).getFieldTypes();
+        String[] inputFieldNames = ((ARecordType) inputType).getFieldNames();
 
-        List<LogicalVariable> projectVariables = new ArrayList<LogicalVariable>();
-        VariableUtilities.getProducedVariables(oldAssignOperator, projectVariables);
-        projectVariables.add(newAssignVar);
-        ProjectOperator projectOperator = new ProjectOperator(projectVariables);
-        projectOperator.getInputs().add(new MutableObject<ILogicalOperator>(newAssignOperator));
-
-        ILogicalExpression payloadExpr = new VariableReferenceExpression(newAssignVar);
-        MutableObject<ILogicalExpression> payloadRef = new MutableObject<ILogicalExpression>(payloadExpr);
-        InsertDeleteOperator newInserDeleteOperator = new InsertDeleteOperator(insertDeleteOperator.getDataSource(),
-                payloadRef, insertDeleteOperator.getPrimaryKeyExpressions(), insertDeleteOperator.getOperation());
-        newInserDeleteOperator.getInputs().add(new MutableObject<ILogicalOperator>(projectOperator));
-        insertDeleteOperator.getInputs().clear();
-        op1.getInputs().get(0).setValue(newInserDeleteOperator);
+        if (reqTypes.length != inputTypes.length) {
+            return false;
+        }
+        for (int i = 0; i < reqTypes.length; i++) {
+            if (!reqFieldNames[i].equals(inputFieldNames[i])) {
+                return false;
+            }
+            IAType reqTypeInside = reqTypes[i];
+            if (isOptional(reqTypes[i])) {
+                reqTypeInside = ((AUnionType) reqTypes[i]).getUnionList().get(
+                        NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+            }
+            IAType inputTypeInside = inputTypes[i];
+            if (isOptional(inputTypes[i])) {
+                if (!isOptional(reqTypes[i])) {
+                    /** if the required type is not optional, the two types are incompatible */
+                    return false;
+                }
+                inputTypeInside = ((AUnionType) inputTypes[i]).getUnionList().get(
+                        NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+            }
+            if (inputTypeInside.getTypeTag() != ATypeTag.NULL && !reqTypeInside.equals(inputTypeInside)) {
+                return false;
+            }
+        }
         return true;
     }
 
+    /**
+     * Decide whether a type is an optional type
+     * 
+     * @param type
+     * @return true if it is optional; false otherwise
+     */
+    private boolean isOptional(IAType type) {
+        return type.getTypeTag() == ATypeTag.UNION && NonTaggedFormatUtil.isOptionalField((AUnionType) type);
+    }
 }
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 c99e4bc..6985753 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
@@ -64,10 +64,16 @@
         }
 
         FunctionIdentifier fid = null;
+        /** find the record variable */
+        InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
+        ILogicalExpression recordExpr = insertOp.getPayloadExpression().getValue();
+        List<LogicalVariable> recordVar = new ArrayList<LogicalVariable>();
+        /** assume the payload is always a single variable expression */
+        recordExpr.getUsedVariables(recordVar);
+
         /** op2 is the assign operator which extract primary keys from the record variable */
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
-        List<LogicalVariable> recordVar = new ArrayList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(op2, recordVar);
+
         if (recordVar.size() == 0) {
             /**
              * For the case primary key-assignment expressions are constant expressions,
@@ -92,7 +98,6 @@
             AssignOperator assignOp2 = (AssignOperator) op2;
             recordVar.addAll(assignOp2.getVariables());
         }
-        InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
         AqlDataSource datasetSource = (AqlDataSource) insertOp.getDataSource();
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         String dataverseName = datasetSource.getId().getDataverseName();
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 3aae2dd..f42782b 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
@@ -42,7 +42,7 @@
 /**
  * Statically cast a constant from its type to a specified required type, in a
  * 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
+ * cast of an 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",
@@ -90,9 +90,6 @@
         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
          */
@@ -101,21 +98,21 @@
         IAType[] schemaTypes = (IAType[]) dataSource.getSchemaTypes();
         IAType requiredRecordType = schemaTypes[schemaTypes.length - 1];
 
-        AssignOperator topAssignOperator = (AssignOperator) assignOp;
         List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(topAssignOperator, usedVariables);
+        insertDeleteOperator.getPayloadExpression().getValue().getUsedVariables(usedVariables);
 
         // the used variable should contain the record that will be inserted
         // but it will not fail in many cases even if the used variable set is
         // empty
         if (usedVariables.size() == 0)
             return false;
+
         oldRecordVariable = usedVariables.get(0);
         LogicalVariable inputRecordVar = usedVariables.get(0);
-        IVariableTypeEnvironment env = topAssignOperator.computeOutputTypeEnvironment(context);
+        IVariableTypeEnvironment env = insertDeleteOperator.computeOutputTypeEnvironment(context);
         IAType inputRecordType = (IAType) env.getVarType(inputRecordVar);
 
-        AbstractLogicalOperator currentOperator = assignOp;
+        AbstractLogicalOperator currentOperator = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
         /**
          * find the assign operator for the "input record" to the insert_delete
          * operator
@@ -123,6 +120,7 @@
         do {
             context.addToDontApplySet(this, currentOperator);
             if (currentOperator.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                AssignOperator assignOp = (AssignOperator) currentOperator;
                 producedVariables.clear();
                 VariableUtilities.getProducedVariables(currentOperator, producedVariables);
                 int position = producedVariables.indexOf(oldRecordVariable);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
new file mode 100644
index 0000000..c5d415e
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.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 org.apache.commons.lang3.mutable.MutableObject;
+
+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.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.ConstantExpression;
+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.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * replace Subplan operators with nested loop joins where the join condition is true, if the Subplan
+ * does not contain free variables (does not have correlations to the input stream).
+ * 
+ * @author yingyib
+ */
+public class NestedSubplanToJoinRule 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;
+        context.addToDontApplySet(this, opRef.getValue());
+
+        ILogicalOperator op1 = opRef.getValue();
+        if (op1.getInputs().size() == 0) {
+            return false;
+        }
+
+        boolean rewritten = false;
+        for (int index = 0; index < op1.getInputs().size(); index++) {
+            AbstractLogicalOperator child = (AbstractLogicalOperator) op1.getInputs().get(index).getValue();
+            if (child.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+                continue;
+            }
+
+            AbstractOperatorWithNestedPlans subplan = (AbstractOperatorWithNestedPlans) child;
+            Set<LogicalVariable> freeVars = new HashSet<LogicalVariable>();
+            OperatorPropertiesUtil.getFreeVariablesInSubplans(subplan, freeVars);
+            if (!freeVars.isEmpty()) {
+                /**
+                 * the subplan is correlated with the outer plan, other rules can deal with it
+                 */
+                continue;
+            }
+
+            /** get the input operator of the subplan operator */
+            ILogicalOperator subplanInput = subplan.getInputs().get(0).getValue();
+
+            /** get all nested top operators */
+            List<ILogicalPlan> nestedPlans = subplan.getNestedPlans();
+            List<Mutable<ILogicalOperator>> nestedRoots = new ArrayList<Mutable<ILogicalOperator>>();
+            for (ILogicalPlan nestedPlan : nestedPlans) {
+                nestedRoots.addAll(nestedPlan.getRoots());
+            }
+            if (nestedRoots.size() == 0) {
+                /** there is no nested top operators */
+                return false;
+            }
+
+            /** expend the input and roots into a DAG of nested loop joins */
+            Mutable<ILogicalExpression> expr = new MutableObject<ILogicalExpression>(ConstantExpression.TRUE);
+            Mutable<ILogicalOperator> nestedRootRef = nestedRoots.get(0);
+            ILogicalOperator join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(subplanInput),
+                    nestedRootRef);
+
+            /** rewrite the nested tuple source to be empty tuple source */
+            rewriteNestedTupleSource(nestedRootRef);
+
+            for (int i = 1; i < nestedRoots.size(); i++) {
+                join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(join), nestedRoots.get(i));
+            }
+            op1.getInputs().get(index).setValue(join);
+            context.computeAndSetTypeEnvironmentForOperator(join);
+            rewritten = true;
+        }
+        return rewritten;
+    }
+
+    /**
+     * rewrite NestedTupleSource operators to EmptyTupleSource operators
+     * 
+     * @param nestedRootRef
+     */
+    private void rewriteNestedTupleSource(Mutable<ILogicalOperator> nestedRootRef) {
+        AbstractLogicalOperator nestedRoot = (AbstractLogicalOperator) nestedRootRef.getValue();
+        if (nestedRoot.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+            nestedRootRef.setValue(new EmptyTupleSourceOperator());
+        }
+        List<Mutable<ILogicalOperator>> inputs = nestedRoot.getInputs();
+        for (Mutable<ILogicalOperator> input : inputs) {
+            rewriteNestedTupleSource(input);
+        }
+    }
+}
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
index c5a1cb0..bee8c40 100644
--- 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
@@ -111,19 +111,7 @@
         
         // 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);
         
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 3dad464..dc86722 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
@@ -186,6 +186,8 @@
                 case DATE:
                 case TIME:
                 case DURATION:
+                case YEARMONTHDURATION:
+                case DAYTIMEDURATION:
                 case INTERVAL:
                 case POINT:
                 case POINT3D:
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
index 3ba3e96..d6e0f42 100644
--- 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
@@ -17,11 +17,14 @@
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -39,10 +42,12 @@
 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.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.functions.IFunctionInfo;
 
 /**
  * This class is utility to do type cast.
@@ -124,6 +129,11 @@
      */
     public static boolean rewriteFuncExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
             IVariableTypeEnvironment env) throws AlgebricksException {
+        /**
+         * sanity check: if there are list(ordered or unordered)/record variable expressions in the funcExpr, we will not do STATIC type casting
+         * because they are not "statically cast-able".
+         * instead, the record will be dynamically casted at the runtime
+         */
         if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR) {
             if (reqType.equals(BuiltinType.ANY)) {
                 reqType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
@@ -152,6 +162,10 @@
                     changed = changed || rewriteFuncExpr(argFuncExpr, exprType, exprType, env);
                 }
             }
+            if (!compatible(reqType, inputType)) {
+                throw new AlgebricksException("type mistmach, requred: " + reqType.toString() + " actual: "
+                        + inputType.toString());
+            }
             return changed;
         }
     }
@@ -175,8 +189,7 @@
         if (TypeComputerUtilities.getRequiredType(funcExpr) != null)
             return false;
         TypeComputerUtilities.setRequiredAndInputTypes(funcExpr, requiredRecordType, inputRecordType);
-        staticRecordTypeCast(funcExpr, requiredRecordType, inputRecordType, env);
-        return true;
+        return staticRecordTypeCast(funcExpr, requiredRecordType, inputRecordType, env);
     }
 
     /**
@@ -231,7 +244,7 @@
      *            The type environment.
      * @throws AlgebricksException
      */
-    private static void staticRecordTypeCast(AbstractFunctionCallExpression func, ARecordType reqType,
+    private static boolean staticRecordTypeCast(AbstractFunctionCallExpression func, ARecordType reqType,
             ARecordType inputType, IVariableTypeEnvironment env) throws AlgebricksException {
         IAType[] reqFieldTypes = reqType.getFieldTypes();
         String[] reqFieldNames = reqType.getFieldNames();
@@ -327,8 +340,10 @@
                 }
             }
             // the input has extra fields
-            if (!matched && !reqType.isOpen())
-                throw new AlgebricksException("static type mismatch: including an extra closed field " + fieldName);
+            if (!matched && !reqType.isOpen()) {
+                throw new AlgebricksException("static type mismatch: the input record includes an extra closed field "
+                        + fieldName + ":" + fieldType + "! Please check the field name and type.");
+            }
         }
 
         // backward match: match from required to actual
@@ -371,7 +386,14 @@
                 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);
+                if (inputType.isOpen()) {
+                    //if the input type is open, return false, give that to dynamic type cast to defer the error to the runtime
+                    return false;
+                } else {
+                    throw new AlgebricksException(
+                            "static type mismatch: the input record misses a required closed field " + reqFieldName
+                                    + ":" + reqFieldType + "! Please check the field name and type.");
+                }
             }
         }
 
@@ -399,32 +421,125 @@
         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();
-
+                Mutable<ILogicalExpression> expRef = originalArguments.get(2 * i + 1);
+                ILogicalExpression argExpr = expRef.getValue();
+                List<LogicalVariable> parameterVars = new ArrayList<LogicalVariable>();
+                argExpr.getUsedVariables(parameterVars);
                 // 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) {
+                boolean castInjected = false;
+                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+                        || argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
                     IAType reqFieldType = inputFieldTypes[i];
+                    // do not enforce nested type in the case of no-used variables
                     if (inputFieldTypes[i].getTypeTag() == ATypeTag.RECORD) {
                         reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+                            //inject dynamic type casting
+                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_RECORD),
+                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
+                            castInjected = true;
+                        }
                     }
                     if (inputFieldTypes[i].getTypeTag() == ATypeTag.ORDEREDLIST) {
                         reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+                            //inject dynamic type casting
+                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
+                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
+                            castInjected = true;
+                        }
                     }
                     if (inputFieldTypes[i].getTypeTag() == ATypeTag.UNORDEREDLIST) {
                         reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+                            //inject dynamic type casting
+                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
+                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
+                            castInjected = true;
+                        }
                     }
-                    if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null) {
-                        ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
-                        rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
+                    if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        //recursively rewrite function arguments
+                        if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null
+                                && reqFieldType != null) {
+                            if (castInjected) {
+                                //rewrite the arg expression inside the dynamic cast
+                                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+                                rewriteFuncExpr(argFunc, inputFieldTypes[i], inputFieldTypes[i], env);
+                            } else {
+                                //rewrite arg
+                                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+                                rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
+                            }
+                        }
                     }
                 }
-                arguments.add(fExprRef);
+                arguments.add(expRef);
             }
         }
+        return true;
     }
 
+    /**
+     * Inject a dynamic cast function wrapping an existing expression
+     * 
+     * @param funcInfo
+     *            the cast function
+     * @param reqType
+     *            the required type
+     * @param inputType
+     *            the original type
+     * @param exprRef
+     *            the expression reference
+     * @param argExpr
+     *            the original expression
+     */
+    private static void injectCastFunction(IFunctionInfo funcInfo, IAType reqType, IAType inputType,
+            Mutable<ILogicalExpression> exprRef, ILogicalExpression argExpr) {
+        ScalarFunctionCallExpression cast = new ScalarFunctionCallExpression(funcInfo);
+        cast.getArguments().add(new MutableObject<ILogicalExpression>(argExpr));
+        exprRef.setValue(cast);
+        TypeComputerUtilities.setRequiredAndInputTypes(cast, reqType, inputType);
+    }
+
+    /**
+     * Determine if two types are compatible
+     * 
+     * @param reqType
+     *            the required type
+     * @param inputType
+     *            the input type
+     * @return true if the two types are compatiable; false otherwise
+     */
+    public static boolean compatible(IAType reqType, IAType inputType) {
+        if (reqType.getTypeTag() == ATypeTag.ANY || inputType.getTypeTag() == ATypeTag.ANY) {
+            return true;
+        }
+        if (reqType.getTypeTag() != ATypeTag.UNION && inputType.getTypeTag() != ATypeTag.UNION) {
+            if (reqType.equals(inputType)) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+        Set<IAType> reqTypePossible = new HashSet<IAType>();
+        Set<IAType> inputTypePossible = new HashSet<IAType>();
+        if (reqType.getTypeTag() == ATypeTag.UNION) {
+            AUnionType unionType = (AUnionType) reqType;
+            reqTypePossible.addAll(unionType.getUnionList());
+        } else {
+            reqTypePossible.add(reqType);
+        }
+
+        if (inputType.getTypeTag() == ATypeTag.UNION) {
+            AUnionType unionType = (AUnionType) inputType;
+            inputTypePossible.addAll(unionType.getUnionList());
+        } else {
+            inputTypePossible.add(inputType);
+        }
+        return reqTypePossible.equals(inputTypePossible);
+    }
 }
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 eed0f80..e07c1d0 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
@@ -71,7 +71,8 @@
 import edu.uci.ics.asterix.aql.expression.WriteStatement;
 import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
-import edu.uci.ics.asterix.common.config.AsterixProperties;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 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;
@@ -91,6 +92,7 @@
 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.AsterixBuiltinFunctions.FunctionNamespace;
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
@@ -196,7 +198,7 @@
 
             List<Mutable<ILogicalExpression>> writeExprList = new ArrayList<Mutable<ILogicalExpression>>(1);
             writeExprList.add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(resVar)));
-            ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId(), resultNodeName);
+            ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId());
             ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
             topOp = new DistributeResultOperator(writeExprList, sink);
             topOp.getInputs().add(new MutableObject<ILogicalOperator>(project));
@@ -288,7 +290,8 @@
         String outputDir = System.getProperty("java.io.tmpDir");
         String filePath = outputDir + System.getProperty("file.separator") + OUTPUT_FILE_PREFIX
                 + outputFileID.incrementAndGet();
-        return new FileSplit(AsterixProperties.INSTANCE.getMetadataNodeName(), new FileReference(new File(filePath)));
+        AsterixMetadataProperties metadataProperties = AsterixAppContextInfo.getInstance().getMetadataProperties();
+        return new FileSplit(metadataProperties.getMetadataNodeName(), new FileReference(new File(filePath)));
     }
 
     @Override
@@ -504,10 +507,14 @@
         if (builtinAquafi != null) {
             fi = builtinAquafi;
         } else {
-            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, functionName, arity);
+            fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(), functionName, arity);
             afi = AsterixBuiltinFunctions.lookupFunction(fi);
             if (afi == null) {
-                return null;
+                fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(), functionName, arity);
+                afi = AsterixBuiltinFunctions.lookupFunction(fi);
+                if (afi == null) {
+                    return null;
+                }
             }
         }
         if (AsterixBuiltinFunctions.isBuiltinAggregateFunction(fi)) {
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 0f636a9..50e79a5 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
@@ -90,6 +90,7 @@
 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.AsterixBuiltinFunctions.FunctionNamespace;
 import edu.uci.ics.asterix.om.functions.AsterixFunctionInfo;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -464,10 +465,14 @@
         if (builtinAquafi != null) {
             fi = builtinAquafi;
         } else {
-            fi = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, signature.getName());
+            fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(), signature.getName());
             FunctionIdentifier builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
             if (builtinAsterixFi != null) {
                 fi = builtinAsterixFi;
+            } else {
+                fi = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(), signature.getName());
+                builtinAsterixFi = AsterixBuiltinFunctions.getBuiltinFunctionIdentifier(fi);
+                fi = builtinAsterixFi;
             }
         }
         AbstractFunctionCallExpression f;
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
index 6cf8d28..a91f2eb 100644
--- 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
@@ -35,10 +35,10 @@
 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.functions.AsterixBuiltinFunctions;
 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;
@@ -495,8 +495,8 @@
             LiteralExpr argumentLiteral = new LiteralExpr(new StringLiteral(arg));
             arguments.add(argumentLiteral);
 
-            CallExpr callExpression = new CallExpr(new FunctionSignature(FunctionConstants.ASTERIX_NS, "dataset", 1),
-                    arguments);
+            CallExpr callExpression = new CallExpr(new FunctionSignature(
+                    AsterixBuiltinFunctions.FunctionNamespace.ASTERIX_PUBLIC.name(), "dataset", 1), arguments);
             List<Clause> clauseList = new ArrayList<Clause>();
             Clause forClause = new ForClause(var, callExpression);
             clauseList.add(forClause);
diff --git a/asterix-app/.gitignore b/asterix-app/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/asterix-app/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/asterix-app/data/fbu-dml-insert-shuffled.adm b/asterix-app/data/fbu-dml-insert-shuffled.adm
new file mode 100644
index 0000000..5e42fd2
--- /dev/null
+++ b/asterix-app/data/fbu-dml-insert-shuffled.adm
@@ -0,0 +1,29 @@
+{"id":11381089,"id-copy":11381089,"alias":"Earlene","name":"EarleneAmmons","user-since":datetime("2010-03-24T05:25:35"),"user-since-copy":datetime("2010-03-24T05:25:35"),"friend-ids":{{25392364,36996951,16110083,9799716,22893553,28551996,7706432,14225386,15633254,39395931,46707062,37226919,8532306,3765988,20939685,31136325,45222021,15355741,8760941,12045616,6890610,13560532,44914868,37744233}},"employment":[{"organization-name":"Roundhex","start-date":date("2000-06-10")}]}
+{"id":10495420,"id-copy":10495420,"alias":"Wendy","name":"WendyMcloskey","user-since":datetime("2011-04-26T23:38:24"),"user-since-copy":datetime("2011-04-26T23:38:24"),"friend-ids":{{16762653,46262691,12313140,20481262,347993,23105127,1680519,20880265,45611347,21907223,46615281,17188244,44019800,46943250,28647738,16792673,29406270,42714079}},"employment":[{"organization-name":"Qvohouse","start-date":date("2008-08-27")}]}
+{"id":10957867,"id-copy":10957867,"alias":"Zach","name":"ZachOppenheimer","user-since":datetime("2012-01-01T14:40:11"),"user-since-copy":datetime("2012-01-01T14:40:11"),"friend-ids":{{27759480,2112389,8560433,10052851,37714587,16717012,36648956,44803993,36030695,5359496,32302980,27143894,19287706}},"employment":[{"organization-name":"Tanzumbam","start-date":date("2003-05-14"),"end-date":date("2004-02-23")}]}
+{"id":9988417,"id-copy":9988417,"alias":"Coline","name":"ColineLane","user-since":datetime("2010-01-01T00:12:39"),"user-since-copy":datetime("2010-01-01T00:12:39"),"friend-ids":{{17656229,42804152}},"employment":[{"organization-name":"Fax-fax","start-date":date("2012-05-01")}]}
+{"id":10272571,"id-copy":10272571,"alias":"Jarrett","name":"JarrettGoldvogel","user-since":datetime("2010-04-28T23:24:22"),"user-since-copy":datetime("2010-04-28T23:24:22"),"friend-ids":{{47024505,36647273,32152567,28239957,11739703,47515825,17408763,41224279,41487670,43339913}},"employment":[{"organization-name":"Transhigh","start-date":date("2004-04-06"),"end-date":date("2010-02-14")}]}
+{"id":11307946,"id-copy":11307946,"alias":"Helga","name":"HelgaStough","user-since":datetime("2007-01-12T21:50:11"),"user-since-copy":datetime("2007-01-12T21:50:11"),"friend-ids":{{22768365}},"employment":[{"organization-name":"subtam","start-date":date("2007-01-04"),"end-date":date("2009-06-25")}]}
+{"id":11061631,"id-copy":11061631,"alias":"Maxene","name":"MaxeneKellogg","user-since":datetime("2005-11-13T01:09:31"),"user-since-copy":datetime("2005-11-13T01:09:31"),"friend-ids":{{31578394,39466620,35741359,14244925,3000582,39031643,5008430,18315325,30440631,37868108,12014032,32314102,42887702,1853960,28022174,2024670,38864358,42073112,16259942,34693959,25315399,37475597,33599283}},"employment":[{"organization-name":"Unijobam","start-date":date("2008-05-13")}]}
+{"id":10874791,"id-copy":10874791,"alias":"Haydee","name":"HaydeeGarratt","user-since":datetime("2007-04-14T00:19:00"),"user-since-copy":datetime("2007-04-14T00:19:00"),"friend-ids":{{12247794,10306863,33161811,43877113,37745696}},"employment":[{"organization-name":"Opeelectronics","start-date":date("2008-03-07"),"end-date":date("2011-12-27")}]}
+{"id":11570326,"id-copy":11570326,"alias":"Linden","name":"LindenFilby","user-since":datetime("2007-08-16T03:11:11"),"user-since-copy":datetime("2007-08-16T03:11:11"),"friend-ids":{{6549689,15243636,3147666}},"employment":[{"organization-name":"Solfix","start-date":date("2010-02-23"),"end-date":date("2010-04-22")}]}
+{"id":10498285,"id-copy":10498285,"alias":"Kiley","name":"KileyBridger","user-since":datetime("2006-05-14T21:55:34"),"user-since-copy":datetime("2006-05-14T21:55:34"),"friend-ids":{{38780484,46190003,905670,35609390,46621151,5099226,24328595,16340411,13326485,13872400,35896828,9196151,8525875,7461206,28379538,46461267,45270205,35718577,5310596,7080391}},"employment":[{"organization-name":"Newcom","start-date":date("2009-11-11"),"end-date":date("2009-06-23")}]}
+{"id":9629395,"id-copy":9629395,"alias":"Julius","name":"JuliusWire","user-since":datetime("2008-03-22T13:36:24"),"user-since-copy":datetime("2008-03-22T13:36:24"),"friend-ids":{{}},"employment":[{"organization-name":"Tranzap","start-date":date("2006-11-19")}]}
+{"id":11447332,"id-copy":11447332,"alias":"Sherisse","name":"SherisseMaugham","user-since":datetime("2012-02-09T14:21:08"),"user-since-copy":datetime("2012-02-09T14:21:08"),"friend-ids":{{}},"employment":[{"organization-name":"Tripplelane","start-date":date("2011-09-16")}]}
+{"id":10179538,"id-copy":10179538,"alias":"Orlando","name":"OrlandoBaxter","user-since":datetime("2006-02-06T08:33:07"),"user-since-copy":datetime("2006-02-06T08:33:07"),"friend-ids":{{6233497,33888281,44259464,19279042,22534429,13084190,38886041,41675566,3155617}},"employment":[{"organization-name":"Ontohothex","start-date":date("2009-07-06")}]}
+{"id":10001080,"id-copy":10001080,"alias":"Garrett","name":"GarrettBode","user-since":datetime("2005-10-25T18:07:35"),"user-since-copy":datetime("2005-10-25T18:07:35"),"friend-ids":{{35858744,16426061,11473961,4769664,29038930,33070686,46271872,42593454,36202882,46642640,22243678,20222041,29014540,7389258,7172909,12787979,146736,21081030,21615179,2936936,44934891}},"employment":[{"organization-name":"Tanzimcare","start-date":date("2007-06-24")}]}
+{"id":11675221,"id-copy":11675221,"alias":"Calanthe","name":"CalantheGearhart","user-since":datetime("2007-06-08T02:44:20"),"user-since-copy":datetime("2007-06-08T02:44:20"),"friend-ids":{{19185575}},"employment":[{"organization-name":"Vivaace","start-date":date("2010-05-21")}]}
+{"id":11140213,"id-copy":11140213,"alias":"Montgomery","name":"MontgomeryWhittier","user-since":datetime("2007-06-19T17:46:13"),"user-since-copy":datetime("2007-06-19T17:46:13"),"friend-ids":{{32831460,6030454,30437362,21866470,17388602,40815157,20000967,47555494,5818137,40634742,21692148,2365521,33290069,46471164,9192561,35768343,7552168,3577338,5346012,31129868}},"employment":[{"organization-name":"Y-geohex","start-date":date("2008-02-24")}]}
+{"id":11954992,"id-copy":11954992,"alias":"Caitlin","name":"CaitlinLangston","user-since":datetime("2007-01-02T01:50:34"),"user-since-copy":datetime("2007-01-02T01:50:34"),"friend-ids":{{23355687,22474136,28513847,32515387,44041844,33706721,10874992,36341753,34431157,16146113,15462591,18188151,29554174,44940738,25888018,42795884,14382632,12734889,11724519,15830341,25725320,37580394,24124411,47984339}},"employment":[{"organization-name":"Kanelectrics","start-date":date("2010-05-26"),"end-date":date("2010-03-28")}]}
+{"id":9510451,"id-copy":9510451,"alias":"Chuck","name":"ChuckFinck","user-since":datetime("2011-09-10T08:27:31"),"user-since-copy":datetime("2011-09-10T08:27:31"),"friend-ids":{{5559039,8997599,8311284,20478562,13734713,21511695,30393493}},"employment":[{"organization-name":"Inchdox","start-date":date("2001-10-12")}]}
+{"id":11068231,"id-copy":11068231,"alias":"Dinah","name":"DinahSwink","user-since":datetime("2012-05-02T04:24:33"),"user-since-copy":datetime("2012-05-02T04:24:33"),"friend-ids":{{31542440,17451543,32642661,27867264,32718667,43042567,7921827}},"employment":[{"organization-name":"highfax","start-date":date("2003-04-10"),"end-date":date("2003-10-03")}]}
+{"id":10361965,"id-copy":10361965,"alias":"Arlen","name":"ArlenFlick","user-since":datetime("2011-07-14T18:38:37"),"user-since-copy":datetime("2011-07-14T18:38:37"),"friend-ids":{{34249140,2887282,47622716,3897801,33692288,14374380,14183995,41311739,6378075,17721901,20807501,8908974,41080464,26497672}},"employment":[{"organization-name":"Medflex","start-date":date("2008-05-18"),"end-date":date("2011-09-18")}]}
+{"id":10423588,"id-copy":10423588,"alias":"Shirlene","name":"ShirleneRuch","user-since":datetime("2006-04-09T05:52:24"),"user-since-copy":datetime("2006-04-09T05:52:24"),"friend-ids":{{15418780,12724265,27282306,13592995,24753166,32824252,40619106,27563604,12337625,45387219,27749581,44912564,37470078,19663516}},"employment":[{"organization-name":"Newphase","start-date":date("2003-06-17")}]}
+{"id":11951098,"id-copy":11951098,"alias":"Tera","name":"TeraByers","user-since":datetime("2012-08-03T19:41:26"),"user-since-copy":datetime("2012-08-03T19:41:26"),"friend-ids":{{15537238,13699967,10587728,23542817,12703626,25024772,19223339,5547239,42576945,27351017,22726496,25268071,4361323,24631578,38669047,44781738,34646381}},"employment":[{"organization-name":"Sublamdox","start-date":date("2008-01-04"),"end-date":date("2011-01-14")}]}
+{"id":9594523,"id-copy":9594523,"alias":"Tam","name":"TamWillcox","user-since":datetime("2011-12-23T11:41:58"),"user-since-copy":datetime("2011-12-23T11:41:58"),"friend-ids":{{27383896,20745988,10063024,8241427,40299998,32408463,25171835,22380586,15344194,25951348,28733234,45421004,2273747,2229862,6241144,6704115,8659430,47431991,47929530,24393021}},"employment":[{"organization-name":"Keytech","start-date":date("2001-07-27")}]}
+{"id":9478720,"id-copy":9478720,"alias":"Angelia","name":"AngeliaKettlewell","user-since":datetime("2005-05-27T06:29:30"),"user-since-copy":datetime("2005-05-27T06:29:30"),"friend-ids":{{42556433,20033025,38112512,19420757,31822717,7116081,39544900,19203395,46787205,32303456,4509345,45558040,42616291,6929369,9272653,37459048,37113569,38942369,47741031,46761451,14163845}},"employment":[{"organization-name":"Alphadax","start-date":date("2012-03-28"),"end-date":date("2012-03-04")}]}
+{"id":9142198,"id-copy":9142198,"alias":"Sherry","name":"SherryFea","user-since":datetime("2011-03-28T23:09:22"),"user-since-copy":datetime("2011-03-28T23:09:22"),"friend-ids":{{6835080,34471872,30942941,34858577,5996593,47293442,43097072,44809621,33969893,26410931,6628186,29944391,35957320,20326929,40284077,11681583,43878314,40265961,16871274,28406169,1349311}},"employment":[{"organization-name":"Mathtech","start-date":date("2004-07-28")}]}
+{"id":9313492,"id-copy":9313492,"alias":"Tera","name":"TeraWolfe","user-since":datetime("2010-12-20T12:47:25"),"user-since-copy":datetime("2010-12-20T12:47:25"),"friend-ids":{{45424983,18345704,14849759,31638064,38670515,48015953,36114769}},"employment":[{"organization-name":"Redelectronics","start-date":date("2001-04-26"),"end-date":date("2004-12-06")}]}
+{"id":10307032,"id-copy":10307032,"alias":"Quentin","name":"QuentinSauter","user-since":datetime("2012-07-11T07:16:43"),"user-since-copy":datetime("2012-07-11T07:16:43"),"friend-ids":{{1926278,42211794,1508832,14973540,6721046,28872485,5047722,7805271,31508326,20891455,38735410,13190567,18209753,44468536,34640135,47290587,25576626}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2012-02-13")}]}
+{"id":10733617,"id-copy":10733617,"alias":"Leonardo","name":"LeonardoKight","user-since":datetime("2008-10-20T17:30:29"),"user-since-copy":datetime("2008-10-20T17:30:29"),"friend-ids":{{39687903,7235506,34696496,25995345,18435380,47473591,15710408,44232442,39520147,36384026,25160887,245860,1195579,4587411,536916,47052672,33953823,13203710}},"employment":[{"organization-name":"tresline","start-date":date("2007-07-12"),"end-date":date("2010-03-16")}]}
+{"id":10394488,"id-copy":10394488,"alias":"Oswald","name":"OswaldRay","user-since":datetime("2006-02-12T17:39:23"),"user-since-copy":datetime("2006-02-12T17:39:23"),"friend-ids":{{14370372,14174983,7749259,39375970,1755409,9056913}},"employment":[{"organization-name":"Canline","start-date":date("2011-12-04"),"end-date":date("2011-06-08")}]}
diff --git a/asterix-app/data/temporal/temporalData.json b/asterix-app/data/temporal/temporalData.json
index 5b3a0f5..bedf2c7 100644
--- a/asterix-app/data/temporal/temporalData.json
+++ b/asterix-app/data/temporal/temporalData.json
@@ -1,3 +1,4 @@
 {"id": "001", "dateField": date("-2012-12-12"), "dateFieldPlus": date("0990-01-01"), "timeField": time("23:49:12.39Z"), "timeFieldPlus": time("03:23:12.2"), "datetimeField": datetime("2012-12-12T00:00:00.001"), "datetimeFieldPlus": datetime("-00130810T221015398"), "durationField": duration("P20Y19DT3H74M23.34S"), "durationFieldPlus": duration("-P2MT4M300.68S"), "intervalField": interval-datetime("2012-12-12T00:00:00.001,20130810T221015398") }
 {"id": "002", "timeField": time("12:12:12.219+08:00"), "timeField2": time("13:13:13.321-08:00"), "datetimeField": datetime("19201220T232918478-1200"), "intervalField": interval-time("12:29:30+08:00, 235959999-0800") }
-{"id": "003", "durationField": duration("-P27Y48DT2400H"), "datetimeField": datetime("-0290-03-23T00:59:48.999+07:00"), "intervalField": interval-date("-2012-03-17, 2013-04-01"), "intervalPlus": interval-time("19:23:32.328Z,23:20:20") }
\ No newline at end of file
+{"id": "003", "durationField": duration("-P27Y48DT2400H"), "datetimeField": datetime("-0290-03-23T00:59:48.999+07:00"), "intervalField": interval-date("-2012-03-17, 2013-04-01"), "intervalPlus": interval-time("19:23:32.328Z,23:20:20") }
+{"id": "003", "dayTimeDurationField": day-time-duration("-P48DT2400H"), "yearMonthDurationField": year-month-duration("P27Y50M") }
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
index c3f48d4..46bdce0 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/APIFramework.java
@@ -26,7 +26,8 @@
 import edu.uci.ics.asterix.aql.expression.Query;
 import edu.uci.ics.asterix.aql.expression.visitor.AQLPrintVisitor;
 import edu.uci.ics.asterix.aql.rewrites.AqlRewriter;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
@@ -43,7 +44,6 @@
 import edu.uci.ics.asterix.optimizer.base.RuleCollections;
 import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
 import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.StorageContext;
 import edu.uci.ics.asterix.translator.AqlExpressionToPlanTranslator;
 import edu.uci.ics.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -181,7 +181,7 @@
                 }
             }
         }
-        AqlRewriter rw = new AqlRewriter(declaredFunctions, q, metadataProvider.getMetadataTxnContext());
+        AqlRewriter rw = new AqlRewriter(declaredFunctions, q, metadataProvider);
         rw.rewrite();
         Query rwQ = rw.getExpr();
         return new Pair(rwQ, rw.getVarCounter());
@@ -258,7 +258,8 @@
             }
         }
 
-        int frameSize = GlobalConfig.getFrameSize();
+        AsterixCompilerProperties compilerProperties = AsterixAppContextInfo.getInstance().getCompilerProperties();
+        int frameSize = compilerProperties.getFrameSize();
 
         HeuristicCompilerFactoryBuilder builder = new HeuristicCompilerFactoryBuilder(
                 AqlOptimizationContextFactory.INSTANCE);
@@ -342,7 +343,7 @@
 
         IJobletEventListenerFactory jobEventListenerFactory = new JobEventListenerFactory(asterixJobId,
                 isWriteTransaction);
-        JobSpecification spec = compiler.createJob(StorageContext.getInstance(), jobEventListenerFactory);
+        JobSpecification spec = compiler.createJob(AsterixAppContextInfo.getInstance(), jobEventListenerFactory);
 
         if (pc.isPrintJob()) {
             switch (pdf) {
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
index 35577fc..65eae64 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContext.java
@@ -1,13 +1,20 @@
 package edu.uci.ics.asterix.api.common;
 
 import java.io.IOException;
-import java.util.logging.Level;
+import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
+import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.context.AsterixFileMapManager;
 import edu.uci.ics.asterix.common.context.ConstantMergePolicy;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.asterix.transaction.management.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
@@ -23,7 +30,6 @@
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
 import edu.uci.ics.hyracks.storage.am.common.dataflow.IndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationCallbackProvider;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMMergePolicy;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMOperationTrackerFactory;
@@ -43,12 +49,19 @@
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactory;
 import edu.uci.ics.hyracks.storage.common.file.ResourceIdFactoryProvider;
 
-public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext {
+public class AsterixAppRuntimeContext implements IAsterixAppRuntimeContext,  IAsterixPropertiesProvider {
     private static final int DEFAULT_BUFFER_CACHE_PAGE_SIZE = 32768;
     private static final int DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET = 1024 * 1024 * 1024; // 1GB
     private static final int DEFAULT_MAX_OPEN_FILES = Integer.MAX_VALUE;
+
     private final INCApplicationContext ncApplicationContext;
 
+    private AsterixCompilerProperties compilerProperties;
+    private AsterixExternalProperties externalProperties;
+    private AsterixMetadataProperties metadataProperties;
+    private AsterixStorageProperties storageProperties;
+    private AsterixTransactionProperties txnProperties;
+
     private IIndexLifecycleManager indexLifecycleManager;
     private IFileMapManager fileMapManager;
     private IBufferCache bufferCache;
@@ -68,24 +81,29 @@
         this.ncApplicationContext = ncApplicationContext;
     }
 
-    public void initialize() throws IOException, ACIDException {
-        int pageSize = getBufferCachePageSize();
-        int numPages = getBufferCacheNumPages();
+    public void initialize() throws IOException, ACIDException, AsterixException {
+        AsterixPropertiesAccessor propertiesAccessor = new AsterixPropertiesAccessor();
+        compilerProperties = new AsterixCompilerProperties(propertiesAccessor);
+        externalProperties = new AsterixExternalProperties(propertiesAccessor);
+        metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
+        storageProperties = new AsterixStorageProperties(propertiesAccessor);
+        txnProperties = new AsterixTransactionProperties(propertiesAccessor);
+
+        Logger.getLogger("edu.uci.ics").setLevel(externalProperties.getLogLevel());
 
         fileMapManager = new AsterixFileMapManager();
         ICacheMemoryAllocator allocator = new HeapBufferAllocator();
         IPageReplacementStrategy prs = new ClockPageReplacementStrategy();
-        ioManager = ncApplicationContext.getRootContext().getIOManager();
-        indexLifecycleManager = new IndexLifecycleManager(DEFAULT_LIFECYCLEMANAGER_MEMORY_BUDGET);
-        IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProviderForRecovery(
-                this);
-        txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider);
         IPageCleanerPolicy pcp = new DelayPageCleanerPolicy(600000);
-        bufferCache = new BufferCache(ioManager, allocator, prs, pcp, fileMapManager, pageSize, numPages,
-                DEFAULT_MAX_OPEN_FILES);
+        ioManager = ncApplicationContext.getRootContext().getIOManager();
+        bufferCache = new BufferCache(ioManager, allocator, prs, pcp, fileMapManager,
+                storageProperties.getBufferCachePageSize(), storageProperties.getBufferCacheNumPages(),
+                storageProperties.getBufferCacheMaxOpenFiles());
+
+        indexLifecycleManager = new IndexLifecycleManager(storageProperties.getMemoryComponentGlobalBudget());
 
         lsmIOScheduler = SynchronousScheduler.INSTANCE;
-        mergePolicy = new ConstantMergePolicy(3, this);
+        mergePolicy = new ConstantMergePolicy(storageProperties.getLSMIndexMergeThreshold(), this);
         lsmBTreeOpTrackerFactory = new IndexOperationTrackerFactory(LSMBTreeIOOperationCallbackFactory.INSTANCE);
         lsmRTreeOpTrackerFactory = new IndexOperationTrackerFactory(LSMRTreeIOOperationCallbackFactory.INSTANCE);
         lsmInvertedIndexOpTrackerFactory = new IndexOperationTrackerFactory(
@@ -96,6 +114,10 @@
         localResourceRepository = (PersistentLocalResourceRepository) persistentLocalResourceRepositoryFactory
                 .createRepository();
         resourceIdFactory = (new ResourceIdFactoryProvider(localResourceRepository)).createResourceIdFactory();
+
+        IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider = new AsterixAppRuntimeContextProviderForRecovery(
+                this);
+        txnSubsystem = new TransactionSubsystem(ncApplicationContext.getNodeId(), asterixAppRuntimeContextProvider);
         isShuttingdown = false;
     }
 
@@ -107,49 +129,6 @@
         this.isShuttingdown = isShuttingdown;
     }
 
-    private int getBufferCachePageSize() {
-        int pageSize = DEFAULT_BUFFER_CACHE_PAGE_SIZE;
-        String pageSizeStr = System.getProperty(GlobalConfig.BUFFER_CACHE_PAGE_SIZE_PROPERTY, null);
-        if (pageSizeStr != null) {
-            try {
-                pageSize = Integer.parseInt(pageSizeStr);
-            } catch (NumberFormatException nfe) {
-                if (GlobalConfig.ASTERIX_LOGGER.isLoggable(Level.WARNING)) {
-                    GlobalConfig.ASTERIX_LOGGER.warning("Wrong buffer cache page size argument. "
-                            + "Using default value: " + pageSize);
-                }
-            }
-        }
-
-        if (GlobalConfig.ASTERIX_LOGGER.isLoggable(Level.INFO)) {
-            GlobalConfig.ASTERIX_LOGGER.info("Buffer cache page size: " + pageSize);
-        }
-
-        return pageSize;
-    }
-
-    private int getBufferCacheNumPages() {
-        int numPages = GlobalConfig.DEFAULT_BUFFER_CACHE_NUM_PAGES;
-        String numPagesStr = System.getProperty(GlobalConfig.BUFFER_CACHE_NUM_PAGES_PROPERTY, null);
-        if (numPagesStr != null) {
-            try {
-                numPages = Integer.parseInt(numPagesStr);
-            } catch (NumberFormatException nfe) {
-                if (GlobalConfig.ASTERIX_LOGGER.isLoggable(Level.WARNING)) {
-                    GlobalConfig.ASTERIX_LOGGER.warning("Wrong buffer cache size argument. " + "Using default value: "
-                            + numPages);
-                }
-                return numPages;
-            }
-        }
-
-        if (GlobalConfig.ASTERIX_LOGGER.isLoggable(Level.INFO)) {
-            GlobalConfig.ASTERIX_LOGGER.info("Buffer cache size (number of pages): " + numPages);
-        }
-
-        return numPages;
-    }
-
     public void deinitialize() throws HyracksDataException {
         bufferCache.close();
         for (IIndex index : indexLifecycleManager.getOpenIndexes()) {
@@ -177,6 +156,10 @@
         return mergePolicy;
     }
 
+    public double getBloomFilterFalsePositiveRate() {
+        return storageProperties.getBloomFilterFalsePositiveRate();
+    }
+
     public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory() {
         return lsmBTreeOpTrackerFactory;
     }
@@ -205,4 +188,28 @@
         return ioManager;
     }
 
+    @Override
+    public AsterixStorageProperties getStorageProperties() {
+        return storageProperties;
+    }
+
+    @Override
+    public AsterixTransactionProperties getTransactionProperties() {
+        return txnProperties;
+    }
+
+    @Override
+    public AsterixCompilerProperties getCompilerProperties() {
+        return compilerProperties;
+    }
+
+    @Override
+    public AsterixMetadataProperties getMetadataProperties() {
+        return metadataProperties;
+    }
+
+    @Override
+    public AsterixExternalProperties getExternalProperties() {
+        return externalProperties;
+    }
 }
\ No newline at end of file
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
index c0769df..8c73a63 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixAppRuntimeContextProviderForRecovery.java
@@ -42,6 +42,11 @@
     }
 
     @Override
+    public double getBloomFilterFalsePositiveRate() {
+        return asterixAppRuntimeContext.getBloomFilterFalsePositiveRate();
+    }
+
+    @Override
     public ILSMMergePolicy getLSMMergePolicy() {
         return asterixAppRuntimeContext.getLSMMergePolicy();
     }
@@ -105,6 +110,4 @@
         return null;
     }
 
-   
-
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
index d207acb..272050f 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -48,6 +48,7 @@
         ncConfig1.clusterNetIPAddress = "127.0.0.1";
         ncConfig1.dataIPAddress = "127.0.0.1";
         ncConfig1.datasetIPAddress = "127.0.0.1";
+        ncConfig1.resultHistorySize = 1000;
         ncConfig1.nodeId = NC1_ID;
         ncConfig1.appNCMainClass = NCApplicationEntryPoint.class.getName();
         nc1 = new NodeControllerService(ncConfig1);
@@ -59,6 +60,7 @@
         ncConfig2.clusterNetIPAddress = "127.0.0.1";
         ncConfig2.dataIPAddress = "127.0.0.1";
         ncConfig2.datasetIPAddress = "127.0.0.1";
+        ncConfig2.resultHistorySize = 1000;
         ncConfig2.nodeId = NC2_ID;
         ncConfig2.appNCMainClass = NCApplicationEntryPoint.class.getName();
         nc2 = new NodeControllerService(ncConfig2);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
index 12fa51a..e729742 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/APIServlet.java
@@ -13,13 +13,23 @@
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import edu.uci.ics.asterix.api.common.APIFramework;
 import edu.uci.ics.asterix.api.common.APIFramework.DisplayFormat;
+import edu.uci.ics.asterix.api.common.Job;
 import edu.uci.ics.asterix.api.common.SessionConfig;
 import edu.uci.ics.asterix.aql.base.Statement;
 import edu.uci.ics.asterix.aql.parser.AQLParser;
 import edu.uci.ics.asterix.aql.parser.ParseException;
 import edu.uci.ics.asterix.aql.parser.TokenMgrError;
 import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.asterix.api.common.APIFramework.DisplayFormat;
+import edu.uci.ics.asterix.api.common.SessionConfig;
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.result.ResultReader;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
index 2c15578..2cfa59e 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/http/servlet/RESTAPIServlet.java
@@ -30,8 +30,6 @@
 import edu.uci.ics.asterix.api.common.SessionConfig;
 import edu.uci.ics.asterix.aql.base.Statement;
 import edu.uci.ics.asterix.aql.base.Statement.Kind;
-import edu.uci.ics.asterix.aql.parser.AQLParser;
-import edu.uci.ics.asterix.aql.parser.ParseException;
 import edu.uci.ics.asterix.aql.translator.AqlTranslator;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -40,6 +38,18 @@
 import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
 import edu.uci.ics.hyracks.api.dataset.IHyracksDataset;
 import edu.uci.ics.hyracks.client.dataset.HyracksDataset;
+import edu.uci.ics.asterix.api.common.APIFramework;
+import edu.uci.ics.asterix.api.common.APIFramework.DisplayFormat;
+import edu.uci.ics.asterix.api.common.Job;
+import edu.uci.ics.asterix.api.common.SessionConfig;
+import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.parser.AQLParser;
+import edu.uci.ics.asterix.aql.parser.ParseException;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 
 abstract class RESTAPIServlet extends HttpServlet {
     private static final long serialVersionUID = 1L;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
index cb786d7..0fbe9fa 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/api/java/AsterixJavaClient.java
@@ -9,6 +9,11 @@
 import edu.uci.ics.asterix.api.common.Job;
 import edu.uci.ics.asterix.api.common.SessionConfig;
 import edu.uci.ics.asterix.aql.base.Statement;
+import edu.uci.ics.asterix.aql.translator.AqlTranslator;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.metadata.MetadataManager;
+import edu.uci.ics.hyracks.api.client.IHyracksClientConnection;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
 import edu.uci.ics.asterix.aql.parser.AQLParser;
 import edu.uci.ics.asterix.aql.parser.ParseException;
 import edu.uci.ics.asterix.aql.translator.AqlTranslator;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
index eea35cb..e84dedf 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/aql/translator/AqlTranslator.java
@@ -59,12 +59,14 @@
 import edu.uci.ics.asterix.aql.expression.TypeDropStatement;
 import edu.uci.ics.asterix.aql.expression.WriteFromQueryResultStatement;
 import edu.uci.ics.asterix.aql.expression.WriteStatement;
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.file.DatasetOperations;
+import edu.uci.ics.asterix.file.DataverseOperations;
 import edu.uci.ics.asterix.file.FeedOperations;
 import edu.uci.ics.asterix.file.IndexOperations;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
@@ -765,6 +767,7 @@
                     jobsToExecute.add(DatasetOperations.createDropDatasetJobSpec(cds, metadataProvider));
                 }
             }
+            jobsToExecute.add(DataverseOperations.createDropDataverseJobSpec(dv, metadataProvider));
 
             //#. mark PendingDropOp on the dataverse record by 
             //   first, deleting the dataverse record from the DATAVERSE_DATASET
@@ -1356,6 +1359,8 @@
             }
             bfs.initialize(metadataProvider.getMetadataTxnContext(), dataset);
             cbfs.setQuery(bfs.getQuery());
+            metadataProvider.getConfig().put(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS, "" + Boolean.TRUE);
+
             JobSpecification compiled = rewriteCompileQuery(metadataProvider, bfs.getQuery(), cbfs);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
index 2ddab49..612d523 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DatasetOperations.java
@@ -22,6 +22,8 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.api.common.Job;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
@@ -118,14 +120,16 @@
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataset.getDataverseName(), datasetName,
                         datasetName);
+        AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor primaryBtreeDrop = new IndexDropOperatorDescriptor(specPrimary,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES));
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(specPrimary, primaryBtreeDrop,
                 splitsAndConstraint.second);
 
@@ -163,10 +167,11 @@
         }
         LOGGER.info("CREATING File Splits: " + sb.toString());
 
+        AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
-                comparatorFactories, blooFilterKeyFields, true, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                comparatorFactories, blooFilterKeyFields, true, storageProperties.getMemoryComponentPageSize(),
+                storageProperties.getMemoryComponentNumPages());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
@@ -176,8 +181,9 @@
                 new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), localResourceFactoryProvider,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
+                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, indexCreateOp,
                 splitsAndConstraint.second);
@@ -258,6 +264,7 @@
             numElementsHint = Long.parseLong(dataset.getHints().get("CARDINALITY"));
         }
 
+        AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         TreeIndexBulkLoadOperatorDescriptor btreeBulkLoad = new TreeIndexBulkLoadOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 splitsAndConstraint.first, typeTraits, comparatorFactories, blooFilterKeyFields, fieldPermutation,
@@ -265,8 +272,9 @@
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), NoOpOperationCallbackFactory.INSTANCE);
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, btreeBulkLoad,
                 splitsAndConstraint.second);
 
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/DataverseOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DataverseOperations.java
new file mode 100644
index 0000000..0654ff7
--- /dev/null
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/DataverseOperations.java
@@ -0,0 +1,22 @@
+package edu.uci.ics.asterix.file;
+
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
+import edu.uci.ics.asterix.metadata.entities.Dataverse;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
+import edu.uci.ics.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraintHelper;
+import edu.uci.ics.hyracks.algebricks.common.utils.Pair;
+import edu.uci.ics.hyracks.api.job.JobSpecification;
+import edu.uci.ics.hyracks.dataflow.std.file.FileRemoveOperatorDescriptor;
+import edu.uci.ics.hyracks.dataflow.std.file.IFileSplitProvider;
+
+public class DataverseOperations {
+    public static JobSpecification createDropDataverseJobSpec(Dataverse dataverse, AqlMetadataProvider metadata) {
+        JobSpecification jobSpec = new JobSpecification();
+        Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadata
+                .splitProviderAndPartitionConstraintsForDataverse(dataverse.getDataverseName());
+        FileRemoveOperatorDescriptor frod = new FileRemoveOperatorDescriptor(jobSpec, splitsAndConstraint.first);
+        AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(jobSpec, frod, splitsAndConstraint.second);
+        jobSpec.addRoot(frod);
+        return jobSpec;
+    }
+}
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
index a4d5dbe..b0a0177 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/IndexOperations.java
@@ -1,6 +1,7 @@
 package edu.uci.ics.asterix.file;
 
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.OptimizationConfUtil;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.MetadataException;
@@ -48,14 +49,16 @@
 
         Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitsAndConstraint = metadataProvider
                 .splitProviderAndPartitionConstraintsForInternalOrFeedDataset(dataverseName, datasetName, indexName);
+        AsterixStorageProperties storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
         IndexDropOperatorDescriptor btreeDrop = new IndexDropOperatorDescriptor(spec,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 splitsAndConstraint.first, new LSMBTreeDataflowHelperFactory(
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES));
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()));
         AlgebricksPartitionConstraintHelper
                 .setPartitionConstraintInJobSpec(spec, btreeDrop, splitsAndConstraint.second);
         spec.addRoot(btreeDrop);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
index 8839001..fb83871 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryBTreeCreator.java
@@ -1,6 +1,7 @@
 package edu.uci.ics.asterix.file;
 
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.transaction.management.resource.ILocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
@@ -26,18 +27,20 @@
 
 public class SecondaryBTreeCreator extends SecondaryIndexCreator {
 
-    protected SecondaryBTreeCreator(PhysicalOptimizationConfig physOptConf) {
-        super(physOptConf);
+    protected SecondaryBTreeCreator(PhysicalOptimizationConfig physOptConf,
+            IAsterixPropertiesProvider propertiesProvider) {
+        super(physOptConf, propertiesProvider);
     }
 
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
 
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(
                 secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, secondaryBloomFilterKeyFields, false,
-                GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMBTreeResource);
 
@@ -48,8 +51,9 @@
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), localResourceFactoryProvider,
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
+                        storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
                 NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
@@ -80,13 +84,17 @@
         // Sort by secondary keys.
         ExternalSortOperatorDescriptor sortOp = createSortOp(spec, secondaryComparatorFactories, secondaryRecDesc);
 
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         // Create secondary BTree bulk load op.
-        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(spec, numSecondaryKeys,
+        TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
+                spec,
+                numSecondaryKeys,
                 new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), BTree.DEFAULT_FILL_FACTOR);
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
+                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index 253b675..5a7de34 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -19,8 +19,10 @@
 import java.io.IOException;
 import java.util.List;
 
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -103,29 +105,34 @@
     protected RecordDescriptor secondaryRecDesc;
     protected ICopyEvaluatorFactory[] secondaryFieldAccessEvalFactories;
 
+    protected IAsterixPropertiesProvider propertiesProvider;
+
     // Prevent public construction. Should be created via createIndexCreator().
-    protected SecondaryIndexCreator(PhysicalOptimizationConfig physOptConf) {
+    protected SecondaryIndexCreator(PhysicalOptimizationConfig physOptConf,
+            IAsterixPropertiesProvider propertiesProvider) {
         this.physOptConf = physOptConf;
+        this.propertiesProvider = propertiesProvider;
     }
 
     public static SecondaryIndexCreator createIndexCreator(CompiledCreateIndexStatement createIndexStmt,
             AqlMetadataProvider metadataProvider, PhysicalOptimizationConfig physOptConf) throws AsterixException,
             AlgebricksException {
+        IAsterixPropertiesProvider asterixPropertiesProvider = AsterixAppContextInfo.getInstance();
         SecondaryIndexCreator indexCreator = null;
         switch (createIndexStmt.getIndexType()) {
             case BTREE: {
-                indexCreator = new SecondaryBTreeCreator(physOptConf);
+                indexCreator = new SecondaryBTreeCreator(physOptConf, asterixPropertiesProvider);
                 break;
             }
             case RTREE: {
-                indexCreator = new SecondaryRTreeCreator(physOptConf);
+                indexCreator = new SecondaryRTreeCreator(physOptConf, asterixPropertiesProvider);
                 break;
             }
             case WORD_INVIX:
             case NGRAM_INVIX:
             case FUZZY_WORD_INVIX:
             case FUZZY_NGRAM_INVIX: {
-                indexCreator = new SecondaryInvertedIndexCreator(physOptConf);
+                indexCreator = new SecondaryInvertedIndexCreator(physOptConf, asterixPropertiesProvider);
                 break;
             }
             default: {
@@ -266,6 +273,7 @@
         int[] lowKeyFields = null;
         // +Infinity
         int[] highKeyFields = null;
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         BTreeSearchOperatorDescriptor primarySearchOp = new BTreeSearchOperatorDescriptor(spec, primaryRecDesc,
                 AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER, AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER,
                 primaryFileSplitProvider, primaryRecDesc.getTypeTraits(), primaryComparatorFactories,
@@ -273,8 +281,10 @@
                 new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                        GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), false, NoOpOperationCallbackFactory.INSTANCE);
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
+                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()), false,
+                NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, primarySearchOp,
                 primaryPartitionConstraint);
         return primarySearchOp;
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
index 78ffbc9..a2e28da 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryInvertedIndexCreator.java
@@ -2,8 +2,9 @@
 
 import java.util.List;
 
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Index;
@@ -60,8 +61,9 @@
     private RecordDescriptor tokenKeyPairRecDesc;
     private boolean isPartitioned;
 
-    protected SecondaryInvertedIndexCreator(PhysicalOptimizationConfig physOptConf) {
-        super(physOptConf);
+    protected SecondaryInvertedIndexCreator(PhysicalOptimizationConfig physOptConf,
+            IAsterixPropertiesProvider propertiesProvider) {
+        super(physOptConf, propertiesProvider);
     }
 
     @Override
@@ -150,10 +152,12 @@
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
 
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMInvertedIndexLocalResourceMetadata(invListsTypeTraits,
                 primaryComparatorFactories, tokenTypeTraits, tokenComparatorFactories, tokenizerFactory,
-                GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES, isPartitioned);
+                storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                isPartitioned);
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMInvertedIndexResource);
 
@@ -259,20 +263,23 @@
     }
 
     private IIndexDataflowHelperFactory createDataflowHelperFactory() {
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         if (!isPartitioned) {
             return new LSMInvertedIndexDataflowHelperFactory(
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                    storageProperties.getBloomFilterFalsePositiveRate());
         } else {
             return new PartitionedLSMInvertedIndexDataflowHelperFactory(
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                     AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                    GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                    storageProperties.getBloomFilterFalsePositiveRate());
         }
     }
 }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
index 5c69cba..ad61cbe 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryRTreeCreator.java
@@ -2,7 +2,8 @@
 
 import java.util.List;
 
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
@@ -49,20 +50,22 @@
     protected int numNestedSecondaryKeyFields;
     protected ATypeTag keyType;
 
-    protected SecondaryRTreeCreator(PhysicalOptimizationConfig physOptConf) {
-        super(physOptConf);
+    protected SecondaryRTreeCreator(PhysicalOptimizationConfig physOptConf,
+            IAsterixPropertiesProvider propertiesProvider) {
+        super(physOptConf, propertiesProvider);
     }
 
     @Override
     public JobSpecification buildCreationJobSpec() throws AsterixException, AlgebricksException {
         JobSpecification spec = new JobSpecification();
 
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         //prepare a LocalResourceMetadata which will be stored in NC's local resource repository
         ILocalResourceMetadata localResourceMetadata = new LSMRTreeLocalResourceMetadata(
                 secondaryRecDesc.getTypeTraits(), secondaryComparatorFactories, primaryComparatorFactories,
                 valueProviderFactories, RTreePolicyType.RTREE, AqlMetadataProvider.proposeLinearizer(keyType,
-                        secondaryComparatorFactories.length), GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE,
-                GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                        secondaryComparatorFactories.length), storageProperties.getMemoryComponentPageSize(),
+                storageProperties.getMemoryComponentNumPages());
         ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                 localResourceMetadata, LocalResource.LSMRTreeResource);
 
@@ -74,9 +77,10 @@
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
-                                keyType, secondaryComparatorFactories.length),
-                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
-                localResourceFactoryProvider, NoOpOperationCallbackFactory.INSTANCE);
+                                keyType, secondaryComparatorFactories.length), storageProperties
+                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()), localResourceFactoryProvider,
+                NoOpOperationCallbackFactory.INSTANCE);
         AlgebricksPartitionConstraintHelper.setPartitionConstraintInJobSpec(spec, secondaryIndexCreateOp,
                 secondaryPartitionConstraint);
         spec.addRoot(secondaryIndexCreateOp);
@@ -149,6 +153,7 @@
             selectOp = createFilterNullsSelectOp(spec, numNestedSecondaryKeyFields);
         }
 
+        AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
         // Create secondary RTree bulk load op.
         TreeIndexBulkLoadOperatorDescriptor secondaryBulkLoadOp = createTreeIndexBulkLoadOp(
                 spec,
@@ -158,9 +163,9 @@
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                         AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, AqlMetadataProvider.proposeLinearizer(
-                                keyType, secondaryComparatorFactories.length),
-                        GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
-                BTree.DEFAULT_FILL_FACTOR);
+                                keyType, secondaryComparatorFactories.length), storageProperties
+                                .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                        storageProperties.getBloomFilterFalsePositiveRate()), BTree.DEFAULT_FILL_FACTOR);
 
         // Connect the operators.
         spec.connect(new OneToOneConnectorDescriptor(spec), keyProviderOp, 0, primaryScanOp, 0);
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
index 592009c..2ac6051 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/CCApplicationEntryPoint.java
@@ -13,13 +13,13 @@
 import edu.uci.ics.asterix.api.http.servlet.QueryResultAPIServlet;
 import edu.uci.ics.asterix.api.http.servlet.QueryStatusAPIServlet;
 import edu.uci.ics.asterix.api.http.servlet.UpdateAPIServlet;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
 import edu.uci.ics.asterix.common.api.AsterixContextInfo;
-import edu.uci.ics.asterix.common.config.AsterixProperties;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 import edu.uci.ics.asterix.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.bootstrap.AsterixStateProxy;
-import edu.uci.ics.asterix.transaction.management.service.transaction.StorageContext;
 import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
 import edu.uci.ics.hyracks.api.application.ICCApplicationEntryPoint;
 import edu.uci.ics.hyracks.api.client.HyracksConnection;
@@ -30,10 +30,6 @@
 
     private static final String HYRACKS_CONNECTION_ATTR = "edu.uci.ics.asterix.HYRACKS_CONNECTION";
 
-    private static final int DEFAULT_WEB_SERVER_PORT = 19001;
-
-    private static final int DEFAULT_JSON_API_SERVER_PORT = 19101;
-
     private Server webServer;
     private Server jsonAPIServer;
     private static IAsterixStateProxy proxy;
@@ -46,17 +42,18 @@
             LOGGER.info("Starting Asterix cluster controller");
         }
 
+        AsterixAppContextInfo.initialize(appCtx);
+
         proxy = AsterixStateProxy.registerRemoteObject();
-        proxy.setAsterixProperties(AsterixProperties.INSTANCE);
         appCtx.setDistributedState(proxy);
 
-        MetadataManager.INSTANCE = new MetadataManager(proxy);
+        AsterixMetadataProperties metadataProperties = AsterixAppContextInfo.getInstance().getMetadataProperties();
+        MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
 
-        setupWebServer();
+        AsterixExternalProperties externalProperties = AsterixAppContextInfo.getInstance().getExternalProperties();
+        setupWebServer(externalProperties);
         webServer.start();
-
-        // Setup and start the web interface
-        setupJSONAPIServer();
+        setupJSONAPIServer(externalProperties);
         jsonAPIServer.start();
 
         AsterixContextInfo.initialize(ccAppCtx);
@@ -79,10 +76,9 @@
         return new HyracksConnection(strIP, port);
     }
 
-    private void setupWebServer() throws Exception {
-        int port = Integer.parseInt((String) AsterixProperties.INSTANCE
-                .getProperty(AsterixProperties.AsterixConfigurationKeys.WEB_INTERFACE_PORT));
-        webServer = new Server(port);
+    private void setupWebServer(AsterixExternalProperties externalProperties) throws Exception {
+
+        webServer = new Server(externalProperties.getWebInterfacePort());
 
         ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
         context.setContextPath("/");
@@ -94,13 +90,8 @@
         context.addServlet(new ServletHolder(new APIServlet()), "/*");
     }
 
-    private void setupJSONAPIServer() throws Exception {
-        String portStr = System.getProperty(GlobalConfig.JSON_API_SERVER_PORT_PROPERTY);
-        int port = DEFAULT_JSON_API_SERVER_PORT;
-        if (portStr != null) {
-            port = Integer.parseInt(portStr);
-        }
-        jsonAPIServer = new Server(port);
+    private void setupJSONAPIServer(AsterixExternalProperties externalProperties) throws Exception {
+        jsonAPIServer = new Server(externalProperties.getAPIServerPort());
 
         ServletContextHandler context = new ServletContextHandler(ServletContextHandler.SESSIONS);
         context.setContextPath("/");
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 71639ca..4569088 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -7,6 +7,8 @@
 
 import edu.uci.ics.asterix.api.common.AsterixAppRuntimeContext;
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager.SystemState;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -87,32 +89,31 @@
     @Override
     public void notifyStartupComplete() throws Exception {
         IAsterixStateProxy proxy = (IAsterixStateProxy) ncApplicationContext.getDistributedState();
+        AsterixMetadataProperties metadataProperties = ((IAsterixPropertiesProvider)runtimeContext).getMetadataProperties();
 
         if (systemState == SystemState.NEW_UNIVERSE) {
-            PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
-                    .getLocalResourceRepository();
-
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("nodeid" + nodeId);
-                LOGGER.info("proxy" + proxy);
-                LOGGER.info("stores" + proxy.getAsterixProperties().getStores());
-                LOGGER.info("store" + proxy.getAsterixProperties().getStores().get(nodeId)[0]);
+                LOGGER.info("System state: " + SystemState.NEW_UNIVERSE);
+                LOGGER.info("Node ID: " + nodeId);
+                LOGGER.info("Stores: " + metadataProperties.getStores());
+                LOGGER.info("Root Metadata Store: " + metadataProperties.getStores().get(nodeId)[0]);
             }
 
-            localResourceRepository.initialize(nodeId, proxy.getAsterixProperties().getStores().get(nodeId)[0], true,
-                    null);
+            PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
+                    .getLocalResourceRepository();
+            localResourceRepository.initialize(nodeId, metadataProperties.getStores().get(nodeId)[0], true, null);
         }
 
-        isMetadataNode = nodeId.equals(proxy.getAsterixProperties().getMetadataNodeName());
+        isMetadataNode = nodeId.equals(metadataProperties.getMetadataNodeName());
         if (isMetadataNode) {
             registerRemoteMetadataNode(proxy);
 
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info("Bootstrapping metadata");
             }
-            MetadataManager.INSTANCE = new MetadataManager(proxy);
+            MetadataManager.INSTANCE = new MetadataManager(proxy, metadataProperties);
             MetadataManager.INSTANCE.init();
-            MetadataBootstrap.startUniverse(proxy.getAsterixProperties(), ncApplicationContext,
+            MetadataBootstrap.startUniverse( ((IAsterixPropertiesProvider)runtimeContext), ncApplicationContext,
                     systemState == SystemState.NEW_UNIVERSE);
             MetadataBootstrap.startDDLRecovery();
         }
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
index 8b11f2b..48ba0b1 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/result/ResultReader.java
@@ -60,4 +60,4 @@
     public IFrameTupleAccessor getFrameTupleAccessor() {
         return frameTupleAccessor;
     }
-}
\ No newline at end of file
+}
diff --git a/asterix-app/src/main/resources/asterix-build-configuration.xml b/asterix-app/src/main/resources/asterix-build-configuration.xml
index bd984f0..3e1b4b2 100644
--- a/asterix-app/src/main/resources/asterix-build-configuration.xml
+++ b/asterix-app/src/main/resources/asterix-build-configuration.xml
@@ -9,8 +9,8 @@
      <storeDirs>nc2data</storeDirs> 
   </store>
   <property>
-     <name>log_level</name>
+     <name>log.level</name>
      <value>WARNING</value>
-     <description></description>
+     <description>Log level for running tests/build</description>
   </property>
 </asterixConfiguration>
diff --git a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
index 5ca397b..2289484 100644
--- a/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
+++ b/asterix-app/src/test/java/edu/uci/ics/asterix/test/runtime/ExecutionTest.java
@@ -94,4 +94,4 @@
     public void test() throws Exception {
         TestsUtils.executeTest(PATH_ACTUAL, tcCtx);
     }
-}
\ No newline at end of file
+}
diff --git a/asterix-app/src/test/resources/AQLTS/queries/createInternalDataSet.aql b/asterix-app/src/test/resources/AQLTS/queries/createInternalDataSet.aql
new file mode 100644
index 0000000..f141e45
--- /dev/null
+++ b/asterix-app/src/test/resources/AQLTS/queries/createInternalDataSet.aql
@@ -0,0 +1,3 @@
+create dataset ds1(someType) primary key id;
+create internal dataset ds2(someType) primary key id;
+
diff --git a/asterix-app/src/test/resources/AQLTS/queries/fieldAccessor.aql b/asterix-app/src/test/resources/AQLTS/queries/fieldAccessor.aql
new file mode 100644
index 0000000..3995374
--- /dev/null
+++ b/asterix-app/src/test/resources/AQLTS/queries/fieldAccessor.aql
@@ -0,0 +1,3 @@
+let $bla := { "name" : "value" }
+return
+  $bla."name" = $bla.name
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/AQLTS/queries/functionDecl3.aql b/asterix-app/src/test/resources/AQLTS/queries/functionDecl3.aql
new file mode 100644
index 0000000..80629be5
--- /dev/null
+++ b/asterix-app/src/test/resources/AQLTS/queries/functionDecl3.aql
@@ -0,0 +1,5 @@
+declare function "function with spaces"($a, $b) {
+  "string with spaces"
+};
+
+"function with spaces" (1, 2)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/AQLTS/queries/utf-8.aql b/asterix-app/src/test/resources/AQLTS/queries/utf-8.aql
new file mode 100644
index 0000000..b9c58ea
--- /dev/null
+++ b/asterix-app/src/test/resources/AQLTS/queries/utf-8.aql
@@ -0,0 +1,3 @@
+string-to-codepoint("äöß");
+string-to-codepoint("迎");
+/* currently fails (issue 277) string-to-codepoint("欢")  */
diff --git a/asterix-app/src/test/resources/AQLTS/queries/variables.aql b/asterix-app/src/test/resources/AQLTS/queries/variables.aql
new file mode 100644
index 0000000..57bfa6b
--- /dev/null
+++ b/asterix-app/src/test/resources/AQLTS/queries/variables.aql
@@ -0,0 +1,4 @@
+let $a:=1
+let $b:=1
+return
+  $b-$a
diff --git a/asterix-app/src/test/resources/logging.properties b/asterix-app/src/test/resources/logging.properties
index 6f8cd00..f04eb3de 100644
--- a/asterix-app/src/test/resources/logging.properties
+++ b/asterix-app/src/test/resources/logging.properties
@@ -60,7 +60,8 @@
 # For example, set the com.xyz.foo logger to only log SEVERE
 # messages:
 
+
 edu.uci.ics.asterix.test.level = INFO
 #edu.uci.ics.asterix.level = FINE
-#edu.uci.ics.algebricks.level = FINE
+#edu.uci.ics.hyracks.algebricks.level = FINE
 #edu.uci.ics.hyracks.level = INFO
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.1.ddl.aql
new file mode 100644
index 0000000..c04af59
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : create a dataset using year-month-duration as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:year-month-duration,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.2.update.aql
new file mode 100644
index 0000000..d07ed34
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using year-month-duration as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":year-month-duration("P16Y"), "name": "John"})
+insert into dataset Employee({"id":year-month-duration("-P37M"), "name": "Alex"})
+insert into dataset Employee({"id":year-month-duration("P2013Y"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.3.query.aql
new file mode 100644
index 0000000..a25bfb8
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_0/issue_363_temporal_key_0.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : create a dataset using year-month-duration as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.id > year-month-duration("P350M")
+return $x
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.1.ddl.aql
new file mode 100644
index 0000000..1f59a51
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : create a dataset using datetime as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:datetime,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.2.update.aql
new file mode 100644
index 0000000..0640b52
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using datetime as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":datetime("1900-01-01T00:00:00"), "name": "John"})
+insert into dataset Employee({"id":datetime("2000-01-01T00:00:00"), "name": "Alex"})
+insert into dataset Employee({"id":datetime("2013-01-01T00:00:00"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.3.query.aql
new file mode 100644
index 0000000..2b82717
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_1/issue_363_temporal_key_1.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using datetime as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.id > datetime("2007-07-07T07:07:07.777Z")
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.1.ddl.aql
new file mode 100644
index 0000000..97c62cf
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : create a dataset using time as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:time,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.2.update.aql
new file mode 100644
index 0000000..f98a754
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using time as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":time("03:10:00.493Z"), "name": "John"})
+insert into dataset Employee({"id":time("20:37:19+08:00"), "name": "Alex"})
+insert into dataset Employee({"id":time("21:39:17.948-04:00"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.3.query.aql
new file mode 100644
index 0000000..197b461
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_2/issue_363_temporal_key_2.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using time as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.id > time("07:07:07.777Z")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.1.ddl.aql
new file mode 100644
index 0000000..5e4ae49
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : create a dataset using date as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:date,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.2.update.aql
new file mode 100644
index 0000000..624cdc8
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using date as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":date("2010-01-01"), "name": "John"})
+insert into dataset Employee({"id":date("-1912-10-11"), "name": "Alex"})
+insert into dataset Employee({"id":date("0732-02-02"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.3.query.aql
new file mode 100644
index 0000000..3f9960e
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_3/issue_363_temporal_key_3.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using date as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.id > date("2007-07-07")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.1.ddl.aql
new file mode 100644
index 0000000..9a3da89
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : create a dataset using day-time-duration as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:day-time-duration,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.2.update.aql
new file mode 100644
index 0000000..04a35de
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using day-time-duration as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":day-time-duration("P380DT983M"), "name": "John"})
+insert into dataset Employee({"id":day-time-duration("-P3829H849.392S"), "name": "Alex"})
+insert into dataset Employee({"id":day-time-duration("PT93847M0.392S"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.3.query.aql
new file mode 100644
index 0000000..e45250b
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_key_4/issue_363_temporal_key_4.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using day-time-duration as the primary key 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * Issue        : 363
+ */
+ 
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.id > day-time-duration("P350D")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_0/issue363_temporal_key_0.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_0/issue363_temporal_key_0.1.adm
new file mode 100644
index 0000000..be11d3c
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_0/issue363_temporal_key_0.1.adm
@@ -0,0 +1 @@
+{ "id": year-month-duration("P2013Y"), "name": "Bob" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_1/issue363_temporal_key_1.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_1/issue363_temporal_key_1.1.adm
new file mode 100644
index 0000000..f4a131a
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_1/issue363_temporal_key_1.1.adm
@@ -0,0 +1 @@
+{ "id": datetime("2013-01-01T00:00:00.000Z"), "name": "Bob" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_2/issue363_temporal_key_2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_2/issue363_temporal_key_2.1.adm
new file mode 100644
index 0000000..e9829f4
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_2/issue363_temporal_key_2.1.adm
@@ -0,0 +1,2 @@
+{ "id": time("12:37:19.000Z"), "name": "Alex" }
+{ "id": time("01:39:17.948Z"), "name": "Bob" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_3/issue363_temporal_key_3.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_3/issue363_temporal_key_3.1.adm
new file mode 100644
index 0000000..125c38a
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_3/issue363_temporal_key_3.1.adm
@@ -0,0 +1 @@
+{ "id": date("2010-01-01"), "name": "John" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_4/issue363_temporal_key_4.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_4/issue363_temporal_key_4.1.adm
new file mode 100644
index 0000000..43bda78
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_key_4/issue363_temporal_key_4.1.adm
@@ -0,0 +1 @@
+{ "id": day-time-duration("P380DT16H23M"), "name": "John" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
index 2de89a6..d193dd1 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/meta17/meta17.1.adm
@@ -42,6 +42,7 @@
 { "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
@@ -58,3 +59,4 @@
 { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Feb 08 15:49:29 PST 2013" }
diff --git a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
index 84e2ca8..cecdb85 100644
--- a/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
+++ b/asterix-app/src/test/resources/metadata/results/basic/metadata_datatype/metadata_datatype.1.adm
@@ -42,6 +42,7 @@
 { "DataverseName": "Metadata", "DatatypeName": "circle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "date", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "datetime", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "day-time-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "double", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "float", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
@@ -58,3 +59,4 @@
 { "DataverseName": "Metadata", "DatatypeName": "rectangle", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "string", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
 { "DataverseName": "Metadata", "DatatypeName": "time", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
+{ "DataverseName": "Metadata", "DatatypeName": "year-month-duration", "Derived": null, "Timestamp": "Fri Mar 29 11:19:47 PDT 2013" }
diff --git a/asterix-app/src/test/resources/metadata/testsuite.xml b/asterix-app/src/test/resources/metadata/testsuite.xml
index 47bcafa..63487d4 100644
--- a/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -150,6 +150,31 @@
         <output-dir compare="Text">issue_251_dataset_hint_4</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_key_0">
+        <output-dir compare="Text">issue_363_temporal_key_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_key_1">
+        <output-dir compare="Text">issue_363_temporal_key_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_key_2">
+        <output-dir compare="Text">issue_363_temporal_key_2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_key_3">
+        <output-dir compare="Text">issue_363_temporal_key_3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_key_4">
+        <output-dir compare="Text">issue_363_temporal_key_4</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="exception">
     <test-case FilePath="exception">
diff --git a/asterix-app/src/test/resources/optimizerts/ignore.txt b/asterix-app/src/test/resources/optimizerts/ignore.txt
index babe741..34d7259 100644
--- a/asterix-app/src/test/resources/optimizerts/ignore.txt
+++ b/asterix-app/src/test/resources/optimizerts/ignore.txt
@@ -1 +1,4 @@
-distinct_aggregate.aql
\ No newline at end of file
+distinct_aggregate.aql
+unnest_list_in_subplan.aql
+fj-phase2-with-hints.aql
+fj-dblp-csx.aql
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
index 5e9ac0e..911b63f 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-fuzzyeq-jaccard.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
index dbb9c71..a42fc6b 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard-check.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -24,4 +25,4 @@
 
 for $o in dataset('DBLP')
 where similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)[0]
-return $o
\ No newline at end of file
+return $o
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
index f25fb8b..b11629f 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-jaccard.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
index fb59040..486fe66 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-let.aql
@@ -8,6 +8,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -26,4 +27,4 @@
 for $o in dataset('DBLP')
 let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
 where $jacc[0]
-return $o
\ No newline at end of file
+return $o
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
index b86c6b2..7dfc9f6 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/ngram-jaccard-check-multi-let.aql
@@ -8,6 +8,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -29,4 +30,4 @@
 let $query_tokens := gram-tokens("Transactions for Cooperative Environments", 3, false)
 let $jacc := similarity-jaccard-check($paper_tokens, $query_tokens, 0.5f)
 where $jacc[0]
-return {"Paper": $paper_tokens, "Query": $query_tokens }
\ No newline at end of file
+return {"Paper": $paper_tokens, "Query": $query_tokens }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.aql
index 1a97936..5b2be0c 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.aql
@@ -8,6 +8,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -39,4 +40,4 @@
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
 where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title}
\ No newline at end of file
+return {"atitle": $a.title, "btitle": $b.title}
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
index bf6bb82..d4ec177 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline.aql
@@ -9,6 +9,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -28,4 +29,4 @@
 for $b in dataset('DBLP')
 let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
 where $jacc >= 0.5f and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
\ No newline at end of file
+return {"atitle": $a.title, "btitle": $b.title, "jacc": $jacc}
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard.aql
index dae2a78..1e7df4b 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join-noeqjoin/ngram-jaccard.aql
@@ -8,6 +8,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -37,4 +38,4 @@
 for $b in dataset('CSX')
 where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
       and $a.id < $b.id
-return {"atitle": $a.title, "btitle": $b.title}
\ No newline at end of file
+return {"atitle": $a.title, "btitle": $b.title}
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
index 84fb8f3..931734c 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_01.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -38,4 +39,4 @@
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
 where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
index d0c3c6f..ecbeea7 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_02.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -38,4 +39,4 @@
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
 where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
index a725edc..b437183 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-fuzzyeq-jaccard_03.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -28,4 +29,4 @@
 for $a in dataset('DBLP')
 for $b in dataset('DBLP')
 where gram-tokens($a.title, 3, false) ~= gram-tokens($b.title, 3, false) and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql
index c25e2e3..d554d76 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_01.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -36,4 +37,4 @@
 for $b in dataset('CSX')
 where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
       and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
index 0e9f4d5..d60baad 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_02.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -36,4 +37,4 @@
 for $b in dataset('CSX')
 where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
       and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
index bbd0f51..cc7dded 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_03.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -26,4 +27,4 @@
 for $b in dataset('DBLP')
 where similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)[0]
       and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
index 956f712..e51adbd 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard-check_04.aql
@@ -8,6 +8,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -27,4 +28,4 @@
 for $b in dataset('DBLP')
 let $jacc := similarity-jaccard-check(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false), 0.5f)
 where $jacc[0] and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
\ No newline at end of file
+return {"arec": $a, "brec": $b, "jacc": $jacc[1] }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql
index 742cb69..3893857 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_01.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -36,4 +37,4 @@
 for $b in dataset('CSX')
 where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
       and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
index a620399..372d622 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_02.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -36,4 +37,4 @@
 for $b in dataset('CSX')
 where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
       and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
index d12bf98..568bcfa 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_03.aql
@@ -7,6 +7,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -26,4 +27,4 @@
 for $b in dataset('DBLP')
 where similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false)) >= 0.5f
       and $a.id < $b.id
-return {"arec": $a, "brec": $b }
\ No newline at end of file
+return {"arec": $a, "brec": $b }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
index 6b308c8..c1a08b9 100644
--- a/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
+++ b/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-jaccard_04.aql
@@ -8,6 +8,7 @@
 drop dataverse test if exists;
 create dataverse test;
 use dataverse test;
+set import-private-functions 'true';
 
 create type DBLPType as closed {
   id: int32, 
@@ -27,4 +28,4 @@
 for $b in dataset('DBLP')
 let $jacc := similarity-jaccard(gram-tokens($a.title, 3, false), gram-tokens($b.title, 3, false))
 where $jacc >= 0.5f and $a.id < $b.id
-return {"arec": $a, "brec": $b, "jacc": $jacc }
\ No newline at end of file
+return {"arec": $a, "brec": $b, "jacc": $jacc }
diff --git a/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql b/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql
new file mode 100644
index 0000000..f1b2678
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql
@@ -0,0 +1,44 @@
+/*
+ * Description  : This test case is to verify the fix for issue353
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=353
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use dataverse tpch;
+
+create type LineItemType as closed {
+  l_orderkey: int32, 
+  l_partkey: int32, 
+  l_suppkey: int32, 
+  l_linenumber: int32, 
+  l_quantity: double, 
+  l_extendedprice: double,
+  l_discount: double, 
+  l_tax: double,
+  l_returnflag: string, 
+  l_linestatus: string, 
+  l_shipdate: string,
+  l_commitdate: string, 
+  l_receiptdate: string, 
+  l_shipinstruct: string, 
+  l_shipmode: string, 
+  l_comment: string
+}
+
+create dataset LineItem(LineItemType)
+  primary key l_orderkey, l_linenumber;
+  
+write output to nc1:"/tmp/push_limit.adm";
+
+for $l in dataset('LineItem')
+limit 2
+return {
+"l_returnflag": $l.l_returnflag,
+"l_linestatus": $l.l_linestatus,
+"l_shipmode": $l.l_shipmode
+}
+
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-multipred.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-multipred.plan
index 5a5b715..e87dd88 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-multipred.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-multipred.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_01.plan
index c81244a..8315f2b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- HYBRID_HASH_JOIN [$$7][$$10]  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_02.plan
index abbed4a..6269a2b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join-neg_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- HYBRID_HASH_JOIN [$$9][$$8]  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_01.plan
index 0f3828c..c8fd4cb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_02.plan
index c1c1f33..be25081 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_03.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_03.plan
index 2ecd027..f847790 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
index 2bc4b23..8e1be33 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_05.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_05.plan
index fa60146..d3134f2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_05.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-equi-join_05.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_01.plan
index 0f0c06b..66ec32d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_02.plan
index 5cb2903..28645d5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-ge-join_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_01.plan
index 0f0c06b..66ec32d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_02.plan
index 5cb2903..28645d5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-gt-join_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_01.plan
index 0f0c06b..66ec32d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_02.plan
index 5cb2903..28645d5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-le-join_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_01.plan
index 0f0c06b..66ec32d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_02.plan
index 5cb2903..28645d5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/primary-lt-join_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
index be64ee5..a22212e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multiindex.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multipred.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multipred.plan
index 96632a5..0438a5d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multipred.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join-multipred.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
index fac45f2..77aba01 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
index fac45f2..77aba01 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
index fac45f2..77aba01 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-equi-join_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-01.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-01.plan
index 426a99d..3d9e18f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-02.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-02.plan
index 426a99d..3d9e18f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-03.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-03.plan
index 426a99d..3d9e18f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-04.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-04.plan
index 426a99d..3d9e18f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-05.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-05.plan
index 426a99d..3d9e18f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-05.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-05.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-06.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-06.plan
index 426a99d..3d9e18f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-06.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-06.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-07.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-07.plan
index 426a99d..3d9e18f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-07.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-07.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-08.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-08.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-08.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-08.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-09.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-09.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-09.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-09.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-10.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-10.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-10.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-10.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-11.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-11.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-11.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-11.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-12.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-12.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-12.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-12.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-13.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-13.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-13.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-13.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-14.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-14.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-14.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-14.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-15.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-15.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-15.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-15.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-16.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-16.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-16.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-16.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-17.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-17.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-17.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-17.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-18.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-18.plan
index a78e43f..011cf66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-18.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-18.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-19.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-19.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-19.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-19.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-20.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-20.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-20.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-20.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-21.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-21.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-21.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-21.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-22.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-22.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-22.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-22.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-23.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-23.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-23.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-23.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-24.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-24.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-24.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-24.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-25.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-25.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-25.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-25.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-26.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-26.plan
index 50b1c04..6f13ebb 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-26.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-26.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- BTREE_SEARCH  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-27.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-27.plan
index 40853fa..632a813 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-27.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-27.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-28.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-28.plan
index 5021595..d85d357 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-28.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-28.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-29.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-29.plan
index 5021595..d85d357 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-29.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-29.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-30.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-30.plan
index 5021595..d85d357 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-30.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-primary-30.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-31.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-31.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-31.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-31.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-32.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-32.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-32.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-32.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-33.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-34.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-35.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-36.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-37.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-37.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-37.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-37.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
index 2110116..ef8a923 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-38.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-39.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-39.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-39.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-39.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-40.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-41.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-41.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-41.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-41.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-42.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-43.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-44.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
index 7087a22..4b4412e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-45.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
index 7087a22..4b4412e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-46.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-47.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-48.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
index 7087a22..4b4412e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-49.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
index 7087a22..4b4412e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-50.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
index 7087a22..4b4412e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-51.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
index a48d23a..001e10b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-52.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
index a48d23a..001e10b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-53.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
index a48d23a..001e10b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-54.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
index a48d23a..001e10b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-55.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
index f18d0ad..e12cdd8 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-56.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
index f18d0ad..e12cdd8 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-57.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
index 5b0ae2a..d7667d6 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-58.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
index 7087a22..4b4412e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-59.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
index 8d2b4d8..10b5853 100644
--- a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-complex.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-simple.plan b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-simple.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-simple.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/consolidate-selects-simple.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/count-tweets.plan b/asterix-app/src/test/resources/optimizerts/results/count-tweets.plan
index f24833e..600795f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/count-tweets.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/count-tweets.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan b/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
index a1242ff..ab9747f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/cust_group_no_agg.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- SORT_MERGE_EXCHANGE [$$1(ASC) ]  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- PRE_CLUSTERED_GROUP_BY[$$6]  |PARTITIONED|
             {
               -- AGGREGATE  |LOCAL|
diff --git a/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan b/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
index 80f0a9d..86bafc3 100644
--- a/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/denorm-cust-order.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan b/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
index 0b44c4d..494b208 100644
--- a/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/fj-phase1.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check-panic.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check-panic.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check-panic.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
index 7aa19a6..d7a4c06 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-check.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-panic.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-panic.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance-panic.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
index 7aa19a6..d7a4c06 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-edit-distance.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
index b5ea9d7..021f810 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-edit-distance.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
index 504d5bf..aec97d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
index f0adce8..2a2ccc4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard-check.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
index f0adce8..2a2ccc4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
index 7aa19a6..d7a4c06 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
index 504d5bf..aec97d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard-check.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
index 504d5bf..aec97d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
index 7aa19a6..d7a4c06 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
index 504d5bf..aec97d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard-check.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
index 504d5bf..aec97d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ulist-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
index 504d5bf..aec97d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
index f0adce8..2a2ccc4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard-check.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
index f0adce8..2a2ccc4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
index 96697b6..3028f9a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
index 96697b6..3028f9a 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic-nopanic_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- STREAM_SELECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let-panic.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
index 504d5bf..aec97d2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-edit-distance-check-let.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
index 7ae3ecd..78e761f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-let.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
index db4c5c5..57730fe 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ngram-jaccard-check-multi-let.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
index f0adce8..2a2ccc4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-jaccard-check-let.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
index f0adce8..2a2ccc4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/ulist-jaccard-check-let.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
index 7ae3ecd..78e761f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-let.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
index db4c5c5..57730fe 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/word-jaccard-check-multi-let.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
index 73ba563..0fab288 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance-inline.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan
index 37d4f2c..f7d3cb4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-edit-distance.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
index 22cb67b..1be3fb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-edit-distance.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan
index df958f7..7c74f83 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
index 50966d0..c5534d6 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard-inline.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan
index 0bb698c..fdbf50f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ngram-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
index 73ba563..0fab288 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance-inline.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan
index 37d4f2c..f7d3cb4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-edit-distance.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan
index 22cb67b..1be3fb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-edit-distance.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- UNION_ALL  |PARTITIONED|
       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan
index 294d740..88c4469 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
index 141fd9d..8b6b08b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard-inline.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan
index a70fe23..ba9879d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/olist-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan
index 294d740..88c4469 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
index 141fd9d..8b6b08b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard-inline.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan
index a70fe23..ba9879d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/ulist-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan
index df958f7..7c74f83 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-fuzzyeq-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
index 50966d0..c5534d6 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard-inline.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan
index 0bb698c..fdbf50f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join-noeqjoin/word-jaccard.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
index 2181d95..1800e75 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
index dc42118..abd2c39 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
index 2181d95..1800e75 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
index 43dbff9..dffe718 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-edit-distance_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
index 25c87b0..65be14f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
index 98aa0ef..bfee1a5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
index 98aa0ef..bfee1a5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-edit-distance_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
index 700ebb7..ab5cf83 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
index 69948aa..7877b30 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
index 700ebb7..ab5cf83 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-fuzzyeq-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
index d23a6f5..4b767ab 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
index 1d1cd5f..77b00ff 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
index d23a6f5..4b767ab 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
index d1b8f79..32d7e09 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-jaccard_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
index 2181d95..1800e75 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
index dc42118..abd2c39 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
index 2181d95..1800e75 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
index 63c5436..a04e378 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
index 43dbff9..dffe718 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-edit-distance_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
index 98aa0ef..bfee1a5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
index 25c87b0..65be14f 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
index 98aa0ef..bfee1a5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-edit-distance_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
index 4cef1ae..2a4e9de 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
index a21a7b5..06826c1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
index 4cef1ae..2a4e9de 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-fuzzyeq-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
index 6d8460a..3d0f3f1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
index 4a4430c..156dd43 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
index 6d8460a..3d0f3f1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
index b6a694e..47f5c91 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/olist-jaccard_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
index 4cef1ae..2a4e9de 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
index a21a7b5..06826c1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
index 4cef1ae..2a4e9de 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-fuzzyeq-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
index 6d8460a..3d0f3f1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
index 4a4430c..156dd43 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
index 6d8460a..3d0f3f1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
index 44328ef..7c16cef 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
index b6a694e..47f5c91 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ulist-jaccard_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
index 700ebb7..ab5cf83 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
index 69948aa..7877b30 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
index 700ebb7..ab5cf83 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-fuzzyeq-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
index b77169d..0fced26 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
index 1d1cd5f..77b00ff 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
index d23a6f5..4b767ab 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
index 901e0f4..3f7ecb1 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
index d1b8f79..32d7e09 100644
--- a/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/word-jaccard_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan b/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
index 011f8be..8149181 100644
--- a/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/join-super-key_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan b/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
index 591ddbf..92adb3d 100644
--- a/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/join-super-key_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan b/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan
new file mode 100644
index 0000000..829a245
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- STREAM_LIMIT  |UNPARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$6(ASC), $$7(ASC) ]  |PARTITIONED|
+              -- STREAM_LIMIT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan b/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
index 7dac263..60e9899 100644
--- a/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/loj-super-key_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan b/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
index 63a3b4e..78159b2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/loj-super-key_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan b/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
index f0e1d55..41ae699 100644
--- a/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/nested_loj2.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/nested_loj3.plan b/asterix-app/src/test/resources/optimizerts/results/nested_loj3.plan
index b68ef76..6d0e32e 100644
--- a/asterix-app/src/test/resources/optimizerts/results/nested_loj3.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/nested_loj3.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/noncollocated.plan b/asterix-app/src/test/resources/optimizerts/results/noncollocated.plan
index f1e1f5d..e92a84c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/noncollocated.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/noncollocated.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-aggreg.plan b/asterix-app/src/test/resources/optimizerts/results/orders-aggreg.plan
index f32df8e..08ad861 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-aggreg.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-aggreg.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan b/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
index 089064b..39a9eda 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-composite-index-search.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
index 66b693c..b78849b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search-open.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
index 66b693c..b78849b 100644
--- a/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/orders-index-search.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/prim-idx-search-open.plan b/asterix-app/src/test/resources/optimizerts/results/prim-idx-search-open.plan
index e8b196d..2a91fc5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/prim-idx-search-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/prim-idx-search-open.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/prim-idx-search.plan b/asterix-app/src/test/resources/optimizerts/results/prim-idx-search.plan
index e8b196d..2a91fc5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/prim-idx-search.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/prim-idx-search.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan b/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
index d50a885..b0d8e64 100644
--- a/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/pull_select_above_eq_join.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan b/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
index 94273ad..86bcdac 100644
--- a/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/push-project-through-group.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/q1.plan b/asterix-app/src/test/resources/optimizerts/results/q1.plan
index 2695827..aa5daa2 100644
--- a/asterix-app/src/test/resources/optimizerts/results/q1.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/q1.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_01.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_01.plan
index 2f98801..64fdc8c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
index 2f98801..64fdc8c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
index 2f98801..64fdc8c 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-intersect-point_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
index f7382d8..1845ed7 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index-open.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
index f7382d8..1845ed7 100644
--- a/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/rtree-secondary-index.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
index bf8978b..4f0d931 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-delete-rtree-secondary-index.plan
@@ -3,23 +3,21 @@
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INDEX_INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$29(ASC), $$30(ASC), $$31(ASC), $$32(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
                 -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- INSERT_DELETE  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- INSERT_DELETE  |PARTITIONED|
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+                        -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+                          -- HASH_PARTITION_EXCHANGE [$$14]  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
index e6cf237..1458c66 100644
--- a/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/scan-insert-secondary-index.plan
@@ -3,28 +3,24 @@
     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
       -- INDEX_INSERT_DELETE  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$11(ASC)]  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
               -- STREAM_PROJECT  |PARTITIONED|
-                -- ASSIGN  |PARTITIONED|
-                  -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- INDEX_INSERT_DELETE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- INDEX_INSERT_DELETE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$10(ASC)]  |PARTITIONED|
+                          -- INSERT_DELETE  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- ASSIGN  |PARTITIONED|
-                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- INSERT_DELETE  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$6(ASC)]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+                              -- STABLE_SORT [$$6(ASC)]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$6]  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
                                           -- ASSIGN  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
-                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_01.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_02.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_03.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_04.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_05.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_05.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_05.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_06.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_06.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_06.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_07.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_07.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_07.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_08.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_08.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-let-to-edit-distance-check_08.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_01.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_02.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_03.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_04.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_05.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_05.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_05.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_06.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_06.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_06.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_07.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_07.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_07.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_08.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_08.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/edit-distance-to-edit-distance-check_08.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-edit-distance-check.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-edit-distance-check.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-edit-distance-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-edit-distance-check.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-jaccard-check.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-jaccard-check.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-jaccard-check.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/fuzzyeq-to-jaccard-check.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_01.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_02.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_03.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_04.plan
index c0e93c8..fcd1fd5 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_05.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_05.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_05.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_06.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_06.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_06.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_07.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_07.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_07.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_08.plan
index 4d9d3f9..7ec3440 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_08.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-let-to-jaccard-check_08.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_01.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_01.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_01.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_01.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_02.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_02.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_02.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_02.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_03.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_03.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_03.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_03.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_04.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_04.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_04.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_04.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_05.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_05.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_05.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_05.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_06.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_06.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_06.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_06.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_07.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_07.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_07.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_07.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_08.plan b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_08.plan
index ae0d5bb..06194e4 100644
--- a/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_08.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/similarity/jaccard-to-jaccard-check_08.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_SELECT  |PARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan b/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
index 30a53b3..e9fafd7 100644
--- a/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
+++ b/asterix-app/src/test/resources/optimizerts/results/unnest_list_in_subplan.plan
@@ -1,5 +1,5 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.1.ddl.aql
new file mode 100644
index 0000000..c0bbb1f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.1.ddl.aql
@@ -0,0 +1,10 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:int32,
+name:string ?
+}
+
+create dataset Employee(Emp) primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.2.update.aql
new file mode 100644
index 0000000..fa8fb09
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.2.update.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+insert into dataset Employee({"id":12,"name":"John Doe"});
+insert into dataset Employee({"id":42});
+insert into dataset Employee({"id":22,"name":"John Smith"});
+insert into dataset Employee({"id":19});
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.3.query.aql
new file mode 100644
index 0000000..8104319
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue395/issue395.3.query.aql
@@ -0,0 +1,4 @@
+use dataverse test;
+
+count(for $l in dataset Employee
+return $l.name)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.3.query.aql
new file mode 100644
index 0000000..9b0ed8a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_0/issue412_0.3.query.aql
@@ -0,0 +1,2 @@
+let $l := ["ASTERIX", "Hyracks", null]
+return count($l)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.3.query.aql
new file mode 100644
index 0000000..f9bc64e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/issue412_1/issue412_1.3.query.aql
@@ -0,0 +1,2 @@
+let $l := [1, 60, null]
+return { "count": count($l), "average": avg($l), "sum": sum($l), "min": min($l), "max": max($l) } 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.3.query.aql
new file mode 100644
index 0000000..9819455
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/query-issue400/query-issue400.3.query.aql
@@ -0,0 +1,9 @@
+/*
+ * Description  : This test case is to verify the fix for issue400
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=400
+ * Expected Res : Success
+ * Date         : 8th May 2013
+ */
+ 
+let $l := [[1,2,3,4,5],[6,7,8,9]]
+return count(for $i in $l return $i)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.aql
index 432866f..144f427 100644
--- a/asterix-app/src/test/resources/runtimets/queries/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/aggregate/sum_null-with-pred/sum_null-with-pred.3.query.aql
@@ -6,6 +6,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 sum(for $l in dataset('tdst')
 where not(is-null($l.sal))
diff --git a/asterix-app/src/test/resources/runtimets/queries/boolean/not_01/not_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/boolean/not_01/not_01.3.query.aql
index af1a6bc..059da38 100644
--- a/asterix-app/src/test/resources/runtimets/queries/boolean/not_01/not_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/boolean/not_01/not_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $x := true
 let $y := false
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.1.ddl.aql
new file mode 100644
index 0000000..0e39a04
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.1.ddl.aql
@@ -0,0 +1,8 @@
+/*
+ * Description      :   Check comparison and order-by for day-time-duration
+ * Expected Result  :   Success
+ * Date             :   May 18, 2013
+ */
+drop dataverse test if exists;
+create dataverse test;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.2.update.aql
new file mode 100644
index 0000000..335a24a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.2.update.aql
@@ -0,0 +1,5 @@
+/*
+ * Description      :   Check comparison and order-by for day-time-duration
+ * Expected Result  :   Success
+ * Date             :   May 18, 2013
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.3.query.aql
new file mode 100644
index 0000000..69a8b28
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/day_time_duration_order/day_time_duration_order.3.query.aql
@@ -0,0 +1,15 @@
+/*
+ * Description      :   Check comparison and order-by for day-time-duration
+ * Expected Result  :   Success
+ * Date             :   May 18, 2013
+ */
+use dataverse test;
+
+let $dr1 := day-time-duration("P439D")
+let $dr2 := day-time-duration("-PT328M")
+let $dr3 := day-time-duration("-P48DT12M43.932S")
+let $dr4 := day-time-duration("P12H")
+
+for $dr in [$dr1, $dr2, $dr3, $dr4]
+order by $dr
+return $dr
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.1.ddl.aql
new file mode 100644
index 0000000..ba511f2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for equality comparisons for non-total-ordered types
+ * Expected Res : Success
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.2.update.aql
new file mode 100644
index 0000000..dd5635c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for equality comparisons for non-total-ordered types
+ * Expected Res : Success
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.3.query.aql
new file mode 100644
index 0000000..7e898a2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_equality/issue363_equality.3.query.aql
@@ -0,0 +1,54 @@
+/*
+ * Description  : a test case for equality comparisons for non-total-ordered types
+ * Expected Res : Success
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+
+let $v1 := duration("P3Y6M3DT9H5M2.348S")
+let $v2 := year-month-duration("P3Y6M")
+let $v3 := day-time-duration("P3DT9H5M2.348S")
+let $v4 := point("47.44,80.65")
+let $v5 := line("10.1234,11.1e-1 +10.2E-2,-11.22")
+let $v6 := polygon("-1.0,+10.5e2 -02.15E+50,2.5 -1.0,+3.3e3 -2.50E+05,20.15 +3.5e+2,03.6 -4.60E-3,+4.75 -2,+1.0e2 -2.00E+5,20.10 30.5,03.25 -4.33E-3,+4.75")
+let $v7 := circle("0.1234,-1.00e-10 +10.5E-2")
+let $v8 := rectangle("0.1234,-1.00e-10 5.5487,0.48765")
+let $v9 := interval-from-datetime(datetime("-1987-11-19T02:43:57.938+08:00"), datetime("19991112T124935948-0700"))
+let $dv1 := duration("P3Y6M3DT9H5M2.348S")
+let $dv2 := year-month-duration("P3Y6M")
+let $dv3 := day-time-duration("P3DT9H5M2.348S")
+let $dv4 := point("47.44,80.65")
+let $dv5 := line("10.1234,11.1e-1 +10.2E-2,-11.22")
+let $dv6 := polygon("-1.0,+10.5e2 -02.15E+50,2.5 -1.0,+3.3e3 -2.50E+05,20.15 +3.5e+2,03.6 -4.60E-3,+4.75 -2,+1.0e2 -2.00E+5,20.10 30.5,03.25 -4.33E-3,+4.75")
+let $dv7 := circle("0.1234,-1.00e-10 +10.5E-2")
+let $dv8 := rectangle("0.1234,-1.00e-10 5.5487,0.48765")
+let $dv9 := interval-from-datetime(datetime("-1987-11-19T02:43:57.938+08:00"), datetime("19991112T124935948-0700"))
+let $ndv1 := duration("P4Y6M3DT9H5M2.348S")
+let $ndv2 := year-month-duration("P3Y7M")
+let $ndv3 := day-time-duration("P3DT1H5M2.348S")
+let $ndv4 := point("47.4444,80.65")
+let $ndv5 := line("10.5678,11.1e-1 +10.2E-2,-11.22")
+let $ndv6 := polygon("-1.0,+10.5e2 -02.19E+50,2.5 -1.0,+3.3e3 -2.50E+05,20.15 +3.5e+2,03.6 -4.60E-3,+4.75 -2,+1.0e2 -2.00E+5,20.10 30.5,03.25 -4.33E-3,+4.75")
+let $ndv7 := circle("0.5678,-1.00e-10 +10.5E-2")
+let $ndv8 := rectangle("0.5678,-1.00e-10 5.5487,0.48765")
+let $ndv9 := interval-from-datetime(datetime("-1983-11-19T02:43:57.938+08:00"), datetime("19991112T124935948-0700"))
+
+return { "duration":  $v1 = $dv1, 
+"year-month-duration":  $v2 = $dv2, 
+"day-time-duration":  $v3 = $dv3, 
+"point": $v4 = $dv4, 
+"line": $v5 = $dv5, 
+"polygon": $v6 = $dv6,
+"circle": $v7 = $dv7,
+"rectangle": $v8 = $dv8,
+"interval": $v9 = $dv9,
+"duration2":  $v1 != $ndv1, 
+"year-month-duration2": $v2 != $ndv2, 
+"day-time-duration2":  $v3 != $ndv3, 
+"point2": $v4 != $ndv4, 
+"line2": $v5 != $ndv5, 
+"polygon2": $v6 != $ndv6,
+"circle2": $v7 != $ndv7,
+"rectangle2": $v8 != $ndv8,
+"interval2": $v9 != $ndv9
+}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.1.ddl.aql
new file mode 100644
index 0000000..b05c7fc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of circle
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.2.update.aql
new file mode 100644
index 0000000..b05c7fc
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of circle
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.3.query.aql
new file mode 100644
index 0000000..dc9aeb5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_circle/issue363_inequality_circle.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : a test case for inequality comparisons of circle
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+ 
+let $v1 := circle("0.1234,-1.00e-10 +10.5E-2")
+let $dv1 := circle("0.5678,-1.00e-10 +10.5E-2")
+
+return { "circle0":  $v1 > $dv1, "circle1": $dv1 < $v1, "circle2":  $v1 >= $dv1, "circle3": $dv1 <= $v1 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.1.ddl.aql
new file mode 100644
index 0000000..6533b0d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of duration
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.2.update.aql
new file mode 100644
index 0000000..6533b0d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of duration
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.3.query.aql
new file mode 100644
index 0000000..a365aba
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_duration/issue363_inequality_duration.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : a test case for inequality comparisons of duration
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+ 
+let $v1 := duration("P300Y6M3DT9H5M2.348S")
+let $dv1 := duration("P3Y6M3DT9H5M2.348S")
+
+return { "duration0":  $v1 > $dv1, "duration1": $dv1 < $v1, "duration2":  $v1 >= $dv1, "duration3": $dv1 <= $v1 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.1.ddl.aql
new file mode 100644
index 0000000..ac47f3f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of interval
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.2.update.aql
new file mode 100644
index 0000000..ac47f3f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of interval
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.3.query.aql
new file mode 100644
index 0000000..36ac511
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_interval/issue363_inequality_interval.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : a test case for inequality comparisons of interval
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+ 
+let $v1 := interval-from-datetime(datetime("-1987-11-19T02:43:57.938+08:00"), datetime("19991112T124935948-0700"))
+let $dv1 := interval-from-datetime(datetime("-1983-11-19T02:43:57.938+08:00"), datetime("19991112T124935948-0700"))
+
+return { "interval0":  $v1 > $dv1, "interval1": $dv1 < $v1, "interval2":  $v1 >= $dv1, "interval3": $dv1 <= $v1 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.1.ddl.aql
new file mode 100644
index 0000000..3618d18
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of line
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.2.update.aql
new file mode 100644
index 0000000..3618d18
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of line
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.3.query.aql
new file mode 100644
index 0000000..d887233
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_line/issue363_inequality_line.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : a test case for inequality comparisons of line
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+ 
+let $v1 := line("10.1234,11.1e-1 +10.2E-2,-11.22")
+let $dv1 := line("10.1234,11.1e-1 +10.2E-2,-11.22")
+
+return { "line0":  $v1 > $dv1, "line1": $dv1 < $v1, "line2":  $v1 >= $dv1, "line3": $dv1 <= $v1 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.1.ddl.aql
new file mode 100644
index 0000000..434dd0a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of point
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.2.update.aql
new file mode 100644
index 0000000..434dd0a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of point
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.3.query.aql
new file mode 100644
index 0000000..2199d41
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_point/issue363_inequality_point.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : a test case for inequality comparisons of point
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+ 
+let $v1 := point("47.44,80.65")
+let $dv1 := point("47.4444,80.65")
+
+return { "point0":  $v1 > $dv1, "point1": $dv1 < $v1, "point2":  $v1 >= $dv1, "point3": $dv1 <= $v1 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.ddl.aql
new file mode 100644
index 0000000..12e3ca6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of polygon
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.2.update.aql
new file mode 100644
index 0000000..12e3ca6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of polygon
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.3.query.aql
new file mode 100644
index 0000000..04af82a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_polygon/issue363_inequality_polygon.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : a test case for inequality comparisons of polygon
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+ 
+let $v1 := polygon("-1.0,+10.5e2 -02.15E+50,2.5 -1.0,+3.3e3 -2.50E+05,20.15 +3.5e+2,03.6 -4.60E-3,+4.75 -2,+1.0e2 -2.00E+5,20.10 30.5,03.25 -4.33E-3,+4.75")
+let $dv1 := polygon("-1.0,+10.5e2 -02.19E+50,2.5 -1.0,+3.3e3 -2.50E+05,20.15 +3.5e+2,03.6 -4.60E-3,+4.75 -2,+1.0e2 -2.00E+5,20.10 30.5,03.25 -4.33E-3,+4.75")
+
+return { "polygon0":  $v1 > $dv1, "polygon1": $dv1 < $v1, "polygon2":  $v1 >= $dv1, "polygon3": $dv1 <= $v1 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.ddl.aql
new file mode 100644
index 0000000..c62c7a3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.ddl.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of rectangle
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.2.update.aql
new file mode 100644
index 0000000..c62c7a3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : a test case for inequality comparisons of rectangle
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.3.query.aql
new file mode 100644
index 0000000..ed6770e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : a test case for inequality comparisons of rectangle
+ * Expected Res : Failure
+ * Date         : 9 May 2013
+ * Issue        : 363
+ */
+ 
+let $v1 := rectangle("0.1234,-1.00e-10 5.5487,0.48765")
+let $dv1 := rectangle("0.5678,-1.00e-10 5.5487,0.48765")
+
+return { "rectangle0":  $v1 > $dv1, "rectangle1": $dv1 < $v1, "rectangle2":  $v1 >= $dv1, "rectangle3": $dv1 <= $v1 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.1.ddl.aql
new file mode 100644
index 0000000..356810c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.1.ddl.aql
@@ -0,0 +1,8 @@
+/*
+ * Description      :   Check comparison and order-by for year-month-duration
+ * Expected Result  :   Success
+ * Date             :   May 18, 2013
+ */
+drop dataverse test if exists;
+create dataverse test;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.2.update.aql
new file mode 100644
index 0000000..65105d6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.2.update.aql
@@ -0,0 +1,5 @@
+/*
+ * Description      :   Check comparison and order-by for year-month-duration
+ * Expected Result  :   Success
+ * Date             :   May 18, 2013
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.3.query.aql
new file mode 100644
index 0000000..3fbfafe
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/comparison/year_month_duration_order/year_month_duration_order.3.query.aql
@@ -0,0 +1,15 @@
+/*
+ * Description      :   Check comparison and order-by for year-month-duration
+ * Expected Result  :   Success
+ * Date             :   May 18, 2013
+ */
+use dataverse test;
+
+let $dr1 := year-month-duration("P439Y")
+let $dr2 := year-month-duration("-P328M")
+let $dr3 := year-month-duration("-P48Y12M")
+let $dr4 := year-month-duration("P12M")
+
+for $dr in [$dr1, $dr2, $dr3, $dr4]
+order by $dr
+return $dr
diff --git a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.1.ddl.aql
index 754ea81..84fe314 100644
--- a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.1.ddl.aql
@@ -1,3 +1,9 @@
+/*
+ * Description  : test duration constructors 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ */
+
 drop dataverse test if exists;
 create dataverse test;
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.2.update.aql
index e69de29..0c32771 100644
--- a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.2.update.aql
@@ -0,0 +1,5 @@
+/*
+ * Description  : test duration constructors 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.3.query.aql
index 6db5228..898ab18 100644
--- a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_01/duration_01.3.query.aql
@@ -1,3 +1,9 @@
+/*
+ * Description  : test duration constructors 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ */
+
 use dataverse test;
 
 let $c1 := duration("P30Y10M25DT13H12M50S")
diff --git a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.1.ddl.aql
new file mode 100644
index 0000000..f3978bf
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.1.ddl.aql
@@ -0,0 +1,10 @@
+/*
+ * Description  : test sub type duration (year-month-duration and day-time-duration) constructors 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * issue        : 363
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.2.update.aql
new file mode 100644
index 0000000..73d2817
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : test sub type duration (year-month-duration and day-time-duration) constructors 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * issue        : 363
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.3.query.aql
new file mode 100644
index 0000000..f526bc4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/constructor/duration_02/duration_02.3.query.aql
@@ -0,0 +1,22 @@
+/*
+ * Description  : test sub type duration (year-month-duration and day-time-duration) constructors 
+ * Expected Res : Success
+ * Date         : 7 May 2013
+ * issue        : 363
+ */
+
+use dataverse test;
+
+let $c1 := year-month-duration("P30Y10M")
+let $c2 := day-time-duration("P25DT13H12M50S")
+let $c3 := day-time-duration("PT13H12M50S")
+let $c4 := year-month-duration("P30Y")
+let $c5 := day-time-duration("PT13H")
+let $c6 := year-month-duration("-P30Y10M")
+let $c7 := day-time-duration("-P25DT13H12M50S")
+let $c8 := day-time-duration("-PT13H50S")
+let $c9 := day-time-duration("P120D")
+let $c10 := year-month-duration("-P28M")
+let $c11 := day-time-duration("PT29M90.937S")
+let $c12 := year-month-duration("P300Y15M")
+return {"duration1": $c1, "duration2": $c2, "duration3": $c3, "duration4": $c4, "duration5": $c5, "duration6": $c6, "duration7": $c7, "duration8": $c8, "duration9": $c9, "duration10": $c10, "duration11": $c11, "duration12": $c12}
diff --git a/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.1.ddl.aql
index 2fbbf2f..882fecb 100644
--- a/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.1.ddl.aql
@@ -1,3 +1,10 @@
+/*
+ * Description  : This test case is to verify the fix for issue51
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=51
+ * Expected Res : SUCCESS
+ * Date         : 14th May 2013
+ */
+
 drop dataverse test if exists;
 
 create dataverse test;
diff --git a/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.2.update.aql
index e69de29..45fba70 100644
--- a/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.2.update.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.2.update.aql
@@ -0,0 +1,6 @@
+/*
+ * Description  : This test case is to verify the fix for issue51
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=51
+ * Expected Res : SUCCESS
+ * Date         : 14th May 2013
+ */
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.3.query.aql
index 9996053..9781eb5 100644
--- a/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/custord/join_q_04/join_q_04.3.query.aql
@@ -1,7 +1,19 @@
+/*
+ * Description  : This test case is to verify the fix for issue51
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=51
+ * Expected Res : SUCCESS
+ * Date         : 14th May 2013
+ */
+
 use dataverse test;
 
-for $c in dataset('Customers')
-return {"order_id" :
-for $o in dataset('Orders')
-where $c.cid = $o.cid
-return $o.oid } 
+for $c in dataset Customers
+order by $c.name
+return {
+		"cust_name": $c.name,
+		"order_ids":
+			for $o in dataset Orders
+				where $c.cid = $o.cid 
+			order by $o.oid
+			return $o.oid 
+} 
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.3.query.aql
new file mode 100644
index 0000000..6b09eec
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443-2/query-issue443-2.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue443
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=443
+ * Expected Res : Fail
+ * Date         : 22th May 2013
+ */
+
+
+for $a in [ {"f" : 19, "g": 1} , {"f" : 12, "g": 2} , {"f" : 10, "g": 1} , {"f" : 17, "g": 1}, {"f" : 12, "g": 4} ]
+distinct by $a.f
+return $a
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.3.query.aql
new file mode 100644
index 0000000..602468c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/distinct/query-issue443/query-issue443.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue443
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=443
+ * Expected Res : Fail
+ * Date         : 22th May 2013
+ */
+
+
+for $a in [ {"f" : 19} , {"f" : 12} , {"f" : 10} , {"f" : 17}, {"f" : 12} ]
+distinct by $a.f
+return $a
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.1.ddl.aql
new file mode 100644
index 0000000..d09e16d
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.1.ddl.aql
@@ -0,0 +1,26 @@
+create dataverse SocialNetworkData;
+
+use dataverse SocialNetworkData;
+
+create type EmploymentType as closed {
+organization-name: string,
+start-date: date,
+end-date: date?
+}
+
+create type FacebookUserType as closed {
+id: int32,
+id-copy: int32,
+alias: string,
+name: string,
+user-since: datetime,
+user-since-copy: datetime,
+friend-ids: {{ int32 }},
+employment: [EmploymentType]
+}
+
+create dataset FacebookUsers(FacebookUserType)
+primary key id;
+
+create dataset HandbookUsers(FacebookUserType)
+primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.2.update.aql
new file mode 100644
index 0000000..72f89e2
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.2.update.aql
@@ -0,0 +1,10 @@
+use dataverse SocialNetworkData;
+
+load dataset FacebookUsers using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/fbu-dml-insert-shuffled.adm"),("format"="adm"));
+
+insert into dataset HandbookUsers (
+for $x in dataset FacebookUsers
+where $x.id < 10307032
+return $x
+)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.3.query.aql
new file mode 100644
index 0000000..4a2f1ff
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue382/query-issue382.3.query.aql
@@ -0,0 +1,3 @@
+use dataverse SocialNetworkData;
+
+count(for $h in dataset HandbookUsers return $h);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.1.ddl.aql
new file mode 100644
index 0000000..e7ab9e8
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.1.ddl.aql
@@ -0,0 +1,23 @@
+/*
+ * Description  : This test case is to verify the fix for issue433
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=433
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+
+create dataverse insertIssue;
+use dataverse insertIssue;
+
+create type subElem as closed {
+n: string,
+e: int32?
+}
+
+create type elem as closed {
+id: int32,
+name: string,
+sub: [subElem]
+}
+
+create dataset myDataset(elem)
+primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.2.update.aql
new file mode 100644
index 0000000..d06b4f5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.2.update.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : This test case is to verify the fix for issue433
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=433
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+
+use dataverse insertIssue;
+
+insert into dataset myDataset (
+for $t in [ {"id":1, "name":"u1","sub":[{"n":"se1","e":100}]}, {"id":2, "name":"u2","sub":[{"n":"se2","e":200}]} ]
+return $t
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.3.query.aql
new file mode 100644
index 0000000..b863cdf
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/dml/query-issue433/query-issue433.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue433
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=433
+ * Expected Res : Success
+ * Date         : 3th April 2013
+ */
+
+use dataverse insertIssue;
+
+for $d in dataset myDataset
+order by $d.id
+return $d;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
index 01ef318..230aa40 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_02/feeds_02.3.query.aql
@@ -8,4 +8,5 @@
 use dataverse feeds;
 
 for $x in dataset('TweetFeed')
+order by $x.id
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
index 5146fb5..714dd80 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/feeds_04/feeds_04.3.query.aql
@@ -8,4 +8,5 @@
 use dataverse feeds;
 
 for $x in dataset('TweetFeed')
+order by $x.id
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
index d94576b..48e18e2 100644
--- a/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/feeds/issue_230_feeds/issue_230_feeds.3.query.aql
@@ -7,4 +7,5 @@
 use dataverse feeds;
 
 for $x in dataset('TweetFeed')
+order by $x.id
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_1/dblp-1_1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_1/dblp-1_1.3.query.aql
index bc6490d..c3b22f0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_1/dblp-1_1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_1/dblp-1_1.3.query.aql
@@ -1,5 +1,6 @@
 use dataverse fuzzyjoin;
 
+set import-private-functions 'true';
             //
             // -- - Stage 1 - --
             //
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1.1/dblp-1_2.1.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1.1/dblp-1_2.1.1.3.query.aql
index e6a2e36..5f0ede9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1.1/dblp-1_2.1.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1.1/dblp-1_2.1.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
             //
             // -- - Stage 1 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1/dblp-1_2.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1/dblp-1_2.1.3.query.aql
index e6a2e36..dd0f1ba 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1/dblp-1_2.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2.1/dblp-1_2.1.3.query.aql
@@ -1,5 +1,5 @@
 use dataverse fuzzyjoin;
-
+set import-private-functions 'true';
             //
             // -- - Stage 1 - --
             //
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2/dblp-1_2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2/dblp-1_2.3.query.aql
index aa6a746..6d3d69b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2/dblp-1_2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-1_2/dblp-1_2.3.query.aql
@@ -1,5 +1,5 @@
 use dataverse fuzzyjoin;
-
+set import-private-functions 'true';
             //
             // -- - Stage 1 - --
             //
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.1_5.3.1/dblp-2.1_5.3.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.1_5.3.1/dblp-2.1_5.3.1.3.query.aql
index 70de806..adcb9c7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.1_5.3.1/dblp-2.1_5.3.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.1_5.3.1/dblp-2.1_5.3.1.3.query.aql
@@ -1,5 +1,5 @@
 use dataverse fuzzyjoin;
-
+set import-private-functions 'true';
     //
     // -- - Stage 2 - --
     //
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.2/dblp-2.2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.2/dblp-2.2.3.query.aql
index 03add70..b7a71e9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.2/dblp-2.2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2.2/dblp-2.2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_1/dblp-2_1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_1/dblp-2_1.3.query.aql
index 977a2a9..621f40e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_1/dblp-2_1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_1/dblp-2_1.3.query.aql
@@ -1,5 +1,5 @@
 use dataverse fuzzyjoin;
-
+set import-private-functions 'true';
     //
     // -- - Stage 2 - --
     //
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_2/dblp-2_2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_2/dblp-2_2.3.query.aql
index 275a4cf..ddcded7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_2/dblp-2_2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_2/dblp-2_2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_3/dblp-2_3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_3/dblp-2_3.3.query.aql
index 8f75103..cfa6079 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_3/dblp-2_3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_3/dblp-2_3.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_4/dblp-2_4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_4/dblp-2_4.3.query.aql
index abb7736..a1eca39 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_4/dblp-2_4.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_4/dblp-2_4.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.1/dblp-2_5.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.1/dblp-2_5.1.3.query.aql
index 1950989..1bf3ea3 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.1/dblp-2_5.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.1/dblp-2_5.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.2/dblp-2_5.2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.2/dblp-2_5.2.3.query.aql
index f6b4832..ad5fbad 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.2/dblp-2_5.2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.2/dblp-2_5.2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3.1/dblp-2_5.3.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3.1/dblp-2_5.3.1.3.query.aql
index 6ea3207..666e91b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3.1/dblp-2_5.3.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3.1/dblp-2_5.3.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3/dblp-2_5.3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3/dblp-2_5.3.3.query.aql
index 6ea3207..666e91b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3/dblp-2_5.3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5.3/dblp-2_5.3.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5/dblp-2_5.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5/dblp-2_5.3.query.aql
index 81013bb..b14940f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5/dblp-2_5.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-2_5/dblp-2_5.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.1/dblp-3_1.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.1/dblp-3_1.1.3.query.aql
index 039ed1b..c6260c2 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.1/dblp-3_1.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.1/dblp-3_1.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.2/dblp-3_1.2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.2/dblp-3_1.2.3.query.aql
index 2c2d349..cac9513 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.2/dblp-3_1.2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1.2/dblp-3_1.2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1/dblp-3_1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1/dblp-3_1.3.query.aql
index 0d1976b..9763760 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1/dblp-3_1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-3_1/dblp-3_1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_1/dblp-csx-2_1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_1/dblp-csx-2_1.3.query.aql
index 2d3a1f2..51266cd 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_1/dblp-csx-2_1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_1/dblp-csx-2_1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_2/dblp-csx-2_2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_2/dblp-csx-2_2.3.query.aql
index 8847306..1482237 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_2/dblp-csx-2_2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_2/dblp-csx-2_2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_3/dblp-csx-2_3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_3/dblp-csx-2_3.3.query.aql
index 4438e50..07846e7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_3/dblp-csx-2_3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_3/dblp-csx-2_3.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_4/dblp-csx-2_4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_4/dblp-csx-2_4.3.query.aql
index e2fc8d1..6a6826a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_4/dblp-csx-2_4.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_4/dblp-csx-2_4.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.1/dblp-csx-2_5.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.1/dblp-csx-2_5.1.3.query.aql
index d6e7183..b92449f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.1/dblp-csx-2_5.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.1/dblp-csx-2_5.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.2/dblp-csx-2_5.2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.2/dblp-csx-2_5.2.3.query.aql
index f379cf4..cab57b4 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.2/dblp-csx-2_5.2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.2/dblp-csx-2_5.2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3.1/dblp-csx-2_5.3.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3.1/dblp-csx-2_5.3.1.3.query.aql
index de3c8e8..5ef887e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3.1/dblp-csx-2_5.3.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3.1/dblp-csx-2_5.3.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3/dblp-csx-2_5.3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3/dblp-csx-2_5.3.3.query.aql
index 803db8f..65b86d6 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3/dblp-csx-2_5.3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5.3/dblp-csx-2_5.3.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5/dblp-csx-2_5.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5/dblp-csx-2_5.3.query.aql
index 60ea2ee..c562ba9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5/dblp-csx-2_5.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-2_5/dblp-csx-2_5.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
     //
     // -- - Stage 2 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_1/dblp-csx-3_1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_1/dblp-csx-3_1.3.query.aql
index 8757f02..a8e5821 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_1/dblp-csx-3_1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_1/dblp-csx-3_1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_2/dblp-csx-3_2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_2/dblp-csx-3_2.3.query.aql
index 7d40888..e643523 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_2/dblp-csx-3_2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_2/dblp-csx-3_2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_3/dblp-csx-3_3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_3/dblp-csx-3_3.3.query.aql
index 85ace1c..2dae3af 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_3/dblp-csx-3_3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_3/dblp-csx-3_3.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_4/dblp-csx-3_4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_4/dblp-csx-3_4.3.query.aql
index 3574534..3f5a623 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_4/dblp-csx-3_4.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_4/dblp-csx-3_4.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.1/dblp-csx-3_5.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.1/dblp-csx-3_5.1.3.query.aql
index 6a6dc0c..832ec5c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.1/dblp-csx-3_5.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.1/dblp-csx-3_5.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.2/dblp-csx-3_5.2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.2/dblp-csx-3_5.2.3.query.aql
index 5359b9f..e17ad08 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.2/dblp-csx-3_5.2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.2/dblp-csx-3_5.2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3.1/dblp-csx-3_5.3.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3.1/dblp-csx-3_5.3.1.3.query.aql
index f0dbf23..52456d7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3.1/dblp-csx-3_5.3.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3.1/dblp-csx-3_5.3.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3/dblp-csx-3_5.3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3/dblp-csx-3_5.3.3.query.aql
index 40aba87..ba532b9 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3/dblp-csx-3_5.3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.3/dblp-csx-3_5.3.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4.1/dblp-csx-3_5.4.1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4.1/dblp-csx-3_5.4.1.3.query.aql
index 103af68..f68ef0e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4.1/dblp-csx-3_5.4.1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4.1/dblp-csx-3_5.4.1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4/dblp-csx-3_5.4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4/dblp-csx-3_5.4.3.query.aql
index 2199203..ecc30bc 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4/dblp-csx-3_5.4.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5.4/dblp-csx-3_5.4.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5/dblp-csx-3_5.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5/dblp-csx-3_5.3.query.aql
index 44807b2..9b80d4d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5/dblp-csx-3_5.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/fuzzyjoin/dblp-csx-3_5/dblp-csx-3_5.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse fuzzyjoin;
+set import-private-functions 'true';
 
 //
 // -- - Stage 3 - --
diff --git a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
index efa1f9c..f1127f0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/hints/issue_251_dataset_hint_7/issue_251_dataset_hint_7.3.query.aql
@@ -9,5 +9,6 @@
 use dataverse feeds;
 
 for $x in dataset('TweetFeed')
+order by $x.id
 return $x
 
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql
index 79b65bb..dc18182 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/fuzzy-inverted-index-ngram-jaccard/fuzzy-inverted-index-ngram-jaccard.3.query.aql
@@ -1,6 +1,7 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $o in dataset('DBLP')
 let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
 where $jacc[0]
-return $o
\ No newline at end of file
+return $o
diff --git a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.4.query.aql
index 5d9ad67..dc18182 100644
--- a/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/index-selection/inverted-index-ngram-jaccard/inverted-index-ngram-jaccard.4.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $o in dataset('DBLP')
 let $jacc := similarity-jaccard-check(gram-tokens($o.title, 3, false), gram-tokens("Transactions for Cooperative Environments", 3, false), 0.5f)
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql
index dcd5245..1851f7c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql
@@ -7,6 +7,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.4.query.aql
index 4364545..8bd1d9c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ngram-jaccard/ngram-jaccard.4.query.aql
@@ -6,6 +6,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline/olist-edit-distance-inline.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline/olist-edit-distance-inline.4.query.aql
index d904c09..1a4be9b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline/olist-edit-distance-inline.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance-inline/olist-edit-distance-inline.4.query.aql
@@ -7,6 +7,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('Customers')
 for $b in dataset('Customers2')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance/olist-edit-distance.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance/olist-edit-distance.4.query.aql
index b9221da..545db4a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance/olist-edit-distance.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-edit-distance/olist-edit-distance.4.query.aql
@@ -6,6 +6,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('Customers')
 for $b in dataset('Customers2')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard-inline/olist-jaccard-inline.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard-inline/olist-jaccard-inline.4.query.aql
index 8dda04e..159b30e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard-inline/olist-jaccard-inline.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard-inline/olist-jaccard-inline.4.query.aql
@@ -7,6 +7,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('Customers')
 for $b in dataset('Customers2')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard/olist-jaccard.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard/olist-jaccard.4.query.aql
index 0af3efe..dd4f610 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard/olist-jaccard.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/olist-jaccard/olist-jaccard.4.query.aql
@@ -6,6 +6,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('Customers')
 for $b in dataset('Customers2')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline/ulist-jaccard-inline.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline/ulist-jaccard-inline.4.query.aql
index b7b056e..260a871 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline/ulist-jaccard-inline.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard-inline/ulist-jaccard-inline.4.query.aql
@@ -7,6 +7,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('Customers')
 for $b in dataset('Customers2')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard/ulist-jaccard.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard/ulist-jaccard.4.query.aql
index 57fd37c..2fe6e28 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard/ulist-jaccard.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/ulist-jaccard/ulist-jaccard.4.query.aql
@@ -6,6 +6,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('Customers')
 for $b in dataset('Customers2')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.4.query.aql
index 6faa6a7..8debbfd 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard-inline/word-jaccard-inline.4.query.aql
@@ -7,6 +7,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.4.query.aql
index 3db8b5f..a8684e0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join-noeqjoin/word-jaccard/word-jaccard.4.query.aql
@@ -6,6 +6,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql
index e7fc13e..4068296 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard-inline/ngram-jaccard-inline.4.query.aql
@@ -6,6 +6,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
diff --git a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard/ngram-jaccard.4.query.aql b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard/ngram-jaccard.4.query.aql
index b237130..b64e7e1 100644
--- a/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard/ngram-jaccard.4.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/inverted-index-join/ngram-jaccard/ngram-jaccard.4.query.aql
@@ -5,6 +5,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 for $a in dataset('DBLP')
 for $b in dataset('CSX')
diff --git a/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.3.query.aql
new file mode 100644
index 0000000..aa6bf88
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/list/query-issue428/query-issue428.3.query.aql
@@ -0,0 +1,10 @@
+/*
+ * Description  : This test case is to verify the fix for issue400
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=400
+ * Expected Res : Success
+ * Date         : 8th May 2013
+ */
+
+for $a in [[1,2],[3,4,5]]
+for $b in [[6,7],[8,9,10]]
+return some $a1 in $a,$b1 in $b satisfies $a1 < $b1
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
index e69de29..e6a3879 100644
--- a/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
@@ -0,0 +1,2 @@
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/is-null_01/is-null_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/misc/is-null_01/is-null_01.3.query.aql
index 7725698..61220f7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/misc/is-null_01/is-null_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/is-null_01/is-null_01.3.query.aql
@@ -1,3 +1,4 @@
 use dataverse test;
+set import-private-functions 'true';
 
 [is-null(null), is-null(10)]
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/range_01/range_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/misc/range_01/range_01.3.query.aql
index 439c244..a2bcb99 100644
--- a/asterix-app/src/test/resources/runtimets/queries/misc/range_01/range_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/range_01/range_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $x in range(20,30)
 return $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql
new file mode 100644
index 0000000..59e814a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql
@@ -0,0 +1,22 @@
+/*
+ * Description  : This test case is to verify the fix for issue196
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date         : 5th May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type testtype1 as open {
+id : int32
+}
+
+create type testtype2 as open {
+id : int32
+}
+
+create dataset t1(testtype1) primary key id;
+create dataset t2(testtype2) primary key id;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql
new file mode 100644
index 0000000..18756f9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : This test case is to verify the fix for issue196
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date         : 5th May 2013
+ */
+
+use dataverse test;
+
+insert into dataset t1({"id":24});
+insert into dataset t1({"id":23});
+insert into dataset t1({"id":21});
+insert into dataset t1({"id":44});
+insert into dataset t1({"id":64});
+
+insert into dataset t2({"id":24});
+insert into dataset t2({"id":23});
+insert into dataset t2({"id":21});
+insert into dataset t2({"id":44});
+insert into dataset t2({"id":64});
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql
new file mode 100644
index 0000000..43d1980
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue196
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date         : 5th May 2013
+ */
+
+use dataverse test;
+
+let $a := (for $l in dataset('t1') order by $l.id return $l)
+let $b := (for $m in dataset('t2') order by $m.id return $m)
+return {"a":$a,"b":$b}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.1.ddl.aql
new file mode 100644
index 0000000..5b219b5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.1.ddl.aql
@@ -0,0 +1,18 @@
+/*
+ * Description  : This test case is to verify the fix for issue258
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=258
+ * Expected Res : Success
+ * Date         : 21 May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test if not exists;
+use dataverse test;
+
+create type t1 as closed {
+id:int32
+};
+
+
+create dataset ds1(t1) primary key id;
+create dataset ds2(t1) primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.2.update.aql
new file mode 100644
index 0000000..fb334ad
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.2.update.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : This test case is to verify the fix for issue258
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=258
+ * Expected Res : Success
+ * Date         : 21 May 2013
+ */
+
+use dataverse test;
+
+insert into dataset ds1(
+let $L:= 
+  for $x in dataset('ds2')
+  where $x.id = 10
+  return $x
+return
+  if (count($L) <= 0) then
+    {"id": 10}
+  else
+    {"id": 5}
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.3.query.aql
new file mode 100644
index 0000000..fff488a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue258/query-issue258.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue258
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=258
+ * Expected Res : Success
+ * Date         : 21 May 2013
+ */
+
+use dataverse test;
+
+for $d in dataset ds1
+return $d;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql
new file mode 100644
index 0000000..a49658e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql
@@ -0,0 +1,47 @@
+/*
+ * Description  : This test case is to verify the fix for issue343.  It is a more general case.
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32,
+  street: string,
+  city: string
+}
+
+create type AllType as open {
+  id: int32,
+  name: string,
+  age: float,
+  salary: double,
+  married: boolean,
+  interests: {{string}},
+  children: [string],
+  address: AddressType,
+  dob: date,
+  time: time,
+  datetime: datetime,
+  duration: duration,
+  location2d: point,
+  location3d: point3d,
+  line: line,
+  polygon: polygon,
+  circle: circle
+}
+
+create type MyListType as open{
+	id: int32,
+	mylist: [string]
+}
+
+create dataset All(AllType)
+  primary key id;
+  
+create dataset MyList(MyListType)
+  primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql
new file mode 100644
index 0000000..39b85f3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql
@@ -0,0 +1,21 @@
+/*
+ * Description  : This test case is to verify the fix for issue343.  It is a more general case.
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+insert into dataset MyList (
+ {
+   "id" : 1,
+   "mylist": ["blah"]
+ }
+);
+
+insert into dataset All (
+for $m in dataset MyList
+let $record:= { "id": 13, "name": string("Nancy"), "age": 32.5f, "salary": 12.000 ,"married": boolean("true"), "interests": {{"reading", "writing"}}, "children": ["Brad", "Scott"],  "address": {  "number": 8389,  "street": "Hill St.",  "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30Z"), "datetime": datetime("-1951-12-27T12:20:30"),  "duration": duration("P10Y11M12DT10H50M30S"),  "location2d": point("41.00,44.00"),  "location3d": point3d("44.00,13.00,41.00"), "line" : line("10.1,11.1 10.2,11.2"), "polygon" : polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle" : circle("10.1,11.1 10.2"), "mylist" : $m.mylist }
+return $record
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql
new file mode 100644
index 0000000..e17a6e0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue343.  It is a more general case.
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+for $x in dataset All
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql
new file mode 100644
index 0000000..bd2ab1a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql
@@ -0,0 +1,41 @@
+/*
+ * Description  : This test case is to verify the fix for issue343
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32,
+  street: string,
+  city: string
+}
+
+create type AllType as open {
+  id: int32,
+  name: string,
+  age: float,
+  salary: double,
+  married: boolean,
+  interests: {{string}},
+  children: [string],
+  address: AddressType,
+  dob: date,
+  time: time,
+  datetime: datetime,
+  duration: duration,
+  location2d: point,
+  location3d: point3d,
+  line: line,
+  polygon: polygon,
+  circle: circle
+}
+
+create dataset All(AllType)
+  primary key id;
+  
+ 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql
new file mode 100644
index 0000000..eaafddd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql
@@ -0,0 +1,14 @@
+/*
+ * Description  : This test case is to verify the fix for issue343
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+insert into dataset All (
+let $addedList := ["blah"]
+let $record:= { "id": 13, "name": string("Nancy"), "age": 32.5f, "salary": 12.000 ,"married": boolean("true"), "interests": {{"reading", "writing"}}, "children": ["Brad", "Scott"],  "address": {  "number": 8389,  "street": "Hill St.",  "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30Z"), "datetime": datetime("-1951-12-27T12:20:30"),  "duration": duration("P10Y11M12DT10H50M30S"),  "location2d": point("41.00,44.00"),  "location3d": point3d("44.00,13.00,41.00"), "line" : line("10.1,11.1 10.2,11.2"), "polygon" : polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle" : circle("10.1,11.1 10.2"), "mylist" : $addedList }
+return $record
+)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql
new file mode 100644
index 0000000..9dfde51
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue343
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+for $x in dataset All
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql
new file mode 100644
index 0000000..85f2160
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TweetMessageType as open {
+tweetid: string
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid; 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql
new file mode 100644
index 0000000..372d7ca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql
@@ -0,0 +1,50 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+load dataset TweetMessages
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/twitter/tw_messages.adm"),("format"="adm"));
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"13",
+    "user":
+        {"screen-name":"NathanGiesen@211",
+         "lang":"en",
+         "friends_count":39345,
+         "statuses_count":479,
+         "name":"Nathan Giesen",
+         "followers_count":49420,
+         "hobbies":["basket weaving","mud wrestling"]
+        },
+    "sender-location":point("47.44,80.65"),
+    "send-time":datetime("2008-04-26T10:10:35"),
+    "referred-topics":{{"tweeting"}},
+    "message-text":"tweety tweet, my fellow tweeters!"
+   }
+);
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"15",
+    "user":
+        {"screen-name":"Jason17",
+         "lang":"en",
+         "friends_count":393,
+         "statuses_count":47,
+         "name":"Nathan Giesen",
+         "followers_count":420,
+         "hobbies":["swimming"]
+        },
+    "sender-location":point("49.44,80.65"),
+    "send-time":datetime("2009-04-26T10:10:35"),
+    "referred-topics":{{"nothing"}},
+    "message-text":"Nothing to say!"
+   }
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql
new file mode 100644
index 0000000..298940a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+for $tm  in dataset TweetMessages
+where (every $h in $tm.user.hobbies satisfies $h = "basket weaving")
+order by $tm.tweetid
+return $tm;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql
new file mode 100644
index 0000000..85f2160
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TweetMessageType as open {
+tweetid: string
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid; 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql
new file mode 100644
index 0000000..372d7ca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql
@@ -0,0 +1,50 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+load dataset TweetMessages
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/twitter/tw_messages.adm"),("format"="adm"));
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"13",
+    "user":
+        {"screen-name":"NathanGiesen@211",
+         "lang":"en",
+         "friends_count":39345,
+         "statuses_count":479,
+         "name":"Nathan Giesen",
+         "followers_count":49420,
+         "hobbies":["basket weaving","mud wrestling"]
+        },
+    "sender-location":point("47.44,80.65"),
+    "send-time":datetime("2008-04-26T10:10:35"),
+    "referred-topics":{{"tweeting"}},
+    "message-text":"tweety tweet, my fellow tweeters!"
+   }
+);
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"15",
+    "user":
+        {"screen-name":"Jason17",
+         "lang":"en",
+         "friends_count":393,
+         "statuses_count":47,
+         "name":"Nathan Giesen",
+         "followers_count":420,
+         "hobbies":["swimming"]
+        },
+    "sender-location":point("49.44,80.65"),
+    "send-time":datetime("2009-04-26T10:10:35"),
+    "referred-topics":{{"nothing"}},
+    "message-text":"Nothing to say!"
+   }
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql
new file mode 100644
index 0000000..79a620b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+for $tm  in dataset TweetMessages
+where (some $h in $tm.user.hobbies satisfies $h = "basket weaving")
+return $tm;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.1.ddl.aql
new file mode 100644
index 0000000..4722e4f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.1.ddl.aql
@@ -0,0 +1,39 @@
+/*
+ * Description  : This test case is to verify the fix for issue377
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=377
+ * Expected Res : Success
+ * Date         : 11th May 2013
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TwitterUserType as open {
+        screen-name: string
+}
+
+create type TweetMessageType as open {
+        tweetid: string
+}
+
+create type FacebookUserType as open {
+        id: int32
+}
+
+create type FacebookMessageType as open {
+        message-id: int32
+}
+
+create dataset FacebookUsers(FacebookUserType)
+primary key id;
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key message-id;
+
+create dataset TwitterUsers(TwitterUserType)
+primary key screen-name;
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid
+hints(cardinality=100); 
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.2.update.aql
new file mode 100644
index 0000000..1d6c5ba
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.2.update.aql
@@ -0,0 +1,15 @@
+/*
+ * Description  : This test case is to verify the fix for issue377
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=377
+ * Expected Res : Success
+ * Date         : 11th May 2013
+ */
+
+use dataverse TinySocial;
+
+load dataset FacebookUsers using localfs
+(("path"="nc1://data/fbu-dml-insert-shuffled.adm"),("format"="adm"));
+
+load dataset TweetMessages using localfs
+(("path"="nc1://data/twitter/tw_messages.adm"),("format"="adm"));
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql
new file mode 100644
index 0000000..81d6cf6
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql
@@ -0,0 +1,24 @@
+/*
+ * Description  : This test case is to verify the fix for issue377
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=377
+ * Expected Res : Success
+ * Date         : 11th May 2013
+ */
+
+use dataverse TinySocial;
+
+set simfunction "edit-distance";
+set simthreshold "3";
+
+for $fbu in dataset FacebookUsers
+return {
+    "id": $fbu.id,
+    "name": $fbu.name,
+    "similar-users": for $t in dataset TweetMessages
+                        let $tu := $t.user
+                        where $tu.name ~= $fbu.name
+                        return {
+                        "twitter-screenname": $tu.screen-name,
+                        "twitter-name": $tu.name
+                        }
+};
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.1.ddl.aql
new file mode 100644
index 0000000..0da0aa5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : This test case is to verify the fix for issue410
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=410
+ * Expected Res : Fail
+ * Date         : 13th May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:int32,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.2.update.aql
new file mode 100644
index 0000000..796c5a4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.2.update.aql
@@ -0,0 +1,10 @@
+/*
+ * Description  : This test case is to verify the fix for issue410
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=410
+ * Expected Res : Fail
+ * Date         : 11th May 2013
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":float("59138237473282.3293"), "name": double("0.01")});	
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.3.query.aql
new file mode 100644
index 0000000..d76caba
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue410/query-issue410.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue410
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=410
+ * Expected Res : Fail
+ * Date         : 11th May 2013
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.1.ddl.aql
new file mode 100644
index 0000000..05eb126
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.1.ddl.aql
@@ -0,0 +1,8 @@
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TestType as open { id : int32 ,fname:string, lname:string}
+
+create dataset t2(TestType) primary key fname,lname;
+create dataset t1(TestType) primary key fname,lname;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.2.update.aql
new file mode 100644
index 0000000..71904f1
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.2.update.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+insert into dataset t1({"id":123,"fname":"John","lname":"Doe"});
+insert into dataset t1({"id":122,"fname":"Bruce","lname":"Li"});
+insert into dataset t2({"id":23,"fname":"John","lname":"Doe"});
+insert into dataset t2({"id":24,"fname":"Ravi","lname":"Khanna"});
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.3.query.aql
new file mode 100644
index 0000000..a07b185
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue423/query-issue423.3.query.aql
@@ -0,0 +1,6 @@
+use dataverse test;
+
+for $l in dataset t1
+for $m in dataset t2
+	where $l.name=$m.name
+return {"l":$l,"m":$m};
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.1.ddl.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.3.query.aql
new file mode 100644
index 0000000..4da91be
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue442/query-issue442.3.query.aql
@@ -0,0 +1,10 @@
+/*
+ * Description  : This test case is to verify the fix for issue442
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=442
+ * Expected Res : Fail
+ * Date         : 22th May 2013
+ */
+
+for $a in [ {"f" : 100} ,  {"f" : 0},  {"f" : -1}, {"f" : null}, {"f" : 999999}, {"f" : 1} , {"f" : "zzzzz"}]
+order by $a.f desc
+return $a
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.1.ddl.aql
new file mode 100644
index 0000000..6fcce66
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.1.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : This test case is to verify the fix for issue453
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=453
+ * Expected Res : SUCCESS
+ * Date         : 18th May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TypeOpen as open {
+    id : int32,
+    int_m : int32,
+    int_o : int32?,
+    string_m : string,
+    string_o : string?
+};
+
+create dataset DataOpen(TypeOpen) primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.2.update.aql
new file mode 100644
index 0000000..2682e84
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.2.update.aql
@@ -0,0 +1,27 @@
+/*
+ * Description  : This test case is to verify the fix for issue453
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=453
+ * Expected Res : SUCCESS
+ * Date         : 18th May 2013
+ */
+
+use dataverse test;
+
+insert into dataset DataOpen(
+      for $arr at $pos in (
+          for $i1 in [1, 2]
+          for $i2 in [1, null]
+          for $s1 in ["a", "b"]
+          for $s2 in ["a", null]
+          return
+            [ $i1, $i2, $s1, $s2]
+        )
+      return
+          {
+            "id" : $pos,
+            "int_m" : $arr[0], 
+            "int_o" : $arr[1],
+            "string_m" : $arr[2],
+            "string_o" : $arr[3]
+          }
+      )
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.3.query.aql
new file mode 100644
index 0000000..1381365
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453-2/query-issue453-2.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue453
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=453
+ * Expected Res : SUCCESS
+ * Date         : 18th May 2013
+ */
+
+use dataverse test;
+
+for $d in dataset DataOpen
+order by $d.id
+return $d
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.1.ddl.aql
new file mode 100644
index 0000000..6fcce66
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.1.ddl.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : This test case is to verify the fix for issue453
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=453
+ * Expected Res : SUCCESS
+ * Date         : 18th May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TypeOpen as open {
+    id : int32,
+    int_m : int32,
+    int_o : int32?,
+    string_m : string,
+    string_o : string?
+};
+
+create dataset DataOpen(TypeOpen) primary key id;
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.2.update.aql
new file mode 100644
index 0000000..3346f52
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.2.update.aql
@@ -0,0 +1,16 @@
+/*
+ * Description  : This test case is to verify the fix for issue453
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=453
+ * Expected Res : SUCCESS
+ * Date         : 18th May 2013
+ */
+
+use dataverse test;
+
+insert into dataset DataOpen(
+      for $o in {{
+          { "id": 0, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" },
+          { "id": 1, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null }
+        }}
+      return $o
+)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.3.query.aql
new file mode 100644
index 0000000..1381365
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue453/query-issue453.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue453
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=453
+ * Expected Res : SUCCESS
+ * Date         : 18th May 2013
+ */
+
+use dataverse test;
+
+for $d in dataset DataOpen
+order by $d.id
+return $d
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_04/everysat_04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_04/everysat_04.3.query.aql
index 7742254..e5e425f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_04/everysat_04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/quantifiers/everysat_04/everysat_04.3.query.aql
@@ -4,6 +4,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 let $x := [
 every $x in [false,false] satisfies $x,
diff --git a/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_06/somesat_06.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_06/somesat_06.3.query.aql
index e406cda..4d75664 100644
--- a/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_06/somesat_06.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/quantifiers/somesat_06/somesat_06.3.query.aql
@@ -4,6 +4,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 let $x := [
 some $x in [false,false] satisfies $x,
diff --git a/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_01/closed-record-constructor_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_01/closed-record-constructor_01.3.query.aql
index 4ef8508..b4f18bf 100644
--- a/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_01/closed-record-constructor_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_01/closed-record-constructor_01.3.query.aql
@@ -1,3 +1,4 @@
 use dataverse test;
+set import-private-functions 'true';
 
 closed-record-constructor("foo1", 10, "bar1", 20, "foo2", 30, "bar2", 40)
diff --git a/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_02/closed-record-constructor_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_02/closed-record-constructor_02.3.query.aql
index c6e3756..040a7fb 100644
--- a/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_02/closed-record-constructor_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_02/closed-record-constructor_02.3.query.aql
@@ -1,3 +1,4 @@
 use dataverse test;
+set import-private-functions 'true';
 
 closed-record-constructor("foo1", 10, "bar1", closed-record-constructor("bar1.1", 10, "bar1.2", 20, "bar1.3", 30, "bar1.4", closed-record-constructor("bar1.4.1", 10, "bar1.4.2", 20, "bar1.4.3", 30, "bar1.4.4", 40), "foo2", 30, "bar2", 40), "foo2", 30, "bar2", 40)
diff --git a/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_03/closed-record-constructor_03.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_03/closed-record-constructor_03.3.query.aql
index 7285a75..416d22e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_03/closed-record-constructor_03.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/records/closed-record-constructor_03/closed-record-constructor_03.3.query.aql
@@ -1,3 +1,4 @@
 use dataverse test;
+set import-private-functions 'true';
 
 {"foo1": 10, "bar1": {"bar1.1": 10, "bar1.2": 20, "bar1.3": 30, "bar1.4": {"bar1.4.1": 10, "bar1.4.2": 20 } }, "foo2": 30, "bar2": 40}
diff --git a/asterix-app/src/test/resources/runtimets/queries/records/field-access-by-index_01/field-access-by-index_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/records/field-access-by-index_01/field-access-by-index_01.3.query.aql
index d25edc6..b5ff5ed 100644
--- a/asterix-app/src/test/resources/runtimets/queries/records/field-access-by-index_01/field-access-by-index_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/records/field-access-by-index_01/field-access-by-index_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $x := { "foo1": 10, "bar1": 20, "foo2": 30, "bar2": 40 }
 return field-access-by-index($x,2)
diff --git a/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_01/open-record-constructor_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_01/open-record-constructor_01.3.query.aql
index d6c8701..43a8e19 100644
--- a/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_01/open-record-constructor_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_01/open-record-constructor_01.3.query.aql
@@ -1,3 +1,4 @@
 use dataverse test;
+set import-private-functions 'true';
 
 open-record-constructor("foo1", 10, "bar1", 20, "foo2", 30, "bar2", 40)
diff --git a/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_02/open-record-constructor_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_02/open-record-constructor_02.3.query.aql
index 972ecb2..9b1ea45 100644
--- a/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_02/open-record-constructor_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/records/open-record-constructor_02/open-record-constructor_02.3.query.aql
@@ -1,3 +1,4 @@
 use dataverse test;
+set import-private-functions 'true';
 
 open-record-constructor("foo1", 10, "bar1", closed-record-constructor("bar1.1", 10, "bar1.2", 20, "bar1.3", 30, "bar1.4", closed-record-constructor("bar1.4.1", 10, "bar1.4.2", 20, "bar1.4.3", 30, "bar1.4.4", 40), "foo2", 30, "bar2", 40), "foo2", 30, "bar2", 40)
diff --git a/asterix-app/src/test/resources/runtimets/queries/semistructured/has-param1/has-param1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/semistructured/has-param1/has-param1.3.query.aql
index 4247586..53e4ace 100644
--- a/asterix-app/src/test/resources/runtimets/queries/semistructured/has-param1/has-param1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/semistructured/has-param1/has-param1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $o in dataset('Orders')
 where not(is-null($o.param1))
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-list-is-filterable/edit-distance-list-is-filterable.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-list-is-filterable/edit-distance-list-is-filterable.3.query.aql
index 97fbd61..d2dd7e8 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-list-is-filterable/edit-distance-list-is-filterable.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-list-is-filterable/edit-distance-list-is-filterable.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := []
 let $b := [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-string-is-filterable/edit-distance-string-is-filterable.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-string-is-filterable/edit-distance-string-is-filterable.3.query.aql
index 9d55ce7..e78037f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-string-is-filterable/edit-distance-string-is-filterable.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/edit-distance-string-is-filterable/edit-distance-string-is-filterable.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := ""
 let $b := "abcdefghij"
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard/prefix-len-jaccard.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard/prefix-len-jaccard.3.query.aql
index 1f61b15..693f83d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard/prefix-len-jaccard.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/prefix-len-jaccard/prefix-len-jaccard.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in [1]
 return [
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check/similarity-jaccard-prefix-check.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check/similarity-jaccard-prefix-check.3.query.aql
index 46c0389..6863929 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check/similarity-jaccard-prefix-check.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix-check/similarity-jaccard-prefix-check.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in [1]
 return [
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix/similarity-jaccard-prefix.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix/similarity-jaccard-prefix.3.query.aql
index 10db026..18de7bf 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix/similarity-jaccard-prefix.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-prefix/similarity-jaccard-prefix.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in [1]
 return [
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_ints/similarity-jaccard-sorted-check_ints.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_ints/similarity-jaccard-sorted-check_ints.3.query.aql
index 31c8b1b..2e29289 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_ints/similarity-jaccard-sorted-check_ints.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_ints/similarity-jaccard-sorted-check_ints.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := [ ]
 let $b := [1, 2, 3, 4, 5]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_query/similarity-jaccard-sorted-check_query.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_query/similarity-jaccard-sorted-check_query.3.query.aql
index a8089ec..69a9a31 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_query/similarity-jaccard-sorted-check_query.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_query/similarity-jaccard-sorted-check_query.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $paper in dataset('DBLP')
 let $paper_tokens := word-tokens($paper.title)
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_strings/similarity-jaccard-sorted-check_strings.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_strings/similarity-jaccard-sorted-check_strings.3.query.aql
index 326b626..9f2bf71 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_strings/similarity-jaccard-sorted-check_strings.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted-check_strings/similarity-jaccard-sorted-check_strings.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := [ ]
 let $b := ["abc", "bcd", "cde", "def", "efg"]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_ints/similarity-jaccard-sorted_ints.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_ints/similarity-jaccard-sorted_ints.3.query.aql
index 0915bd6..61d4372 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_ints/similarity-jaccard-sorted_ints.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_ints/similarity-jaccard-sorted_ints.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := [ ]
 let $b := [1, 2, 3, 4, 5]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_query/similarity-jaccard-sorted_query.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_query/similarity-jaccard-sorted_query.3.query.aql
index 6d6e2c8..9f1aa77 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_query/similarity-jaccard-sorted_query.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_query/similarity-jaccard-sorted_query.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $paper in dataset('DBLP')
 let $paper_tokens := word-tokens($paper.title)
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_strings/similarity-jaccard-sorted_strings.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_strings/similarity-jaccard-sorted_strings.3.query.aql
index 2271db0..907538f 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_strings/similarity-jaccard-sorted_strings.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard-sorted_strings/similarity-jaccard-sorted_strings.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := [ ]
 let $b := ["abc", "bcd", "cde", "def", "efg"]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_ints/similarity-jaccard_ints.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_ints/similarity-jaccard_ints.3.query.aql
index a76a82a..8bfb1e0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_ints/similarity-jaccard_ints.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_ints/similarity-jaccard_ints.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := [ ]
 let $b := [1, 2, 3, 4, 5]
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_query/similarity-jaccard_query.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_query/similarity-jaccard_query.3.query.aql
index 86cc50f..a731d94 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_query/similarity-jaccard_query.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_query/similarity-jaccard_query.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $paper in dataset('DBLP')
 let $paper_tokens := word-tokens($paper.title)
diff --git a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_strings/similarity-jaccard_strings.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_strings/similarity-jaccard_strings.3.query.aql
index b7c627e..3db4369 100644
--- a/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_strings/similarity-jaccard_strings.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/similarity/similarity-jaccard_strings/similarity-jaccard_strings.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $a := [ ]
 let $b := ["abc", "bcd", "cde", "def", "efg"]
diff --git a/asterix-app/src/test/resources/runtimets/queries/spatial/create-rtree-index/create-rtree-index.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/spatial/create-rtree-index/create-rtree-index.3.query.aql
index 53b1c30..515acc0 100644
--- a/asterix-app/src/test/resources/runtimets/queries/spatial/create-rtree-index/create-rtree-index.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/spatial/create-rtree-index/create-rtree-index.3.query.aql
@@ -6,4 +6,5 @@
 use dataverse test;
 
 for $a in dataset('MyData')
+order by $a.id
 return $a.id
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal1/string-equal1.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal1/string-equal1.3.query.aql
index 954ac57..3e3e57a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/string-equal1/string-equal1.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal1/string-equal1.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $c1 := string-equal("test","tess")
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal2/string-equal2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal2/string-equal2.3.query.aql
index 66af45d..77b837a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/string-equal2/string-equal2.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal2/string-equal2.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $c1 := string-equal("test","test")
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal3/string-equal3.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal3/string-equal3.3.query.aql
index fe3b341..87c0123 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/string-equal3/string-equal3.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal3/string-equal3.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $c1 := string-equal("test11","test")
 return {"result1": $c1}
diff --git a/asterix-app/src/test/resources/runtimets/queries/string/string-equal4/string-equal4.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/string/string-equal4/string-equal4.3.query.aql
index 2b5166c..1128cc5 100644
--- a/asterix-app/src/test/resources/runtimets/queries/string/string-equal4/string-equal4.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/string/string-equal4/string-equal4.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $c1 := string-equal("","")
 let $c3 := string-equal(null,"")
diff --git a/asterix-app/src/test/resources/runtimets/queries/subset-collection/01/01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/subset-collection/01/01.3.query.aql
index 05612a3..4341d92 100644
--- a/asterix-app/src/test/resources/runtimets/queries/subset-collection/01/01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/subset-collection/01/01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in subset-collection([1], 0, 1)
 return $l
diff --git a/asterix-app/src/test/resources/runtimets/queries/subset-collection/02/02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/subset-collection/02/02.3.query.aql
index 14b3e4e..666eb76 100644
--- a/asterix-app/src/test/resources/runtimets/queries/subset-collection/02/02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/subset-collection/02/02.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in subset-collection([1, 2, 3, 4], 1, 2)
 return $l
diff --git a/asterix-app/src/test/resources/runtimets/queries/subset-collection/03/03.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/subset-collection/03/03.3.query.aql
index 248bff3..aaaf487 100644
--- a/asterix-app/src/test/resources/runtimets/queries/subset-collection/03/03.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/subset-collection/03/03.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in subset-collection([1, 2, 3, 4], 0, 0)
 return $l
diff --git a/asterix-app/src/test/resources/runtimets/queries/subset-collection/04/04.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/subset-collection/04/04.3.query.aql
index 0fde28a..1b41033 100644
--- a/asterix-app/src/test/resources/runtimets/queries/subset-collection/04/04.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/subset-collection/04/04.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in [1]
 return subset-collection([1, 2, 3, 4], 1, 2)
diff --git a/asterix-app/src/test/resources/runtimets/queries/subset-collection/05/05.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/subset-collection/05/05.3.query.aql
index 2403754..7d369ef 100644
--- a/asterix-app/src/test/resources/runtimets/queries/subset-collection/05/05.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/subset-collection/05/05.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in subset-collection(['a', 'b', 'c', 'd'], 1, 2)
 return $l
diff --git a/asterix-app/src/test/resources/runtimets/queries/subset-collection/06/06.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/subset-collection/06/06.3.query.aql
index c8e78af..e4dd90d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/subset-collection/06/06.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/subset-collection/06/06.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in subset-collection([1, 2, 3, 4], 2, 2)
 return $l
diff --git a/asterix-app/src/test/resources/runtimets/queries/subset-collection/07/07.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/subset-collection/07/07.3.query.aql
index 5166adf..d651d3c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/subset-collection/07/07.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/subset-collection/07/07.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 for $l in subset-collection([1, 2, 3, 4], 2, 10)
 return $l
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql
index 291d296..016f3b3 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/calendar_duration/calendar_duration.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $t1 := datetime("1987-11-19T23:49:23.938")
 let $t2 := date("-1328-10-23")
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql
index b748429..143a02b 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/date_functions/date_functions.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $d1 := date-from-unix-time-in-days(15600)
 let $null1 := date-from-unix-time-in-days(null)
@@ -20,4 +21,4 @@
 let $null5 := subtract-date(null, $d2)
 let $null6 := subtract-date($d5, null)
 
-return { "date1": $d1, "date2": $d2, "date3": $d3, "date4": $d4, "date5": $d5, "duration1": $dr3, "duration2": $dr4, "c1": $c1, "c2": $c2, "null1": $null1, "null2": $null2, "null3": $null3, "null4": $null4, "null5": $null5, "null6": $null6 }
\ No newline at end of file
+return { "date1": $d1, "date2": $d2, "date3": $d3, "date4": $d4, "date5": $d5, "duration1": $dr3, "duration2": $dr4, "c1": $c1, "c2": $c2, "null1": $null1, "null2": $null2, "null3": $null3, "null4": $null4, "null5": $null5, "null6": $null6 }
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql
index ea6c2a8..3d24549 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/datetime_functions/datetime_functions.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $dt1 := datetime-from-unix-time-in-ms(956007429)
 let $null1 := datetime-from-unix-time-in-ms(null)
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/duration_comps/duration_comps.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/duration_comps/duration_comps.3.query.aql
index 77fcc4f..456c3cc 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/duration_comps/duration_comps.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/duration_comps/duration_comps.3.query.aql
@@ -1,11 +1,12 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $dr1 := duration("-P3D")
 let $dr2 := duration("P1D")
 let $dr3 := duration("P1Y")
 let $dr4 := duration("P13M")
 let $dr5 := duration("PT24H")
-let $dr6 := duration-from-months(months-of-year-month-duration($dr3))
-let $dr7 := duration-from-ms(ms-of-day-time-duration($dr1))
+let $dr6 := duration-from-months(months-from-year-month-duration(get-year-month-duration($dr3)))
+let $dr7 := duration-from-ms(ms-from-day-time-duration(get-day-time-duration($dr1)))
 
-return { "yearMonthGreaterComp" : year-month-duration-greater-than($dr4, $dr3), "dayTimeGreaterComp" : day-time-duration-greater-than($dr2, $dr1), "yearMonthLessComp" : year-month-duration-less-than($dr4, $dr3), "dayTimeLessComp" : day-time-duration-less-than($dr2, $dr1), "equal1": duration-equal($dr2, $dr5), "equal2": duration-equal($dr1, $dr5), "equal3": duration-equal($dr6, $dr3), "equal4": duration-equal($dr7, $dr1) }
\ No newline at end of file
+return { "yearMonthGreaterComp" : year-month-duration-greater-than($dr4, $dr3), "dayTimeGreaterComp" : day-time-duration-greater-than($dr2, $dr1), "yearMonthLessComp" : year-month-duration-less-than($dr4, $dr3), "dayTimeLessComp" : day-time-duration-less-than($dr2, $dr1), "equal1": duration-equal($dr2, $dr5), "equal2": duration-equal($dr1, $dr5), "equal3": duration-equal($dr6, $dr3), "equal4": duration-equal($dr7, $dr1) }
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.1.ddl.aql
index 0338aef..07825c7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.1.ddl.aql
@@ -14,7 +14,9 @@
   timeField: time?,
   datetimeField: datetime?,
   durationField: duration?,
-  intervalField: interval?
+  intervalField: interval?,
+  yearMonthDurationField: year-month-duration?,
+  dayTimeDurationField: day-time-duration?
 }
 
 create external dataset testds(testtype)
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.3.query.aql
index 2defafc..de39156 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/insert_from_ext_ds/insert_from_ext_ds.3.query.aql
@@ -1,4 +1,4 @@
 use dataverse testdvt;
 
 for $r in dataset("testds") 
-return {"date": $r.dateField, "time": $r.timeField, "datetime": $r.datetimeField, "duration": $r.durationField, "interval": $r.intervalField }
\ No newline at end of file
+return {"date": $r.dateField, "time": $r.timeField, "datetime": $r.datetimeField, "duration": $r.durationField, "interval": $r.intervalField, "ymduration": $r.yearMonthDurationField, "dtduration": $r.dayTimeDurationField }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql
index c042df0..86a4a72 100644
--- a/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/temporal/time_functions/time_functions.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $t1 := time-from-unix-time-in-ms(1560074)
 let $null1 := time-from-unix-time-in-ms(null)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_01/counthashed-gram-tokens_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_01/counthashed-gram-tokens_01.3.query.aql
index 7b2148c..59937c7 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_01/counthashed-gram-tokens_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_01/counthashed-gram-tokens_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Jürgen S. Generic's Car"
 let $tokens := counthashed-gram-tokens($txt, 3, false)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_02/counthashed-gram-tokens_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_02/counthashed-gram-tokens_02.3.query.aql
index f871047..1b86c69 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_02/counthashed-gram-tokens_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-gram-tokens_02/counthashed-gram-tokens_02.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Jürgen S. Generic's Car"
 let $tokens := counthashed-gram-tokens($txt, 3, true)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-word-tokens_01/counthashed-word-tokens_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-word-tokens_01/counthashed-word-tokens_01.3.query.aql
index 3d44cfb..f124c3c 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-word-tokens_01/counthashed-word-tokens_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/counthashed-word-tokens_01/counthashed-word-tokens_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Hello World, I would like to inform you of the importance of Foo Bar. Yes, Foo Bar. Jürgen."
 let $tokens := counthashed-word-tokens($txt)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_01/gram-tokens_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_01/gram-tokens_01.3.query.aql
index ae3a339..e25dc2a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_01/gram-tokens_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_01/gram-tokens_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Jürgen S. Generic's Car"
 let $tokens := gram-tokens($txt, 3, false)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_02/gram-tokens_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_02/gram-tokens_02.3.query.aql
index 1fa90ca..4dbcf94 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_02/gram-tokens_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/gram-tokens_02/gram-tokens_02.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Jürgen S. Generic's Car"
 let $tokens := gram-tokens($txt, 3, true)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_01/hashed-gram-tokens_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_01/hashed-gram-tokens_01.3.query.aql
index 694ef83..66ebdc6 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_01/hashed-gram-tokens_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_01/hashed-gram-tokens_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Jürgen S. Generic's Car"
 let $tokens := hashed-gram-tokens($txt, 3, false)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_02/hashed-gram-tokens_02.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_02/hashed-gram-tokens_02.3.query.aql
index 8e46d4a..4fdba79 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_02/hashed-gram-tokens_02.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-gram-tokens_02/hashed-gram-tokens_02.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Jürgen S. Generic's Car"
 let $tokens := hashed-gram-tokens($txt, 3, true)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-word-tokens_01/hashed-word-tokens_01.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-word-tokens_01/hashed-word-tokens_01.3.query.aql
index 62d8387..c87362e 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-word-tokens_01/hashed-word-tokens_01.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tokenizers/hashed-word-tokens_01/hashed-word-tokens_01.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse test;
+set import-private-functions 'true';
 
 let $txt := "Hello World, I would like to inform you of the importance of Foo Bar. Yes, Foo Bar. Jürgen."
 let $tokens := hashed-word-tokens($txt)
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.3.query.aql
index a75b9c9..dd65443 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q13_customer_distribution/q13_customer_distribution.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse tpch;
+set import-private-functions 'true';
 
 for $gco in (
 	for $co in
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.aql
index 3fd46d5..ec10939 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q19_discounted_revenue/q19_discounted_revenue.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse tpch;
+set import-private-functions 'true';
 
 sum( 
 for $l in dataset('LineItem')
diff --git a/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql
index b6d6c48..c27be0a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/tpch/q1_pricing_summary_report_nt/q1_pricing_summary_report_nt.3.query.aql
@@ -1,4 +1,5 @@
 use dataverse tpch;
+set import-private-functions 'true';
  
 for $l in dataset('LineItem')
 where $l.l_shipdate <= '1998-09-02'
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.1.ddl.aql
new file mode 100644
index 0000000..13b7b58
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.1.ddl.aql
@@ -0,0 +1,9 @@
+/*
+ * Description  : Invoke a private function (internal to Asterix) without setting the 'import-private-functions' flag as true
+ * Expected Res : Failure (Unknown Function)
+ * Date         : 18th May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.2.update.aql
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.2.update.aql
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.3.query.aql
new file mode 100644
index 0000000..b7c627e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/invoke-private-function/invoke-private-function.3.query.aql
@@ -0,0 +1,23 @@
+use dataverse test;
+
+let $a := [ ]
+let $b := ["abc", "bcd", "cde", "def", "efg"]
+let $c := ["abc", "bcd", "cde", "def", "efg", "hij", "ijk"]
+let $d := ["abc", "bcd", "cde", "def", "efg", "fgh", "ghi", "hij", "ijk", "jkl"]
+let $e := ["efg", "abc", "cde", "def", "hij", "ijk", "bcd"]
+let $f := ["abc", "ijk", "bcd", "efg", "fgh", "ghi", "def", "hij", "jkl", "cde"]
+let $g := ["Efg", "aBc", "cdE", "DEf", "hIJ", "IjK", "BCD"]
+let $h := ["abc", "ijk", "bcd", "efg", "fgh", "ghi", "def", "hij", "jkl", "cde"]
+let $results :=
+[
+  similarity-jaccard($a, $b),
+  similarity-jaccard($b, $a),
+  similarity-jaccard($c, $d),
+  similarity-jaccard($d, $c),
+  similarity-jaccard($e, $f),
+  similarity-jaccard($f, $e),
+  similarity-jaccard($g, $h),
+  similarity-jaccard($h, $g)
+]
+for $i in $results
+return $i
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue201/query-issue201.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue201/query-issue201.3.query.aql
index 6dc27e2..ac37c1a 100644
--- a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue201/query-issue201.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue201/query-issue201.3.query.aql
@@ -5,6 +5,7 @@
  * Date         : 26th November 2012
  */
 
+set import-private-functions 'true';
 
 let $x:=range(1,100)
 for $i in $x
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.1.ddl.aql
new file mode 100644
index 0000000..062c4c5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.1.ddl.aql
@@ -0,0 +1,15 @@
+/*
+ * Description  : This test case is to verify the fix for issue 455
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=455
+ * Expected Res : Failure
+ * Date         : 19th May November 2013
+ */
+ 
+
+drop dataverse test if exists;
+create dataverse test;
+
+create function printName()
+{
+"AsterixDB Shared nothing parallel BDMS"
+}
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.2.update.aql
new file mode 100644
index 0000000..3b2310e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.2.update.aql
@@ -0,0 +1,7 @@
+/*
+ * Description  : This test case is to verify the fix for issue 455
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=455
+ * Expected Res : Failure
+ * Date         : 19th May 2013
+ */
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.3.query.aql
new file mode 100644
index 0000000..deea3ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.3.query.aql
@@ -0,0 +1,8 @@
+/*
+ * Description  : This test case is to verify the fix for issue 455
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=455
+ * Expected Res : Failure
+ * Date         : 19th May 2013
+ */
+
+printName();
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.4.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.4.ddl.aql
new file mode 100644
index 0000000..c2986d7
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.4.ddl.aql
@@ -0,0 +1,8 @@
+/*
+ * Description  : This test case is to verify the fix for issue 455
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=455
+ * Expected Res : Failure
+ * Date         : 19th May 2013
+ */
+
+drop function test.printName@0;
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.5.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.5.query.aql
new file mode 100644
index 0000000..d8207e4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/query-issue455/query-issue455.5.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue 455
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=455
+ * Expected Res : Failure
+ * Date         : 19th May 2013
+ */
+
+for $l in dataset Metadata.Function
+return $l
+
+printName();
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf18/udf18.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf18/udf18.3.query.aql
index 0bdf0e4..c2af48d 100644
--- a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf18/udf18.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf18/udf18.3.query.aql
@@ -5,6 +5,7 @@
  */
 
 use dataverse test;
+set import-private-functions 'true';
 
 let $val := not(test.fn06())
 return $val
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf27/udf27.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf27/udf27.3.query.aql
index 50a1cb2..fd383cf 100644
--- a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf27/udf27.3.query.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf27/udf27.3.query.aql
@@ -10,4 +10,4 @@
 use dataverse test;
 
 let $a := true
-return some $i in [100,200] satisfies test.f1()
+return some $i in [100,200] satisfies $i<test.f1()
diff --git a/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf30/udf30.1.query.aql b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf30/udf30.1.query.aql
new file mode 100644
index 0000000..ffe6981
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/user-defined-functions/udf30/udf30.1.query.aql
@@ -0,0 +1,14 @@
+/*
+ * Description  : Declare a UDF and try to use the function parameter outside
+ *                of the function.
+ * Expected Res : Failure
+ * Date         : Apr 10th 2013
+ */
+
+declare function abc($y) {
+ let $x:=3
+ return $x
+};
+
+let $z:=$y
+return $z
diff --git a/asterix-app/src/test/resources/runtimets/results/aggregate/count_null/count_null.1.adm b/asterix-app/src/test/resources/runtimets/results/aggregate/count_null/count_null.1.adm
index 51d5f4f..df462fe 100644
--- a/asterix-app/src/test/resources/runtimets/results/aggregate/count_null/count_null.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/aggregate/count_null/count_null.1.adm
@@ -1 +1 @@
-{ "count": null }
\ No newline at end of file
+{ "count": 2 }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/aggregate/issue395/issue395.1.adm b/asterix-app/src/test/resources/runtimets/results/aggregate/issue395/issue395.1.adm
new file mode 100644
index 0000000..bf0d87a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/aggregate/issue395/issue395.1.adm
@@ -0,0 +1 @@
+4
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/aggregate/issue412_0/issue412_0.1.adm b/asterix-app/src/test/resources/runtimets/results/aggregate/issue412_0/issue412_0.1.adm
new file mode 100644
index 0000000..e440e5c
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/aggregate/issue412_0/issue412_0.1.adm
@@ -0,0 +1 @@
+3
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/aggregate/issue412_1/issue412_1.1.adm b/asterix-app/src/test/resources/runtimets/results/aggregate/issue412_1/issue412_1.1.adm
new file mode 100644
index 0000000..7c1107a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/aggregate/issue412_1/issue412_1.1.adm
@@ -0,0 +1 @@
+{ "count": 3, "average": null, "sum": null, "min": null, "max": null }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/aggregate/query-issue400/query-issue400.1.adm b/asterix-app/src/test/resources/runtimets/results/aggregate/query-issue400/query-issue400.1.adm
new file mode 100644
index 0000000..d8263ee
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/aggregate/query-issue400/query-issue400.1.adm
@@ -0,0 +1 @@
+2
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_count_null/scalar_count_null.1.adm b/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_count_null/scalar_count_null.1.adm
index 1abbc3f..4ff1111 100644
--- a/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_count_null/scalar_count_null.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/aggregate/scalar_count_null/scalar_count_null.1.adm
@@ -1,7 +1,7 @@
-null
-null
-null
-null
-null
-null
-null
\ No newline at end of file
+4
+4
+4
+4
+4
+4
+4
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/day_time_duration_order/day_time_duration_order.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/day_time_duration_order/day_time_duration_order.1.adm
new file mode 100644
index 0000000..4c0aea9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/day_time_duration_order/day_time_duration_order.1.adm
@@ -0,0 +1,4 @@
+day-time-duration("-P48DT12M43.932S")
+day-time-duration("-PT5H28M")
+day-time-duration("P12H")
+day-time-duration("P439D")
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_equality/issue363_equality.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_equality/issue363_equality.1.adm
new file mode 100644
index 0000000..ac3caa3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_equality/issue363_equality.1.adm
@@ -0,0 +1 @@
+{ "duration": true, "year-month-duration": true, "day-time-duration": true, "point": true, "line": true, "polygon": true, "circle": true, "rectangle": true, "interval": true, "duration2": true, "year-month-duration2": true, "day-time-duration2": true, "point2": true, "line2": true, "polygon2": true, "circle2": true, "rectangle2": true, "interval2": true }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_circle/issue363_inequality_circle.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_duration/issue363_inequality_duration.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_interval/issue363_inequality_interval.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_line/issue363_inequality_line.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_point/issue363_inequality_point.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_polygon/issue363_inequality_polygon.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/issue363_inequality_rectangle/issue363_inequality_rectangle.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/comparison/year_month_duration_order/year_month_duration_order.1.adm b/asterix-app/src/test/resources/runtimets/results/comparison/year_month_duration_order/year_month_duration_order.1.adm
new file mode 100644
index 0000000..c0159ce
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/comparison/year_month_duration_order/year_month_duration_order.1.adm
@@ -0,0 +1,4 @@
+year-month-duration("-P49Y")
+year-month-duration("-P27Y4M")
+year-month-duration("P1Y")
+year-month-duration("P439Y")
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.1.adm b/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.1.adm
new file mode 100644
index 0000000..108af42
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/constructor/duration_02/duration_02.1.adm
@@ -0,0 +1 @@
+{ "duration1": year-month-duration("P30Y10M"), "duration2": day-time-duration("P25DT13H12M50S"), "duration3": day-time-duration("PT13H12M50S"), "duration4": year-month-duration("P30Y"), "duration5": day-time-duration("PT13H"), "duration6": year-month-duration("-P30Y10M"), "duration7": day-time-duration("-P25DT13H12M50S"), "duration8": day-time-duration("-PT13H50S"), "duration9": day-time-duration("P120D"), "duration10": year-month-duration("-P2Y4M"), "duration11": day-time-duration("PT30M30.937S"), "duration12": year-month-duration("P301Y3M") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/custord/join_q_04/join_q_04.1.adm b/asterix-app/src/test/resources/runtimets/results/custord/join_q_04/join_q_04.1.adm
index d4721dd..51f998a 100644
--- a/asterix-app/src/test/resources/runtimets/results/custord/join_q_04/join_q_04.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/custord/join_q_04/join_q_04.1.adm
@@ -1,3 +1,5 @@
-{ "cust_name": "Jodi Rotruck", "orderedlist": [ 1000, 66, 775 ], "unorderedlist": {{ 1000, 66, 775 }}, "ol_item1": 1000, "ol_item2": 66, "ol_item5": null, "ul_item1": 1000 }
-{ "cust_name": "Jodi Alex", "orderedlist": [ 10, 48, 5 ], "unorderedlist": {{ 10, 48, 5 }}, "ol_item1": 10, "ol_item2": 48, "ol_item5": null, "ul_item1": 10 }
-{ "cust_name": "Jodi Rotruck", "orderedlist": [ 10, 66, 775 ], "unorderedlist": {{ 10, 66, 775 }}, "ol_item1": 10, "ol_item2": 66, "ol_item5": null, "ul_item1": 10 }
+{ "cust_name": "Jodi Alex", "order_ids": [ 10 ] }
+{ "cust_name": "Jodi Rotruck", "order_ids": [ 10, 1000 ] }
+{ "cust_name": "Mary Carey", "order_ids": [  ] }
+{ "cust_name": "Mike Carey", "order_ids": [  ] }
+{ "cust_name": "Mike ley", "order_ids": [  ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/distinct/query-issue443-2/query-issue443.1.adm b/asterix-app/src/test/resources/runtimets/results/distinct/query-issue443-2/query-issue443.1.adm
new file mode 100644
index 0000000..1a6e796
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/distinct/query-issue443-2/query-issue443.1.adm
@@ -0,0 +1,4 @@
+{ "f": 10, "g": 1 }
+{ "f": 12, "g": 4 }
+{ "f": 17, "g": 1 }
+{ "f": 19, "g": 1 }
diff --git a/asterix-app/src/test/resources/runtimets/results/distinct/query-issue443/query-issue443.1.adm b/asterix-app/src/test/resources/runtimets/results/distinct/query-issue443/query-issue443.1.adm
new file mode 100644
index 0000000..a4a6e48
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/distinct/query-issue443/query-issue443.1.adm
@@ -0,0 +1,4 @@
+{ "f": 10 }
+{ "f": 12 }
+{ "f": 17 }
+{ "f": 19 }
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/query-issue382/query-issue382.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/query-issue382/query-issue382.1.adm
new file mode 100644
index 0000000..9a03714
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/query-issue382/query-issue382.1.adm
@@ -0,0 +1 @@
+10
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/dml/query-issue433/query-issue433.1.adm b/asterix-app/src/test/resources/runtimets/results/dml/query-issue433/query-issue433.1.adm
new file mode 100644
index 0000000..ce89449
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/dml/query-issue433/query-issue433.1.adm
@@ -0,0 +1,2 @@
+{ "id": 1, "name": "u1", "sub": [ { "n": "se1", "e": 100 } ] }
+{ "id": 2, "name": "u2", "sub": [ { "n": "se2", "e": 200 } ] }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/list/query-issue428/query-issue428.1.adm b/asterix-app/src/test/resources/runtimets/results/list/query-issue428/query-issue428.1.adm
new file mode 100644
index 0000000..1140ff5
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/list/query-issue428/query-issue428.1.adm
@@ -0,0 +1,4 @@
+true
+true
+true
+true
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm
new file mode 100644
index 0000000..f93766f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm
@@ -0,0 +1 @@
+{ "a": [ { "id": 21 }, { "id": 23 }, { "id": 24 }, { "id": 44 }, { "id": 64 } ], "b": [ { "id": 21 }, { "id": 23 }, { "id": 24 }, { "id": 44 }, { "id": 64 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue258/query-issue258.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue258/query-issue258.1.adm
new file mode 100644
index 0000000..0e739c3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue258/query-issue258.1.adm
@@ -0,0 +1 @@
+{ "id": 10 }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
new file mode 100644
index 0000000..5196a0f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
@@ -0,0 +1 @@
+{ "id": 13, "name": "Nancy", "age": 32.5f, "salary": 12.0d, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
new file mode 100644
index 0000000..5196a0f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
@@ -0,0 +1 @@
+{ "id": 13, "name": "Nancy", "age": 32.5f, "salary": 12.0d, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
new file mode 100644
index 0000000..6466feb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
@@ -0,0 +1,10 @@
+{ "tweetid": "1", "tweetid-copy": "1", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstein", "followers_count": 3311368 }, "sender-location": point("42.13,80.43"), "send-time": datetime("2005-12-05T21:06:41.000Z"), "send-time-copy": datetime("2005-12-05T21:06:41.000Z"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " love samsung the plan is amazing" }
+{ "tweetid": "10", "tweetid-copy": "10", "user": { "screen-name": "Rolldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstful", "followers_count": 3311368 }, "sender-location": point("46.94,93.98"), "send-time": datetime("2011-04-07T14:08:46.000Z"), "send-time-copy": datetime("2011-04-07T14:08:46.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is good" }
+{ "tweetid": "2", "tweetid-copy": "2", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "David Eckhardstein", "followers_count": 3311368 }, "sender-location": point("28.86,70.44"), "send-time": datetime("2007-08-15T06:44:17.000Z"), "send-time-copy": datetime("2007-08-15T06:44:17.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint its voice-clarity is mind-blowing" }
+{ "tweetid": "3", "tweetid-copy": "3", "user": { "screen-name": "RollandEckhard#500", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Hetfield", "followers_count": 3311368 }, "sender-location": point("39.84,86.48"), "send-time": datetime("2008-12-24T00:07:04.000Z"), "send-time-copy": datetime("2008-12-24T00:07:04.000Z"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is terrible:(" }
+{ "tweetid": "4", "tweetid-copy": "4", "user": { "screen-name": "RollandEckhardstein#221", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstinz", "followers_count": 3311368 }, "sender-location": point("27.67,87.32"), "send-time": datetime("2007-02-05T16:39:13.000Z"), "send-time-copy": datetime("2007-02-05T16:39:13.000Z"), "referred-topics": {{ "t-mobile", "customer-service" }}, "message-text": " love t-mobile its customer-service is mind-blowing" }
+{ "tweetid": "5", "tweetid-copy": "5", "user": { "screen-name": "RollandEcstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardst", "followers_count": 3311368 }, "sender-location": point("27.3,92.77"), "send-time": datetime("2010-09-12T06:15:28.000Z"), "send-time-copy": datetime("2010-09-12T06:15:28.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile the customization is amazing:)" }
+{ "tweetid": "6", "tweetid-copy": "6", "user": { "screen-name": "Rollkhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Kirk Hammette ", "followers_count": 3311368 }, "sender-location": point("45.62,84.78"), "send-time": datetime("2012-01-23T06:23:13.000Z"), "send-time-copy": datetime("2012-01-23T06:23:13.000Z"), "referred-topics": {{ "iphone", "network" }}, "message-text": " like iphone its network is awesome:)" }
+{ "tweetid": "7", "tweetid-copy": "7", "user": { "screen-name": "andEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland khardstein", "followers_count": 3311368 }, "sender-location": point("44.12,81.46"), "send-time": datetime("2012-02-17T17:30:26.000Z"), "send-time-copy": datetime("2012-02-17T17:30:26.000Z"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " hate t-mobile the network is bad" }
+{ "tweetid": "8", "tweetid-copy": "8", "user": { "screen-name": "Rolltein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Ron Eckhardstein", "followers_count": 3311368 }, "sender-location": point("36.86,90.71"), "send-time": datetime("2009-03-12T13:18:04.000Z"), "send-time-copy": datetime("2009-03-12T13:18:04.000Z"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " dislike at&t its touch-screen is OMG" }
+{ "tweetid": "9", "tweetid-copy": "9", "user": { "screen-name": "Roldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckdstein", "followers_count": 3311368 }, "sender-location": point("29.07,97.05"), "send-time": datetime("2012-08-15T20:19:46.000Z"), "send-time-copy": datetime("2012-08-15T20:19:46.000Z"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " hate verizon its speed is bad" }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
new file mode 100644
index 0000000..e5ac8ab
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
@@ -0,0 +1 @@
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420, "hobbies": [ "basket weaving", "mud wrestling" ] }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000Z"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm
new file mode 100644
index 0000000..c3bb80f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm
@@ -0,0 +1,29 @@
+{ "id": 9142198, "similar-users": [  ], "name": "SherryFea" }
+{ "id": 9313492, "similar-users": [  ], "name": "TeraWolfe" }
+{ "id": 9478720, "similar-users": [  ], "name": "AngeliaKettlewell" }
+{ "id": 10001080, "similar-users": [  ], "name": "GarrettBode" }
+{ "id": 10179538, "similar-users": [  ], "name": "OrlandoBaxter" }
+{ "id": 10307032, "similar-users": [  ], "name": "QuentinSauter" }
+{ "id": 10394488, "similar-users": [  ], "name": "OswaldRay" }
+{ "id": 10423588, "similar-users": [  ], "name": "ShirleneRuch" }
+{ "id": 10495420, "similar-users": [  ], "name": "WendyMcloskey" }
+{ "id": 11307946, "similar-users": [  ], "name": "HelgaStough" }
+{ "id": 11447332, "similar-users": [  ], "name": "SherisseMaugham" }
+{ "id": 11570326, "similar-users": [  ], "name": "LindenFilby" }
+{ "id": 11951098, "similar-users": [  ], "name": "TeraByers" }
+{ "id": 11954992, "similar-users": [  ], "name": "CaitlinLangston" }
+{ "id": 9510451, "similar-users": [  ], "name": "ChuckFinck" }
+{ "id": 9594523, "similar-users": [  ], "name": "TamWillcox" }
+{ "id": 9629395, "similar-users": [  ], "name": "JuliusWire" }
+{ "id": 9988417, "similar-users": [  ], "name": "ColineLane" }
+{ "id": 10272571, "similar-users": [  ], "name": "JarrettGoldvogel" }
+{ "id": 10361965, "similar-users": [  ], "name": "ArlenFlick" }
+{ "id": 10498285, "similar-users": [  ], "name": "KileyBridger" }
+{ "id": 10733617, "similar-users": [  ], "name": "LeonardoKight" }
+{ "id": 10874791, "similar-users": [  ], "name": "HaydeeGarratt" }
+{ "id": 10957867, "similar-users": [  ], "name": "ZachOppenheimer" }
+{ "id": 11061631, "similar-users": [  ], "name": "MaxeneKellogg" }
+{ "id": 11068231, "similar-users": [  ], "name": "DinahSwink" }
+{ "id": 11140213, "similar-users": [  ], "name": "MontgomeryWhittier" }
+{ "id": 11381089, "similar-users": [  ], "name": "EarleneAmmons" }
+{ "id": 11675221, "similar-users": [  ], "name": "CalantheGearhart" }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue410/query-issue410.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue410/query-issue410.1.adm
new file mode 100644
index 0000000..ee2dfa4
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue410/query-issue410.1.adm
@@ -0,0 +1 @@
+{"id":0, "name": ""}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue423/query-issue423.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue423/query-issue423.1.adm
new file mode 100644
index 0000000..e69de29
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue423/query-issue423.1.adm
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue442/query-issue442.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue442/query-issue442.1.adm
new file mode 100644
index 0000000..e34c554
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue442/query-issue442.1.adm
@@ -0,0 +1,7 @@
+{ "f": "zzzzz" }
+{ "f": 999999 }
+{ "f": 100 }
+{ "f": 1 }
+{ "f": 0 }
+{ "f": -1 }
+{ "f": null }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
new file mode 100644
index 0000000..b573845
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453-2/query-issue453-2.1.adm
@@ -0,0 +1,16 @@
+{ "id": 0, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" }
+{ "id": 1, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null }
+{ "id": 2, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": "a" }
+{ "id": 3, "int_m": 1, "int_o": 1, "string_m": "b", "string_o": null }
+{ "id": 4, "int_m": 1, "int_o": null, "string_m": "a", "string_o": "a" }
+{ "id": 5, "int_m": 1, "int_o": null, "string_m": "a", "string_o": null }
+{ "id": 6, "int_m": 1, "int_o": null, "string_m": "b", "string_o": "a" }
+{ "id": 7, "int_m": 1, "int_o": null, "string_m": "b", "string_o": null }
+{ "id": 8, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": "a" }
+{ "id": 9, "int_m": 2, "int_o": 1, "string_m": "a", "string_o": null }
+{ "id": 10, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": "a" }
+{ "id": 11, "int_m": 2, "int_o": 1, "string_m": "b", "string_o": null }
+{ "id": 12, "int_m": 2, "int_o": null, "string_m": "a", "string_o": "a" }
+{ "id": 13, "int_m": 2, "int_o": null, "string_m": "a", "string_o": null }
+{ "id": 14, "int_m": 2, "int_o": null, "string_m": "b", "string_o": "a" }
+{ "id": 15, "int_m": 2, "int_o": null, "string_m": "b", "string_o": null }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453/query-issue453.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453/query-issue453.1.adm
new file mode 100644
index 0000000..c076685
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue453/query-issue453.1.adm
@@ -0,0 +1,2 @@
+{ "id": 0, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": "a" }
+{ "id": 1, "int_m": 1, "int_o": 1, "string_m": "a", "string_o": null }
diff --git a/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm b/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm
index 6a2a2c6..d4de868 100644
--- a/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/spatial/create-rtree-index/create-rtree-index.1.adm
@@ -18,4 +18,4 @@
 18
 19
 20
-21
\ No newline at end of file
+21
diff --git a/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm b/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm
index b087496..9573677 100644
--- a/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm
+++ b/asterix-app/src/test/resources/runtimets/results/temporal/insert_from_ext_ds/insert_from_ext_ds.1.adm
@@ -1,3 +1,4 @@
-{ "date": date("-2012-12-12"), "time": time("23:49:12.390Z"), "datetime": datetime("2012-12-12T00:00:00.001Z"), "duration": duration("P20Y19DT4H14M23.34S"), "interval": interval-datetime("2012-12-12T00:00:00.001Z, 2013-08-10T22:10:15.398Z") }
-{ "date": null, "time": time("04:12:12.219Z"), "datetime": datetime("1920-12-21T11:29:18.478Z"), "duration": null, "interval": interval-time("04:29:30.000Z, 07:59:59.999Z") }
-{ "date": null, "time": null, "datetime": datetime("-0290-03-22T17:59:48.999Z"), "duration": duration("-P27Y148D"), "interval": interval-date("-2012-03-17, 2013-04-01") }
\ No newline at end of file
+{ "date": date("-2012-12-12"), "time": time("23:49:12.390Z"), "datetime": datetime("2012-12-12T00:00:00.001Z"), "duration": duration("P20Y19DT4H14M23.34S"), "interval": interval-datetime("2012-12-12T00:00:00.001Z, 2013-08-10T22:10:15.398Z"), "ymduration": null, "dtduration": null }
+{ "date": null, "time": time("04:12:12.219Z"), "datetime": datetime("1920-12-21T11:29:18.478Z"), "duration": null, "interval": interval-time("04:29:30.000Z, 07:59:59.999Z"), "ymduration": null, "dtduration": null }
+{ "date": null, "time": null, "datetime": datetime("-0290-03-22T17:59:48.999Z"), "duration": duration("-P27Y148D"), "interval": interval-date("-2012-03-17, 2013-04-01"), "ymduration": null, "dtduration": null }
+{ "date": null, "time": null, "datetime": null, "duration": null, "interval": null, "ymduration": year-month-duration("P31Y2M"), "dtduration": day-time-duration("-P148D") }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 278d04a..9e6c308 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -1,6 +1,26 @@
 <test-suite xmlns="urn:xml.testframework.asterix.ics.uci.edu" ResultOffsetPath="results" QueryOffsetPath="queries" QueryFileExtension=".aql">
   <test-group name="aggregate">
     <test-case FilePath="aggregate">
+      <compilation-unit name="query-issue400">
+        <output-dir compare="Text">query-issue400</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue395">
+        <output-dir compare="Text">issue395</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue412_0">
+        <output-dir compare="Text">issue412_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
+      <compilation-unit name="issue412_1">
+        <output-dir compare="Text">issue412_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="aggregate">
       <compilation-unit name="avg_double">
         <output-dir compare="Text">avg_double</output-dir>
       </compilation-unit>
@@ -97,6 +117,9 @@
       </compilation-unit>
     </test-case>
     -->
+    <!-- TODO(madhusudancs): These tests that test for local_<agg>/global_<agg> functions should be removed, but
+    before that we should modify the code to make sure those built-in functions are still defined but not exposed
+    by AQL, so leaving these test cases commented.
     <test-case FilePath="aggregate">
       <compilation-unit name="global-avg_01">
         <output-dir compare="Text">global-avg_01</output-dir>
@@ -167,6 +190,7 @@
         <output-dir compare="Text">local-avg_int8_null</output-dir>
       </compilation-unit>
     </test-case>
+    -->
     <test-case FilePath="aggregate">
       <compilation-unit name="max_empty_01">
         <output-dir compare="Text">max_empty_01</output-dir>
@@ -367,6 +391,11 @@
   </test-group>
   <test-group name="comparison">
     <test-case FilePath="comparison">
+      <compilation-unit name="year_month_duration_order">
+        <output-dir compare="Text">year_month_duration_order</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
       <compilation-unit name="datetime_order">
         <output-dir compare="Text">datetime_order</output-dir>
       </compilation-unit>
@@ -493,6 +522,53 @@
         <output-dir compare="Text">string_null</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_equality">
+        <output-dir compare="Text">issue363_equality</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_duration">
+        <output-dir compare="Text">issue363_inequality_duration</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_interval">
+        <output-dir compare="Text">issue363_inequality_interval</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_point">
+        <output-dir compare="Text">issue363_inequality_point</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_line">
+        <output-dir compare="Text">issue363_inequality_line</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_polygon">
+        <output-dir compare="Text">issue363_inequality_polygon</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_rectangle">
+        <output-dir compare="Text">issue363_inequality_rectangle</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="comparison">
+      <compilation-unit name="issue363_inequality_circle">
+        <output-dir compare="Text">issue363_inequality_circle</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="constructor">
     <test-case FilePath="constructor">
@@ -531,6 +607,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="constructor">
+      <compilation-unit name="duration_02">
+        <output-dir compare="Text">duration_02</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="constructor">
       <compilation-unit name="float_01">
         <output-dir compare="Text">float_01</output-dir>
       </compilation-unit>
@@ -685,13 +766,11 @@
         <output-dir compare="Text">join_q_03</output-dir>
       </compilation-unit>
     </test-case>
-    <!--
     <test-case FilePath="custord">
       <compilation-unit name="join_q_04">
         <output-dir compare="Text">join_q_04</output-dir>
       </compilation-unit>
     </test-case>
-    -->
     <test-case FilePath="custord">
       <compilation-unit name="load-test">
         <output-dir compare="Text">load-test</output-dir>
@@ -749,6 +828,16 @@
   </test-group>
   <test-group name="dml">
      <test-case FilePath="dml">
+      <compilation-unit name="query-issue382">
+        <output-dir compare="Text">query-issue382</output-dir>
+      </compilation-unit>
+     </test-case>
+     <test-case FilePath="dml">
+      <compilation-unit name="query-issue433">
+        <output-dir compare="Text">query-issue433</output-dir>
+      </compilation-unit>
+     </test-case>
+     <test-case FilePath="dml">
       <compilation-unit name="query-issue288">
         <output-dir compare="Text">query-issue288</output-dir>
       </compilation-unit>
@@ -1668,6 +1757,7 @@
         <output-dir compare="Text">dblp-csx-3_5</output-dir>
       </compilation-unit>
     </test-case>
+    -->
     <test-case FilePath="fuzzyjoin">
       <compilation-unit name="dblp-csx-aqlplus_1">
         <output-dir compare="Text">dblp-csx-aqlplus_1</output-dir>
@@ -2090,6 +2180,11 @@
         <output-dir compare="Text">unordered-list-constructor_03</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="list">
+      <compilation-unit name="query-issue428">
+        <output-dir compare="Text">query-issue428</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="misc">
   <test-case FilePath="misc">
@@ -2123,11 +2218,13 @@
         <output-dir compare="Text">nested-loop-join_01</output-dir>
       </compilation-unit>
     </test-case>
+   <!--
     <test-case FilePath="misc">
       <compilation-unit name="range_01">
         <output-dir compare="Text">range_01</output-dir>
       </compilation-unit>
     </test-case>
+   -->
     <test-case FilePath="misc">
       <compilation-unit name="tid_01">
         <output-dir compare="Text">tid_01</output-dir>
@@ -2657,6 +2754,63 @@
         <output-dir compare="Text">query-proposal</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue350">
+        <output-dir compare="Text">query-issue350</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue350-2">
+        <output-dir compare="Text">query-issue350-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue343">
+        <output-dir compare="Text">query-issue343</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue343-2">
+        <output-dir compare="Text">query-issue343-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue196">
+        <output-dir compare="Text">query-issue196</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue377">
+        <output-dir compare="Text">query-issue377</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue410">
+        <output-dir compare="Text">query-issue410</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue453">
+	<output-dir compare="Text">query-issue453</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue453-2">
+	<output-dir compare="Text">query-issue453-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue258">
+        <output-dir compare="Text">query-issue258</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue442">
+        <output-dir compare="Text">query-issue442</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="quantifiers">
     <test-case FilePath="quantifiers">
@@ -2736,6 +2890,7 @@
         <output-dir compare="Text">closed-record-constructor_03</output-dir>
       </compilation-unit>
     </test-case>
+    -->
     <test-case FilePath="records">
       <compilation-unit name="expFieldName">
         <output-dir compare="Text">expFieldName</output-dir>
@@ -3540,13 +3695,6 @@
         <output-dir compare="Text">03</output-dir>
       </compilation-unit>
     </test-case>
-    <!--
-    <test-case FilePath="subset-collection">
-      <compilation-unit name="04">
-        <output-dir compare="Text">04</output-dir>
-      </compilation-unit>
-    </test-case>
-    -->
     <test-case FilePath="subset-collection">
       <compilation-unit name="05">
         <output-dir compare="Text">05</output-dir>
@@ -3868,6 +4016,12 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-issue455">
+        <output-dir compare="Text">query-issue455</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="udf01">
         <output-dir compare="Text">udf01</output-dir>
       </compilation-unit>
@@ -4022,11 +4176,23 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="udf30">
+        <output-dir compare="Text">udf30</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="f01">
         <output-dir compare="Text">f01</output-dir>
         <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error> 
       </compilation-unit>
     </test-case>
+    <test-case FilePath="user-defined-functions">
+      <compilation-unit name="invoke-private-function">
+        <output-dir compare="Text">invoke-private-function</output-dir>
+        <expected-error>edu.uci.ics.asterix.common.exceptions.AsterixException</expected-error> 
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="load">
     <test-case FilePath="load">
@@ -4144,11 +4310,13 @@
         <output-dir compare="Text">datetime_functions</output-dir>
       </compilation-unit>
   	</test-case>
-<!--   	<test-case FilePath="temporal">
-  		<compilation-unit name="insert_from_delimited_ds">
-        <output-dir compare="Text">insert_from_delimited_ds</output-dir>
-      </compilation-unit>
-  	</test-case> -->
+        <!--
+   	<test-case FilePath="temporal">
+           <compilation-unit name="insert_from_delimited_ds">
+              <output-dir compare="Text">insert_from_delimited_ds</output-dir>
+           </compilation-unit>
+  	</test-case> 
+        -->
   	<test-case FilePath="temporal">
   		<compilation-unit name="insert_from_ext_ds">
         <output-dir compare="Text">insert_from_ext_ds</output-dir>
@@ -4187,4 +4355,16 @@
       </compilation-unit>
     </test-case>
   </test-group>
-</test-suite>
\ No newline at end of file
+  <test-group name="distinct">
+  	<test-case FilePath="distinct">
+      <compilation-unit name="query-issue443">
+        <output-dir compare="Text">query-issue443</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="distinct">
+      <compilation-unit name="query-issue443-2">
+        <output-dir compare="Text">query-issue443-2</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>
diff --git a/asterix-aql/pom.xml b/asterix-aql/pom.xml
index 038ec62..249299c 100644
--- a/asterix-aql/pom.xml
+++ b/asterix-aql/pom.xml
@@ -41,6 +41,24 @@
 					</execution>
 				</executions>
 			</plugin>
+            <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>build-helper-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>add-source</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>add-source</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>${project.build.directory}/generated-sources/javacc/</source>
+                            </sources>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
 		</plugins>
 		<pluginManagement>
 			<plugins>
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
index 422ca79..d7de106 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/expression/BeginFeedStatement.java
@@ -8,7 +8,9 @@
 import edu.uci.ics.asterix.aql.expression.visitor.IAqlVisitorWithVoidReturn;
 import edu.uci.ics.asterix.aql.parser.AQLParser;
 import edu.uci.ics.asterix.aql.parser.ParseException;
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
 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.metadata.MetadataException;
 import edu.uci.ics.asterix.metadata.MetadataManager;
@@ -35,6 +37,7 @@
         FeedDatasetDetails feedDetails = (FeedDatasetDetails) dataset.getDatasetDetails();
         String functionName = feedDetails.getFunction() == null ? null : feedDetails.getFunction().getName();
         StringBuilder builder = new StringBuilder();
+        builder.append("set" + " " + FunctionUtils.IMPORT_PRIVATE_FUNCTIONS + " " + "'" + Boolean.TRUE + "'" + ";\n");
         builder.append("insert into dataset " + datasetName + " ");
 
         if (functionName == null) {
@@ -65,7 +68,7 @@
         List<Statement> statements;
         try {
             statements = parser.Statement();
-            query = ((InsertStatement) statements.get(0)).getQuery();
+            query = ((InsertStatement) statements.get(1)).getQuery();
         } catch (ParseException pe) {
             throw new MetadataException(pe);
         }
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
index 84d8321..7969941 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/rewrites/AqlRewriter.java
@@ -66,10 +66,10 @@
 import edu.uci.ics.asterix.aql.expression.visitor.IAqlExpressionVisitor;
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 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.metadata.MetadataManager;
 import edu.uci.ics.asterix.metadata.MetadataTransactionContext;
+import edu.uci.ics.asterix.metadata.declared.AqlMetadataProvider;
 import edu.uci.ics.asterix.metadata.entities.Function;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.AsterixFunction;
@@ -82,6 +82,7 @@
     private final List<FunctionDecl> declaredFunctions;
     private final AqlRewritingContext context;
     private final MetadataTransactionContext mdTxnCtx;
+    private final AqlMetadataProvider metadataProvider;
 
     private enum DfsColor {
         WHITE,
@@ -89,11 +90,12 @@
         BLACK
     }
 
-    public AqlRewriter(List<FunctionDecl> declaredFunctions, Query topExpr, MetadataTransactionContext mdTxnCtx) {
+    public AqlRewriter(List<FunctionDecl> declaredFunctions, Query topExpr, AqlMetadataProvider metadataProvider) {
         this.topExpr = topExpr;
         context = new AqlRewritingContext(topExpr.getVarCounter());
         this.declaredFunctions = declaredFunctions;
-        this.mdTxnCtx = mdTxnCtx;
+        this.mdTxnCtx = metadataProvider.getMetadataTxnContext();
+        this.metadataProvider = metadataProvider;
     }
 
     public Query getExpr() {
@@ -145,6 +147,7 @@
                 // loop until no more changes
             }
         }
+        declaredFunctions.removeAll(otherFDecls);
     }
 
     private void buildOtherUdfs(Expression expression, List<FunctionDecl> functionDecls,
@@ -169,7 +172,9 @@
                     buildOtherUdfs(functionDecl.getFuncBody(), functionDecls, declaredFunctions);
                 }
             } else {
-                if (isBuiltinFunction(signature)) {
+                String value = metadataProvider.getConfig().get(FunctionUtils.IMPORT_PRIVATE_FUNCTIONS);
+                boolean includePrivateFunctions = (value != null) ? Boolean.valueOf(value.toLowerCase()) : false;
+                if (isBuiltinFunction(signature, includePrivateFunctions)) {
                     continue;
                 } else {
                     throw new AsterixException(" unknown function " + signature);
@@ -190,15 +195,25 @@
 
     }
 
-    private boolean isBuiltinFunction(FunctionSignature functionSignature) {
-        if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-                functionSignature.getName(), functionSignature.getArity()))) {
+    private boolean isBuiltinFunction(FunctionSignature signature, boolean includePrivateFunctions) {
+        signature.setNamespace(AsterixBuiltinFunctions.FunctionNamespace.ASTERIX_PUBLIC.name());
+        if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(signature.getNamespace(),
+                signature.getName(), signature.getArity()))) {
             return true;
         }
 
-        if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(
-                AlgebricksBuiltinFunctions.ALGEBRICKS_NS, functionSignature.getName(), functionSignature.getArity()))) {
-            return true;
+        if (includePrivateFunctions) {
+            signature.setNamespace(AlgebricksBuiltinFunctions.ALGEBRICKS_NS);
+            if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(signature.getNamespace(),
+                    signature.getName(), signature.getArity()))) {
+                return true;
+            }
+
+            signature.setNamespace(AsterixBuiltinFunctions.FunctionNamespace.ASTERIX_PRIVATE.name());
+            if (AsterixBuiltinFunctions.isBuiltinCompilerFunction(new FunctionIdentifier(signature.getNamespace(),
+                    signature.getName(), signature.getArity()))) {
+                return true;
+            }
         }
 
         return false;
diff --git a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java
index e3f3641..87525c9 100644
--- a/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java
+++ b/asterix-aql/src/main/java/edu/uci/ics/asterix/aql/util/FunctionUtils.java
@@ -32,6 +32,8 @@
 
 public class FunctionUtils {
 
+    public static final String IMPORT_PRIVATE_FUNCTIONS = "import-private-functions";
+
     public static FunctionDecl getFunctionDecl(Function function) throws AsterixException {
         String functionBody = function.getFunctionBody();
         List<String> params = function.getParams();
diff --git a/asterix-aql/src/main/javacc/AQL.jj b/asterix-aql/src/main/javacc/AQL.jj
index 68e1497..2a5f534 100644
--- a/asterix-aql/src/main/javacc/AQL.jj
+++ b/asterix-aql/src/main/javacc/AQL.jj
@@ -74,33 +74,41 @@
     
     // data generator hints
     private static final String DGEN_HINT = "dgen";
+    
+    private static class IndexParams {
+      public IndexType type;
+      public int gramLength;
+      
+      public IndexParams(IndexType type, int gramLength) {
+        this.type = type;
+        this.gramLength = gramLength;
+      }
+    };  
    
     private static String getHint(Token t) {
-       if (t.specialToken == null) {
-         return null;
-       }       
-       String s = t.specialToken.image;
-       int n = s.length();
-       if (n < 2) {
-         return null;
-       }  
-       return s.substring(1).trim();
+        if (t.specialToken == null) {
+            return null;
+        }       
+        String s = t.specialToken.image;
+        int n = s.length();
+        if (n < 2) {
+            return null;
+        }  
+        return s.substring(1).trim();
     }
 
     public AQLParser(String s){
-		this(new StringReader(s));
-		super.setInput(s);
-	}
-	
-	public static void main(String args[]) throws ParseException, TokenMgrError, IOException, FileNotFoundException, AsterixException {
-			File file = new File(args[0]);
-			Reader fis = new BufferedReader(new InputStreamReader(new FileInputStream(file), "UTF-8"));
-		    AQLParser parser = new AQLParser(fis);
-		    List<Statement> st = parser.Statement();
-		    //st.accept(new AQLPrintVisitor(), 0);
-	}
+        this(new StringReader(s));
+        super.setInput(s);
+    }
 
-
+    public static void main(String args[]) throws ParseException, TokenMgrError, IOException, FileNotFoundException, AsterixException {
+        File file = new File(args[0]);
+        Reader fis = new BufferedReader(new InputStreamReader(new FileInputStream(file), "UTF-8"));
+        AQLParser parser = new AQLParser(fis);
+        List<Statement> st = parser.Statement();
+        //st.accept(new AQLPrintVisitor(), 0);
+    }
 }
 
 PARSER_END(AQLParser)
@@ -110,235 +118,575 @@
 {
   scopeStack.push(RootScopeFactory.createRootScope(this));
   List<Statement> decls = new ArrayList<Statement>();
-  Query query=null;
+  Statement stmt = null;
 }
 {
-    (
-      (
-        ( 
-          "use"
-            {
-              decls.add(DataverseDeclaration());
-            }           
-          | "declare" "function" { 
-                              decls.add(FunctionDeclaration()); 
-             }
-	  	   |  "create" (
-	  	   	  {
-                String hint = getHint(token);
-                boolean dgen = false;
-         	   	if (hint != null && hint.startsWith(DGEN_HINT)) {
-         	   	  dgen = true;
-         	   	}                  
-              } 
-              "type"     
-              	{ 
-                              decls.add(TypeDeclaration(dgen, hint)); 
-                 }
-              | "nodegroup" 
-              	{
-                              decls.add(NodegroupDeclaration());
-                }   
-              | "external" <DATASET>
-            	{   
-              		decls.add(DatasetDeclaration(DatasetType.EXTERNAL));
-            	}
-              | "feed" <DATASET>
-                {
-                   decls.add(DatasetDeclaration(DatasetType.FEED)); 	
-            	}
-              | <DATASET>
-          		{
-            		decls.add(DatasetDeclaration(DatasetType.INTERNAL));
-          		}
-              | "index" 
-              	{
-              				decls.add(CreateIndexStatement());
-                 }
-	          | "dataverse"
-          		{
-            		decls.add(CreateDataverseStatement());
-          		}
-          	  | "function"
-          	    {
-          	        decls.add(FunctionCreation());
-          	    }	
-            )        
-         	 | "load" {
-                       decls.add(LoadStatement());
-                   }  
-          	                    
-          	| "drop"
-			(
-          	<DATASET>
-          	{
-            		decls.add(DropStatement());
-          	}
-        	| "index"
-          	{
-            		decls.add(IndexDropStatement());
-          	}
-        	| "nodegroup"
-          	{
-            		decls.add(NodeGroupDropStatement());
-          	}
-        	| "type"
-          	{
-            		decls.add(TypeDropStatement());
-          	}
-        	| "dataverse"
-          	{
-            		decls.add(DataverseDropStatement());
-          	}
-          	| "function"
-          	{
-            		decls.add(FunctionDropStatement());
-          	}
-        	)
-          | "write" {
-                       decls.add(WriteStatement());
-                    }              
-          | "set" {
-                       decls.add(SetStatement());
-                    }                                                      
-	  	  | "insert" {
-		       decls.add(InsertStatement());
-			}
-          | "delete" {
-			decls.add(DeleteStatement());
-		    }
-          | "update" {
-	  		decls.add(UpdateStatement());		
-	  	    } 
-	  	  | "begin" "feed"  
-	  	      {
-                Pair<Identifier,Identifier> nameComponents = getDotSeparatedPair();
-                decls.add(new BeginFeedStatement(nameComponents.first, nameComponents.second, getVarCounter()));
-	  	      } ";"
-	  	      
-	  	  | "suspend" "feed"  
-	  	  	 {
-                decls.add(ControlFeedDeclaration(ControlFeedStatement.OperationType.SUSPEND));
-             } ";"
-	  	   | "resume" "feed"   {
-                decls.add(ControlFeedDeclaration(ControlFeedStatement.OperationType.RESUME));
-	  	   } ";"
-	  	   | "end" "feed"   {
-	  	        decls.add(ControlFeedDeclaration(ControlFeedStatement.OperationType.END));
-	  	   } ";" 
-	  	   | "alter" "feed"  {
-      	        decls.add(AlterFeedDeclaration());
-           } ";"
-           
-           | (query = Query()) {
-               decls.add(query);
-           }
-           )*
-          //  (query = Query())?
-      )
-
-      <EOF>
-    )
+  ( stmt = SingleStatement() (";") ?
     {
-     return decls;  
+      decls.add(stmt);
+    }
+  )*
+  <EOF>
+  {
+    return decls;  
+  }
+}
+
+Statement SingleStatement() throws ParseException:
+{
+  Statement stmt = null;
+}
+{
+  (
+    stmt = DataverseDeclaration()
+    | stmt = FunctionDeclaration()
+    | stmt = CreateStatement()
+    | stmt = LoadStatement()
+    | stmt = DropStatement()
+    | stmt = WriteStatement()
+    | stmt = SetStatement()
+    | stmt = InsertStatement()
+    | stmt = DeleteStatement()
+    | stmt = UpdateStatement()       
+    | stmt = FeedStatement()
+    | stmt = Query()
+  )
+  {
+    return stmt;
+  }
+}
+
+DataverseDecl DataverseDeclaration() throws ParseException:
+{
+  String dvName = null;
+}
+{
+  "use" "dataverse" dvName = Identifier()
+    {
+      defaultDataverse = dvName;
+      return new DataverseDecl(new Identifier(dvName));
+    }
+}
+
+Statement CreateStatement() throws ParseException:
+{
+  String hint = null;
+  boolean dgen = false;
+  Statement stmt = null;
+}
+{
+  "create"
+  (
+    {
+      hint = getHint(token);
+      if (hint != null && hint.startsWith(DGEN_HINT)) {
+        dgen = true;
+      }                  
+    }
+    stmt = TypeSpecification(hint, dgen)
+    | stmt = NodegroupSpecification()
+    | stmt = DatasetSpecification()    
+    | stmt = IndexSpecification()
+    | stmt = DataverseSpecification()
+    | stmt = FunctionSpecification()
+  )        
+  {
+    return stmt;
+  }
+}
+
+TypeDecl TypeSpecification(String hint, boolean dgen) throws ParseException:
+{
+  Pair<Identifier,Identifier> nameComponents = null;
+  boolean ifNotExists = false;
+  TypeExpression typeExpr = null;
+}
+{
+  "type" nameComponents = FunctionOrTypeName() ifNotExists = IfNotExists()
+  "as" typeExpr = TypeExpr()
+    {
+      long numValues = -1;
+      String filename = null;
+      if (dgen) {       
+        String splits[] = hint.split(" +");
+        if (splits.length != 3) {
+          throw new ParseException("Expecting /*+ dgen <filename> <numberOfItems> */");
+        } 
+        filename = splits[1];
+        numValues = Long.parseLong(splits[2]);
+      }  
+      TypeDataGen tddg = new TypeDataGen(dgen, filename, numValues);
+      return new TypeDecl(nameComponents.first, nameComponents.second, typeExpr, tddg, ifNotExists);
+    }
+}
+
+
+NodegroupDecl NodegroupSpecification() throws ParseException:
+{
+  String name = null;
+  String tmp = null;
+  boolean ifNotExists = false;
+  List<Identifier>ncNames = null;
+}
+{
+  "nodegroup" name = Identifier()
+  ifNotExists = IfNotExists() "on" tmp = Identifier()
+    {
+      ncNames = new ArrayList<Identifier>();
+      ncNames.add(new Identifier(tmp));
+    }
+  ( "," tmp = Identifier()
+    {
+      ncNames.add(new Identifier(tmp));
+    }
+  )*
+    {
+      return new NodegroupDecl(new Identifier(name), ncNames, ifNotExists);
+    }
+}
+
+DatasetDecl DatasetSpecification() throws ParseException:
+{
+  Pair<Identifier,Identifier> nameComponents = null;  
+  boolean ifNotExists = false;
+  String typeName = null;
+  String adapterName = null;
+  Map<String,String> properties = null;
+  FunctionSignature appliedFunction = null;
+  List<String> primaryKeyFields = null;
+  String nodeGroupName = null;
+  Map<String,String> hints = new HashMap<String,String>();  
+  DatasetDecl dsetDecl = null;
+}
+{
+  (
+    "external" <DATASET> nameComponents = QualifiedName()
+    <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
+    ifNotExists = IfNotExists()
+    "using" adapterName = AdapterName() properties = Configuration()
+    ( "hints" hints = Properties() )?
+      {
+        ExternalDetailsDecl edd = new ExternalDetailsDecl();
+        edd.setAdapter(adapterName);
+        edd.setProperties(properties);
+        dsetDecl = new DatasetDecl(nameComponents.first,
+                                   nameComponents.second,
+                                   new Identifier(typeName),
+                                   hints,
+                                   DatasetType.EXTERNAL,
+                                   edd,
+                                   ifNotExists);
+      } 
+
+    | "feed" <DATASET> nameComponents = QualifiedName()
+    <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
+    ifNotExists = IfNotExists()
+    "using" adapterName = AdapterName() properties = Configuration()
+    (appliedFunction = ApplyFunction())? primaryKeyFields = PrimaryKey()
+    ( "on" nodeGroupName = Identifier() )?
+    ( "hints" hints = Properties() )?
+      {
+        FeedDetailsDecl fdd = new FeedDetailsDecl(adapterName,
+                                                  properties,
+                                                  appliedFunction,
+                                                  nodeGroupName != null
+                                                    ? new Identifier(nodeGroupName)
+                                                    : null,
+                                                  primaryKeyFields);
+        dsetDecl = new DatasetDecl(nameComponents.first,
+                                   nameComponents.second,
+                                   new Identifier(typeName),
+                                   hints,
+                                   DatasetType.FEED,
+                                   fdd,
+                                   ifNotExists);
+      }
+    | ("internal")? <DATASET> nameComponents = QualifiedName()
+    <LEFTPAREN> typeName = Identifier() <RIGHTPAREN>
+    ifNotExists = IfNotExists()
+    primaryKeyFields = PrimaryKey() ("on" nodeGroupName = Identifier() )?
+    ( "hints" hints = Properties() )?
+      {
+        InternalDetailsDecl idd = new InternalDetailsDecl(nodeGroupName != null
+                                                            ? new Identifier(nodeGroupName)
+                                                            : null,
+                                                          primaryKeyFields);
+        dsetDecl = new DatasetDecl(nameComponents.first,
+                                   nameComponents.second,
+                                   new Identifier(typeName),
+                                   hints,
+                                   DatasetType.INTERNAL,
+                                   idd,
+                                   ifNotExists);
+      }
+  )
+    {
+      return dsetDecl;
+    }
+}
+
+CreateIndexStatement IndexSpecification() throws ParseException:
+{
+  CreateIndexStatement cis = new CreateIndexStatement();
+  String indexName = null;
+  String fieldExpr = null;
+  boolean ifNotExists = false;
+  Pair<Identifier,Identifier> nameComponents = null;
+  IndexParams indexType = null;
+}
+{
+  "index" indexName = Identifier()
+  ifNotExists = IfNotExists()
+  "on" nameComponents = QualifiedName()    
+  <LEFTPAREN> ( fieldExpr = Identifier()
+    {
+      cis.addFieldExpr(fieldExpr);
+    }
+  ) ("," fieldExpr = Identifier()
+    {
+      cis.addFieldExpr(fieldExpr);
+    }
+  )* <RIGHTPAREN> ( "type" indexType = IndexType() )?
+    {
+      cis.setIndexName(new Identifier(indexName));
+      cis.setIfNotExists(ifNotExists);
+      cis.setDataverseName(nameComponents.first);
+      cis.setDatasetName(nameComponents.second);
+      if (indexType != null) {
+        cis.setIndexType(indexType.type);
+        cis.setGramLength(indexType.gramLength);
+      }
+      return cis;
+    }
+}
+
+IndexParams IndexType() throws ParseException:
+{
+  IndexType type = null;
+  int gramLength = 0;
+}
+{
+  ("btree"
+    {
+      type = IndexType.BTREE;
+    }         
+  | "rtree"
+    {
+      type = IndexType.RTREE;
+    }
+  | "keyword"
+    {
+      type = IndexType.WORD_INVIX;
+    }
+  | "fuzzy keyword"
+    {
+      type = IndexType.FUZZY_WORD_INVIX;
+    }
+  | "ngram" <LEFTPAREN> <INTEGER_LITERAL>
+    {
+      type = IndexType.NGRAM_INVIX;
+      gramLength = Integer.valueOf(token.image);
+    }
+  <RIGHTPAREN>
+  | "fuzzy ngram" <LEFTPAREN> <INTEGER_LITERAL>
+    {
+      type = IndexType.FUZZY_NGRAM_INVIX;
+      gramLength = Integer.valueOf(token.image);
+    }
+  <RIGHTPAREN>)
+    {
+      return new IndexParams(type, gramLength);
+    }
+}
+
+CreateDataverseStatement DataverseSpecification() throws ParseException :
+{
+  String dvName = null;
+  boolean ifNotExists = false;
+  String format = null;
+}
+{
+  "dataverse" dvName = Identifier()
+  ifNotExists = IfNotExists()
+  ( "with format" format = StringLiteral() )?
+    {
+      return new CreateDataverseStatement(new Identifier(dvName), format, ifNotExists);
+    }
+}
+
+CreateFunctionStatement FunctionSpecification() throws ParseException:
+{
+  FunctionSignature signature;
+  boolean ifNotExists = false;
+  List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
+  String functionBody;
+  Expression functionBodyExpr;
+  Token beginPos;
+  Token endPos;
+  Pair<Identifier,Identifier> nameComponents=null;
+
+  createNewScope();
+}
+{
+  "function" nameComponents = FunctionOrTypeName()
+  ifNotExists = IfNotExists()
+  paramList = ParameterList()
+  "{"
+    {
+      beginPos = token;
+    } 
+  functionBodyExpr = Expression() "}"
+    {
+      endPos = token;
+      functionBody = extractFragment(beginPos.beginLine, beginPos.beginColumn, endPos.beginLine, endPos.beginColumn);
+      String dataverse = nameComponents.first.getValue();
+      String functionName = nameComponents.second.getValue();      
+      signature = new FunctionSignature(dataverse, functionName, paramList.size());
+      getCurrentScope().addFunctionDescriptor(signature, false);
+      return new CreateFunctionStatement(signature, paramList, functionBody, ifNotExists);
+    }
+}
+
+List<VarIdentifier> ParameterList() throws ParseException:
+{
+  List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
+  VarIdentifier var = null;
+}
+{
+  <LEFTPAREN> (<VARIABLE>
+    {
+      var = new VarIdentifier();
+      var.setValue(token.image);
+      paramList.add(var);
+      getCurrentScope().addNewVarSymbolToScope(var);
+    }
+  ("," <VARIABLE>
+    {
+      var = new VarIdentifier();
+      var.setValue(token.image);
+      paramList.add(var);
+      getCurrentScope().addNewVarSymbolToScope(var);
+    }
+  )*)? <RIGHTPAREN>
+    {
+      return paramList;
+    }
+}
+
+boolean IfNotExists() throws ParseException:
+{
+}
+{
+  ( "if not exists"
+    {
+      return true;
+    }
+  )?
+    {
+      return false;
+    }
+}
+
+FunctionSignature ApplyFunction() throws ParseException:
+{
+  FunctionSignature funcSig = null;
+}
+{
+  "apply" "function" funcSig = FunctionSignature()
+    {
+      return funcSig;
+    }
+}
+
+FunctionSignature FunctionSignature() throws ParseException:
+{
+  Pair<Identifier,Identifier> pairId = null;
+  int arity = 0;
+}
+{
+  pairId = FunctionOrTypeName() "@" <INTEGER_LITERAL> 
+    {  
+      arity = new Integer(token.image);
+      if (arity < 0 && arity != FunctionIdentifier.VARARGS) {
+        throw new ParseException(" invalid arity:" + arity);
+      }
+
+      String dataverse = pairId.first.getValue();
+      String functionName = pairId.second.getValue(); 
+      return new FunctionSignature(dataverse, functionName, arity);
+    }
+}
+
+List<String> PrimaryKey() throws ParseException:
+{
+  String tmp = null;
+  List<String> primaryKeyFields = new ArrayList<String>();
+}
+{
+  "primary" "key" tmp = Identifier()
+    {
+      primaryKeyFields.add(tmp);
+    }
+  ( "," tmp = Identifier()
+    {
+      primaryKeyFields.add(tmp);
+    }
+  )*
+    {
+      return primaryKeyFields;
+    }
+}
+
+Statement DropStatement() throws ParseException:
+{
+  String id = null;
+  Pair<Identifier,Identifier> pairId = null;
+  Triple<Identifier,Identifier,Identifier> tripleId = null;
+  FunctionSignature funcSig = null;
+  boolean ifExists = false;
+  Statement stmt = null;
+}
+{
+  "drop"
+  (
+    <DATASET> pairId = QualifiedName() ifExists = IfExists()
+      {
+        stmt = new DropStatement(pairId.first, pairId.second, ifExists);
+      }
+    | "index" tripleId = DoubleQualifiedName() ifExists = IfExists()
+      {
+        stmt = new IndexDropStatement(tripleId.first, tripleId.second, tripleId.third, ifExists);
+      }
+    | "nodegroup" id = Identifier() ifExists = IfExists()
+      {
+        stmt = new NodeGroupDropStatement(new Identifier(id), ifExists);
+      }
+    | "type" pairId = FunctionOrTypeName() ifExists = IfExists()
+      {
+        stmt = new TypeDropStatement(pairId.first, pairId.second, ifExists);
+      }
+    | "dataverse" id = Identifier() ifExists = IfExists()
+      {
+        stmt = new DataverseDropStatement(new Identifier(id), ifExists);
+      }
+    | "function" funcSig = FunctionSignature() ifExists = IfExists()
+      {
+        stmt = new FunctionDropStatement(funcSig, ifExists);
+      }
+  )
+  {
+    return stmt;  
+  }
+}
+
+boolean IfExists() throws ParseException :
+{
+}
+{
+  ( "if" "exists"
+    {
+      return true;
+    }
+  )?
+    {
+      return false;
     }
 }
 
 InsertStatement InsertStatement() throws ParseException:
 {
-	Identifier dataverseName;
-	Identifier datasetName;
-	Pair<Identifier,Identifier> nameComponents = null;
-	Query query;
+  Pair<Identifier,Identifier> nameComponents = null;
+  Query query;
 }
 {
-   "into" <DATASET>
-   
-   {
-    nameComponents = getDotSeparatedPair();
-    dataverseName = nameComponents.first;
-    datasetName = nameComponents.second;
-   }
-    
-    query = Query() (";")?
-   {return new InsertStatement(dataverseName, datasetName, query,  getVarCounter());}
+  "insert" "into" <DATASET> nameComponents = QualifiedName() query = Query()
+    {
+      return new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter());
+    }
 }
 
 DeleteStatement DeleteStatement() throws ParseException:
 {
-	VariableExpr var = null;
-    Identifier dataverseName;
-    Identifier datasetName = null;
-	Expression condition = null;
-	Pair<Identifier, Identifier> nameComponents;
+  VariableExpr var = null;
+  Expression condition = null;
+  Pair<Identifier, Identifier> nameComponents;
 }
 {
-   var = Variable() { getCurrentScope().addNewVarSymbolToScope(var.getVar());  }
-   "from" 
-   <DATASET> 
-   { 
-	  nameComponents  = getDotSeparatedPair();
-   }
-   ("where" condition = Expression())? (";")?
-   {return new DeleteStatement(var, nameComponents.first, nameComponents.second, condition, getVarCounter()); }
+  "delete" var = Variable()
+    {
+      getCurrentScope().addNewVarSymbolToScope(var.getVar());
+    }
+  "from" <DATASET> nameComponents  = QualifiedName() 
+  ("where" condition = Expression())?
+    {
+      return new DeleteStatement(var, nameComponents.first, nameComponents.second, condition, getVarCounter());
+    }
 }
 
 UpdateStatement UpdateStatement() throws ParseException:
 {
-	VariableExpr vars;
-    Expression target;
-	Expression condition;
-	UpdateClause uc;
- 	List<UpdateClause> ucs = new ArrayList<UpdateClause>();
+  VariableExpr vars;
+  Expression target;
+  Expression condition;
+  UpdateClause uc;
+  List<UpdateClause> ucs = new ArrayList<UpdateClause>();
 }
 {
-   vars = Variable()  "in" target = Expression()
-	"where" condition = Expression() 
-	<LEFTPAREN> (uc=UpdateClause() {ucs.add(uc); }  ("," uc=UpdateClause() {ucs.add(uc); } )*) <RIGHTPAREN> ";"
-   {return new UpdateStatement(vars, target, condition, ucs);}
+  "update" vars = Variable() "in" target = Expression()
+  "where" condition = Expression() 
+  <LEFTPAREN> (uc = UpdateClause()
+    {
+      ucs.add(uc);
+    }
+  ("," uc = UpdateClause()
+    {
+      ucs.add(uc);
+    }
+  )*) <RIGHTPAREN>
+    {
+      return new UpdateStatement(vars, target, condition, ucs);
+    }
 }
 
-
-
 UpdateClause UpdateClause() throws ParseException:
 {
-	Expression target = null;
-	Expression value = null ;	
-	InsertStatement is = null;
-	DeleteStatement ds = null;
-	UpdateStatement us = null;
-	Expression condition = null;
-	UpdateClause ifbranch = null;
-	UpdateClause elsebranch = null;
+  Expression target = null;
+  Expression value = null ;	
+  InsertStatement is = null;
+  DeleteStatement ds = null;
+  UpdateStatement us = null;
+  Expression condition = null;
+  UpdateClause ifbranch = null;
+  UpdateClause elsebranch = null;
 }
 {
    "set" target = Expression() ":=" value = Expression() 
-   | "insert" is = InsertStatement()
-   | "delete" ds = DeleteStatement()
-   | "update" us = UpdateStatement()
-   | "if" <LEFTPAREN> condition = Expression() <RIGHTPAREN> "then" ifbranch = UpdateClause() [LOOKAHEAD(1) "else" elsebranch = UpdateClause()] 
-   {return new UpdateClause(target, value, is, ds, us, condition, ifbranch, elsebranch);}
+   | is = InsertStatement()
+   | ds = DeleteStatement()
+   | us = UpdateStatement()
+   | "if" <LEFTPAREN> condition = Expression() <RIGHTPAREN>
+     "then" ifbranch = UpdateClause()
+     [LOOKAHEAD(1) "else" elsebranch = UpdateClause()] 
+     {
+       return new UpdateClause(target, value, is, ds, us, condition, ifbranch, elsebranch);
+     }
 }
 
-
 Statement SetStatement() throws ParseException:
 {
   String pn = null;
-  Statement stmt = null;
+  String pv = null;
 }
 {
-  <IDENTIFIER>  { pn = token.image; }  
-  <STRING_LITERAL>
-    { String pv = removeQuotesAndEscapes(token.image); }
-    ";"
-  {
-    return new SetStatement(pn, pv);
-  }
+  "set" pn = Identifier() pv = StringLiteral()
+    {
+      return new SetStatement(pn, pv);
+    }
 }
 
 Statement WriteStatement() throws ParseException:
 {
-  Identifier nodeName = null;
+  String nodeName = null;
   String fileName = null;
   Statement stmt = null;
   Query query;
@@ -346,288 +694,27 @@
   Pair<Identifier,Identifier> nameComponents = null;
 }
 {
-  (( "output" "to" 
-    <IDENTIFIER> { nodeName = new Identifier(token.image); } 
-    ":" <STRING_LITERAL> { fileName = removeQuotesAndEscapes(token.image); }
-    ( "using" <STRING_LITERAL> { writerClass = removeQuotesAndEscapes(token.image); } )?
-         {                  
-             stmt = new WriteStatement(nodeName, fileName, writerClass);         
-         } )
-    |
-   ( "into" 
-     <DATASET> 
-     
-      {
-       nameComponents = getDotSeparatedPair();
+  "write" ((
+    "output" "to" nodeName = Identifier() ":" fileName = StringLiteral()
+    ( "using" writerClass = StringLiteral() )?
+      {                  
+        stmt = new WriteStatement(new Identifier(nodeName), fileName, writerClass);         
       }
-     
-     <LEFTPAREN> query = Query() <RIGHTPAREN>
-     {
+  ) | (
+    "into" <DATASET> 
+      {
+        nameComponents = QualifiedName();
+      }
+    <LEFTPAREN> query = Query() <RIGHTPAREN>
+      {
         stmt = new WriteFromQueryResultStatement(nameComponents.first, nameComponents.second, query, getVarCounter());
-     } ))  
-        
-    ";"
+      }
+  ))
     {
       return stmt;
     }
 }
 
-CreateIndexStatement CreateIndexStatement() throws ParseException:
-{
-  CreateIndexStatement cis = new CreateIndexStatement();
-  Pair<Identifier,Identifier> nameComponents = null;
-}
-{
-  <IDENTIFIER> { cis.setIndexName(new Identifier(token.image)); }
-  (
-    "if not exists"
-    {
-      cis.setIfNotExists(true);
-    }
-  )?
-  "on"  
-  
-   {
-   nameComponents = getDotSeparatedPair();
-   cis.setDataverseName(nameComponents.first);
-   cis.setDatasetName(nameComponents.second);
-   }
-  
-  <LEFTPAREN>
-  	( <IDENTIFIER> { cis.addFieldExpr(token.image); } )
-  	("," <IDENTIFIER> { cis.addFieldExpr(token.image); })*
-  <RIGHTPAREN>
-    ("type"
-  		("btree" { cis.setIndexType(IndexType.BTREE); }  		
-  		| "rtree" { cis.setIndexType(IndexType.RTREE); }
-  		| "keyword" { cis.setIndexType(IndexType.WORD_INVIX); }
-  		| "fuzzy keyword" { cis.setIndexType(IndexType.FUZZY_WORD_INVIX); }
-  		| "ngram"
-  		  <LEFTPAREN>
-  		  (<INTEGER_LITERAL>
-  		    {
-  		      cis.setIndexType(IndexType.NGRAM_INVIX);
-  		      cis.setGramLength(Integer.valueOf(token.image));
-  		    }
-  		  )
-  		  <RIGHTPAREN>
-  		| "fuzzy ngram"
-  		  <LEFTPAREN>
-  		  (<INTEGER_LITERAL>
-  		    {
-  		      cis.setIndexType(IndexType.FUZZY_NGRAM_INVIX);
-  		      cis.setGramLength(Integer.valueOf(token.image));
-  		    }
-  		  )
-  		  <RIGHTPAREN>
-		)
-  	";"  	
-  	| ";"
-    )
-   {
-     return cis;
-   }
-}
-
-DataverseDecl DataverseDeclaration() throws ParseException:
-{
-  Identifier dvName = null;  
-}
-{
-  "dataverse" <IDENTIFIER> { defaultDataverse = token.image;}
-  ";"
-  {
-    return new DataverseDecl(new Identifier(defaultDataverse));
-  }
-}
-
-DropStatement DropStatement() throws ParseException :
-{
-  Identifier dataverseName = null;
-  Identifier datasetName = null;
-  boolean ifExists = false;
-  Pair<Identifier,Identifier> nameComponents=null;
-}
-{
-   {
-   nameComponents = getDotSeparatedPair();
-   dataverseName = nameComponents.first;
-   datasetName = nameComponents.second;
-   }
-   
-   
-  (
-    "if exists"
-    {
-      ifExists = true;
-    }
-  )? ";"
-  {
-    return new DropStatement(dataverseName, datasetName, ifExists);
-  }
-}
-
-IndexDropStatement IndexDropStatement() throws ParseException :
-{
-  Identifier dataverseName = null;
-  Identifier datasetName = null;
-  Identifier indexName = null;
-  boolean ifExists = false;
-  Triple<Identifier,Identifier,Identifier> nameComponents=null;
-}
-{
-  
-  {
-   nameComponents = getDotSeparatedTriple();
-   dataverseName = nameComponents.first;
-   datasetName = nameComponents.second;
-   indexName = nameComponents.third;
-   }
-  
-  (
-    "if exists"
-    {
-      ifExists = true;
-    }
-  )? ";"
-  {
-    return new IndexDropStatement(dataverseName, datasetName, indexName, ifExists);
-  }
-}
-
-NodeGroupDropStatement NodeGroupDropStatement() throws ParseException :
-{
-  Identifier groupName = null;
-  boolean ifExists = false;
-}
-{
-  < IDENTIFIER >
-  {
-    groupName = new Identifier(token.image);
-  }
-  (
-    "if exists"
-    {
-      ifExists = true;
-    }
-  )? ";"
-  {
-    return new NodeGroupDropStatement(groupName, ifExists);
-  }
-}
-
-TypeDropStatement TypeDropStatement() throws ParseException :
-{
-  Identifier dataverseName = null;
-  Identifier typeName = null;
-  boolean ifExists = false;
-  Pair<Identifier,Identifier> nameComponents;
-}
-{
-  {
-    nameComponents = getDotSeparatedPair();
-    dataverseName = nameComponents.first == null ? new Identifier(defaultDataverse) : nameComponents.first;
-    typeName = nameComponents.second;
-  }
-  (
-    "if exists"
-    {
-      ifExists = true;
-    }
-  )? ";"
-  {
-    return new TypeDropStatement(dataverseName, typeName, ifExists);
-  }
-}
-
-DataverseDropStatement DataverseDropStatement() throws ParseException :
-{
-  Identifier dataverseName = null;
-  boolean ifExists = false;
-}
-{
-  < IDENTIFIER >
-  {
-    dataverseName = new Identifier(token.image);
-  }
-  (
-    "if exists"
-    {
-      ifExists = true;
-    }
-  )? ";"
-  {
-    return new DataverseDropStatement(dataverseName, ifExists);
-  }
-}
-
-CreateDataverseStatement CreateDataverseStatement() throws ParseException :
-{
-  Identifier dvName = null;
-  boolean ifNotExists = false;
-  String format = null;
-}
-{
-  < IDENTIFIER >
-  {
-    dvName = new Identifier(token.image);
-  }
-  (
-    "if not exists"
-    {
-      ifNotExists = true;
-    }
-  )?
-  (
-    "with format" < STRING_LITERAL >
-    {
-      format = removeQuotesAndEscapes(token.image);
-    }
-  )?
-  ";"
-  {
-    return new CreateDataverseStatement(dvName, format, ifNotExists);
-  }
-}
-
-
-FunctionDropStatement FunctionDropStatement() throws ParseException :
-{
-  String dataverse;
-  String functionName;
-  int arity=0;
-  boolean ifExists = false;
-  Pair<Identifier, Identifier> nameComponents=null;
-}
-{
-  {
-     nameComponents = getDotSeparatedPair();
-     dataverse = nameComponents.first != null ? nameComponents.first.getValue() : defaultDataverse;
-     functionName = nameComponents.second.getValue(); 
-  }
-  
-   "@"
-  <INTEGER_LITERAL> 
-  {  
-     Token t= getToken(0);
-	 arity = new Integer(t.image);
-  	 if( arity < 0 && arity != FunctionIdentifier.VARARGS){
-  	 	throw new ParseException(" invalid arity:" + arity);
-  	 } 
-  }
-  
-  (
-    "if exists"
-    {
-      ifExists = true;
-    }
-  )? ";"
-  {
-    return new FunctionDropStatement(new FunctionSignature(dataverse, functionName, arity), ifExists);
-  }
-}
-
-
 LoadFromFileStatement LoadStatement() throws ParseException:
 {
   Identifier dataverseName = null;
@@ -638,483 +725,138 @@
   Pair<Identifier,Identifier> nameComponents = null;
 }
 {
-   <DATASET> 
-   {
-   nameComponents = getDotSeparatedPair();
-   dataverseName = nameComponents.first;
-   datasetName = nameComponents.second;
-   }
-   
-   "using"
-  
+  "load" <DATASET> nameComponents = QualifiedName()
     {
-    	adapterName = getAdapterName();
+      dataverseName = nameComponents.first;
+      datasetName = nameComponents.second;
     }
-   
+  "using" adapterName = AdapterName() properties = Configuration()
+  ("pre-sorted" 
     {
-      properties = getConfiguration();
-    }
-  
-    ("pre-sorted" 
-      {  alreadySorted = true; }
-    )?
-        
-  ";"
-  {
-     return new LoadFromFileStatement(dataverseName, datasetName, adapterName, properties, alreadySorted);
-  }   
-}
-
-
-String getAdapterName() throws ParseException :
-{
-	String adapterName = null;
-}
-{
-    ( 
-      <IDENTIFIER> {
-     	adapterName = (new Identifier(token.image)).getValue();; 
-      }
-      | 
-      <STRING_LITERAL>
-      {
-        adapterName = removeQuotesAndEscapes(token.image);
-      }
-    )
-    {
-	return adapterName;
-	}
-}
-
-
-DatasetDecl DatasetDeclaration(DatasetType datasetType) throws ParseException :
-{
-  DatasetDecl dd = null;
-  Identifier datasetName = null;
-  Identifier dataverseName = null;
-  Identifier itemDataverseName = null;
-  Identifier itemTypeName = null;
-  String nameComponentFirst = null;
-  String nameComponentSecond = null;
-  boolean ifNotExists = false;
-  IDatasetDetailsDecl datasetDetails = null;
-  Pair<Identifier,Identifier> nameComponents = null;
-  Map<String,String> hints = new HashMap<String,String>();	
-}
-{
-  {
-   nameComponents = getDotSeparatedPair();
-   dataverseName = nameComponents.first;
-   datasetName = nameComponents.second;
-   }
-    
-  (
-    "if not exists"
-    {
-      ifNotExists = true;
+      alreadySorted = true;
     }
   )?
-  (
-  	< LEFTPAREN > <IDENTIFIER>
-  	{
-    	itemTypeName = new Identifier(token.image);
-  	}
-  	< RIGHTPAREN >
-  )
-  {
-  	  if(datasetType == DatasetType.INTERNAL) {
-      	datasetDetails = InternalDatasetDeclaration();
-      }
-      else if(datasetType == DatasetType.EXTERNAL) {
-      	datasetDetails = ExternalDatasetDeclaration();
-      }
-      else if(datasetType == DatasetType.FEED) {
-      	datasetDetails = FeedDatasetDeclaration();
-      }
-  }
-  
-  (
-  "hints"
-  {
-      initProperties(hints);
-  }
-  )?
-   ";"
- 
-  {
-   dd = new DatasetDecl(dataverseName, datasetName, itemTypeName, hints, datasetType, datasetDetails,ifNotExists);
-   return dd;
-  }
+    {
+      return new LoadFromFileStatement(dataverseName, datasetName, adapterName, properties, alreadySorted);
+    }   
 }
 
-InternalDetailsDecl InternalDatasetDeclaration() throws ParseException :
-{
-    InternalDetailsDecl idd = null;
-    List<String> primaryKeyFields = new ArrayList<String>();
-    Identifier nodeGroupName=null;
-}
-{
-  (
-    {
-  	  primaryKeyFields = getPrimaryKeyFields();
-    }
-  )
-  
-  (
-  "on" < IDENTIFIER >
-    {
-        nodeGroupName = new Identifier(token.image);
-    }
-  )?
-  
-  {
-    idd = new InternalDetailsDecl(nodeGroupName, primaryKeyFields);
-    return idd;
-  }
-}
 
-ExternalDetailsDecl ExternalDatasetDeclaration() throws ParseException :
+String AdapterName() throws ParseException :
 {
-  ExternalDetailsDecl edd = null;
   String adapterName = null;
-  Map < String, String > properties;
 }
 {
-  {
-    edd = new ExternalDetailsDecl();
-  }
- 
-    "using"
+  adapterName = Identifier()
     {
-    	adapterName = getAdapterName();
+	  return adapterName;
     }
-
-    {
-      properties = getConfiguration();
-    }
-
-    {
-    	  edd = new ExternalDetailsDecl();
-		  edd.setAdapter(adapterName);
-   		  edd.setProperties(properties);
-    } 
- 
-  {
-    return edd;
-  }
 }
 
-FeedDetailsDecl FeedDatasetDeclaration() throws ParseException :
-{
-    FeedDetailsDecl fdd = null;
-    String adapterName = null;
-    Map < String, String > properties;
-	Pair<Identifier,Identifier> nameComponents;
-	List<String> primaryKeyFields = new ArrayList<String>();
-    Identifier nodeGroupName=null;
-    FunctionSignature appliedFunction=null;
-	String dataverse;
-	String functionName;
-	int arity;
-}
-{
-   "using"
-    {
-    	adapterName = getAdapterName();
-    }
-
-    {
-      properties = getConfiguration();
-    }
-  
-  ("apply" "function" 
-  {
-  nameComponents = getDotSeparatedPair();
-  dataverse = nameComponents.first != null ? nameComponents.first.getValue() : defaultDataverse;
-  functionName = nameComponents.second.getValue();
-  }
-  ("@" <INTEGER_LITERAL> 
-     {
-        arity = Integer.parseInt(token.image);
-     }
-  )
-  
-  {
-    appliedFunction = new FunctionSignature(dataverse, functionName, arity);
-  }   
-  )?
-  
-  (
-    {
-  	  primaryKeyFields  = getPrimaryKeyFields();
-    }
-  )
-  
-  (
-  "on" < IDENTIFIER >
-  {
-    	nodeGroupName = new Identifier(token.image);
-  }
-  )?
-  
-  {
-    fdd = new FeedDetailsDecl(adapterName, properties, appliedFunction, nodeGroupName, primaryKeyFields);
-    return fdd;
-  }
-}
-
-List<String> getPrimaryKeyFields()  throws ParseException :
-{
-	List<String> primaryKeyFields = new ArrayList<String>();
-}
-{
-
-  "primary" "key"
-  < IDENTIFIER >
-  {
-    	 primaryKeyFields.add(token.image);
-  }
-  (
-    "," < IDENTIFIER >
-    {
-    	 primaryKeyFields.add(token.image);
-    }
-  )*
-  {
-   	return primaryKeyFields;
-  }
-  
-}
-
-
-
-
-
-ControlFeedStatement ControlFeedDeclaration(ControlFeedStatement.OperationType operationType) throws ParseException :
+Statement FeedStatement() throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
+  Map<String,String> configuration = null;
+  Statement stmt = null;
 }
 {
+  (
+    "begin" "feed" nameComponents = QualifiedName()
+      {
+        stmt = new BeginFeedStatement(nameComponents.first, nameComponents.second, getVarCounter());
+      }
+    | "suspend" "feed" nameComponents = QualifiedName()
+      {
+        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.SUSPEND, nameComponents.first, nameComponents.second);
+      }
+    | "resume" "feed" nameComponents = QualifiedName()
+      {
+        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.RESUME, nameComponents.first, nameComponents.second);
+      }
+    | "end" "feed" nameComponents = QualifiedName()
+      {
+        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.END, nameComponents.first, nameComponents.second);
+      }
+    | "alter" "feed" nameComponents = QualifiedName() "set" configuration = Configuration()
+      {
+        stmt = new ControlFeedStatement(ControlFeedStatement.OperationType.ALTER, nameComponents.first, nameComponents.second, configuration);
+      }
+  )
     {
-    nameComponents = getDotSeparatedPair();
-    return new ControlFeedStatement(operationType, nameComponents.first, nameComponents.second);
+      return stmt;
     }
 }
 
-
-ControlFeedStatement AlterFeedDeclaration() throws ParseException :
-{
-    Pair<Identifier,Identifier> nameComponents = null;
-    Map < String, String > configuration = new HashMap < String, String > ();
-}
-{
-   {
-    nameComponents = getDotSeparatedPair();
-   }
-
-   "set"
-   { 
-   configuration = getConfiguration();
-   }
-  
-  {
-    return new ControlFeedStatement(ControlFeedStatement.OperationType.ALTER, nameComponents.first, nameComponents.second, configuration);
-  }
-}
-
-Map<String,String> getConfiguration()  throws ParseException :
+Map<String,String> Configuration()  throws ParseException :
 {
 	Map<String,String> configuration = new LinkedHashMap<String,String>();
-	String key;
-	String value;
+	Pair<String, String> keyValuePair = null;
 }
 {
-
-<LEFTPAREN>
-    (
-      (
-        <LEFTPAREN>
-        (
-          <STRING_LITERAL>
-          {
-            key = removeQuotesAndEscapes(token.image);
-          }
-          "=" <STRING_LITERAL>
-          {
-            value = removeQuotesAndEscapes(token.image);
-          }
-        )
-        <RIGHTPAREN>
-        {
-          configuration.put(key, value);
-        }
-      )
-      (
-        "," <LEFTPAREN>
-        (
-          <STRING_LITERAL>
-          {
-            key = removeQuotesAndEscapes(token.image);
-          }
-          "=" <STRING_LITERAL>
-          {
-            value = removeQuotesAndEscapes(token.image);
-          }
-        )
-        <RIGHTPAREN>
-        {
-          configuration.put(key, value);
-        }
-      )*
-    )?
-    <RIGHTPAREN>
-     {
-     	return configuration;
-     }
-}
-
-void initProperties(Map<String,String> properties)  throws ParseException :
-{
-	String key;
-	String value;
-}
-{
-    (
-      <LEFTPAREN>
-        (
-          <IDENTIFIER>
-          {
-            key = (new Identifier(token.image)).getValue();
-          }
-          "=" 
-          (
-            (<STRING_LITERAL>
-             {
-              value = removeQuotesAndEscapes(token.image);
-             }
-            ) |
-            (<INTEGER_LITERAL>
-             {
-             try{
-              value = "" + Long.valueOf(token.image);
-              } catch (NumberFormatException nfe){
-                  throw new ParseException("inapproriate value: " + token.image); 
-              }
-             } 
-            )
-          )
-        {
-          properties.put(key.toUpperCase(), value);
-        }
-       ( 
-        "," 
-        (
-          <IDENTIFIER>
-          {
-            key = (new Identifier(token.image)).getValue();
-          }
-          "=" 
-          (
-           (<STRING_LITERAL>
-            {
-              value = removeQuotesAndEscapes(token.image);
-            }
-           ) |
-           (<INTEGER_LITERAL>
-            {
-              try{
-                value = "" + Long.valueOf(token.image);
-              } catch (NumberFormatException nfe){
-              	throw new ParseException("inapproriate value: " + token.image); 
-              }
-            } 
-           )
-          ) 
-        )
-        {
-          properties.put(key.toUpperCase(), value);
-        }
-        
-       )*
-      )
-       <RIGHTPAREN>
-    )?
-}
-
-
-
-NodegroupDecl NodegroupDeclaration() throws ParseException :
-{
-  Identifier name = null;
-  List < Identifier > ncNames = new ArrayList < Identifier > ();
-  boolean ifNotExists = false;
-}
-{
-  < IDENTIFIER >
-  {
-    name = new Identifier(token.image);
-  }
-  (
-    "if not exists"
-    { 
-      ifNotExists = true;
-    }
-  )?
-  "on" < IDENTIFIER >
-  {
-    ncNames.add(new Identifier(token.image));
-  }
-  (
-    "," < IDENTIFIER >
+  <LEFTPAREN> ( keyValuePair = KeyValuePair()
     {
-      ncNames.add(new Identifier(token.image));
+      configuration.put(keyValuePair.first, keyValuePair.second);
     }
-  )*
-  ";"
-  {
-    return new NodegroupDecl(name, ncNames, ifNotExists);
-  }
-}
-
-
-TypeDecl TypeDeclaration(boolean dgen, String hint) throws ParseException:
-{
-  Identifier dataverse;
-  Identifier ident;
-  TypeExpression typeExpr;
-  boolean ifNotExists = false;
-  Pair<Identifier,Identifier> nameComponents=null;	
-}
-{
-  {
-    nameComponents = getDotSeparatedPair();
-    dataverse = nameComponents.first;
-    ident = nameComponents.second; 
-  }
-  
-  (
-    "if not exists"
+  ( "," keyValuePair = KeyValuePair()
     {
-      ifNotExists = true;
+      configuration.put(keyValuePair.first, keyValuePair.second);
     }
-  )?
-  "as"
-  ( typeExpr = TypeExpr() )
-  (";")?
-  {
-    long numValues = -1;
-    String filename = null;
-    if (dgen) {       
-      String splits[] = hint.split(" +");
-      if (splits.length != 3) {
-        throw new ParseException("Expecting /*+ dgen <filename> <numberOfItems> */");
-      } 
-      filename = splits[1];
-      numValues = Long.parseLong(splits[2]);
+  )* )? <RIGHTPAREN>
+    {
+      return configuration;
+    }
+}
+
+Pair<String, String> KeyValuePair() throws ParseException:
+{
+  String key;
+  String value;
+}
+{
+  <LEFTPAREN> key = StringLiteral() "=" value = StringLiteral() <RIGHTPAREN>
+    {
+      return new Pair<String, String>(key, value);
     }  
-    TypeDataGen tddg = new TypeDataGen(dgen, filename, numValues);
-    return new TypeDecl(dataverse, ident, typeExpr, tddg, ifNotExists);
-  }
+}
+
+Map<String,String> Properties() throws ParseException:
+{
+  Map<String,String> properties = new HashMap<String,String>();
+  Pair<String, String> property;
+}
+{
+  ( <LEFTPAREN> property = Property()
+    {
+      properties.put(property.first, property.second);
+    }
+  ( "," property = Property() 
+    {
+      properties.put(property.first, property.second);
+    }
+  )* <RIGHTPAREN> )?
+    {
+      return properties;
+    }
+}
+
+Pair<String, String> Property() throws ParseException:
+{
+  String key;
+  String value;
+}
+{
+  key = Identifier() "=" ( value = StringLiteral() | <INTEGER_LITERAL>
+    {
+      try {
+        value = "" + Long.valueOf(token.image);
+      } catch (NumberFormatException nfe) {
+        throw new ParseException("inapproriate value: " + token.image); 
+      }
+    }
+  )
+    {
+      return new Pair<String, String>(key.toUpperCase(), value);
+    }  
 }
 
 TypeExpression TypeExpr() throws ParseException:
@@ -1181,11 +923,10 @@
         boolean nullable = false;
 }
 {
-      <IDENTIFIER>
+      fieldName = Identifier()
       	{
-	     Token t = getToken(0);
-	     fieldName = t.toString();	     	     
-         String hint = getHint(t);
+	     fieldName = token.image;	     	     
+         String hint = getHint(token);
          IRecordFieldDataGen rfdg = null;
          if (hint != null) { 
            String splits[] = hint.split(" +");
@@ -1237,14 +978,14 @@
 }
 
 TypeReferenceExpression TypeReference() throws ParseException:
-{}
 {
- <IDENTIFIER>
- {
-     Token t = getToken(0);
-     Identifier id = new Identifier(t.toString());
-     return new TypeReferenceExpression(id);
- }
+  String id = null;
+}
+{
+ id = Identifier()
+   {
+     return new TypeReferenceExpression(new Identifier(id));
+   }
 }
 
 OrderedListTypeDefinition OrderedListTypeDef() throws ParseException:
@@ -1274,183 +1015,121 @@
   }
 }
 
-Pair<Identifier,Identifier> getDotSeparatedPair() throws ParseException:
+
+Pair<Identifier,Identifier> FunctionOrTypeName() throws ParseException:
 {
- Identifier first = null;
- Identifier second = null;
+  Pair<Identifier,Identifier> name = null;
 }
 {
-  < IDENTIFIER >
+  name = QualifiedName()
+    {
+      if (name.first == null) {
+        name.first = new Identifier(defaultDataverse);
+      }
+      return name;
+    }
+}
+
+String Identifier() throws ParseException:
+{
+  String lit = null;
+}
+{
+  <IDENTIFIER>
+    {
+      return token.image;
+    }
+  | lit = StringLiteral()
+    {
+      return lit;
+    } 
+}
+
+String StringLiteral() throws ParseException:
+{
+}
+{
+  <STRING_LITERAL>
+    {
+      return removeQuotesAndEscapes(token.image);
+    }
+}
+
+Pair<Identifier,Identifier> QualifiedName() throws ParseException:
+{
+  String first = null;
+  String second = null;
+}
+{
+  first = Identifier() ("." second = Identifier())?
   {
-    first = new Identifier(token.image);
-  } 
-  ("." <IDENTIFIER>
-  {
-    second = new Identifier(token.image);
-  }
-  )?
-  
-  {
-   if(second == null){
-   	second = first;
-   	first = null;
-   } 
-   
-   return new Pair<Identifier,Identifier>(first,second);
+    Identifier id1 = null;
+    Identifier id2 = null;
+    if (second == null) {
+      id2 = new Identifier(first);
+    } else
+    {
+      id1 = new Identifier(first);
+      id2 = new Identifier(second);
+    }
+    return new Pair<Identifier,Identifier>(id1, id2);
   }  
 }  
   
-Triple<Identifier,Identifier,Identifier> getDotSeparatedTriple() throws ParseException:
+Triple<Identifier,Identifier,Identifier> DoubleQualifiedName() throws ParseException:
 {
- Identifier first = null;
- Identifier second = null;
- Identifier third = null;
+  String first = null;
+  String second = null;
+  String third = null;
 }
 {
-  < IDENTIFIER >
+  first = Identifier() "." second = Identifier() ("." third = Identifier())?
   {
-    first = new Identifier(token.image);
-  } 
-  "." <IDENTIFIER>
-  {
-    second = new Identifier(token.image);
-  }
-  (
-  "." <IDENTIFIER>
-  {
-    third = new Identifier(token.image);
-  }
-  )?
-  
-  {
-   if(third == null){
-   	third  = second;
-   	second = first;
-   	first = null;
-   } 
-   
-   return new Triple<Identifier,Identifier,Identifier>(first,second,third);
+    Identifier id1 = null;
+    Identifier id2 = null;
+    Identifier id3 = null;
+    if (third == null) {
+      id2 = new Identifier(first);
+      id3 = new Identifier(second);
+    } else {
+      id1 = new Identifier(first);
+      id2 = new Identifier(second);
+      id3 = new Identifier(third);
+    }
+    return new Triple<Identifier,Identifier,Identifier>(id1, id2, id3);
   }  
 }  
 
-
-  
-
 FunctionDecl FunctionDeclaration() throws ParseException:
 {
   FunctionDecl funcDecl;
   FunctionSignature signature;
   String functionName;
-  int arity = 0;
   List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
   Expression funcBody;
-  VarIdentifier var = null;
   createNewScope();
 }
 {
-
-    <IDENTIFIER>
-	{
-	  Token t = getToken(0);
-	  functionName = t.toString();
-	}
-    <LEFTPAREN> (<VARIABLE>
+  "declare" "function" functionName = Identifier()
+  paramList = ParameterList()
+  "{" funcBody = Expression() "}"
     {
-      var = new VarIdentifier();
-      var.setValue(getToken(0).toString());
-      paramList.add(var);
-      getCurrentScope().addNewVarSymbolToScope(var);
-      arity++;
-    }
-    ("," <VARIABLE>
-    {
-      var = new VarIdentifier();
-      var.setValue(getToken(0).toString());
-      paramList.add(var);
-      getCurrentScope().addNewVarSymbolToScope(var);
-      arity++;
-    })*)? <RIGHTPAREN> "{" funcBody = Expression() "}"
-    (";")?
-    {
-      signature = new FunctionSignature(defaultDataverse, functionName, arity);
+      signature = new FunctionSignature(defaultDataverse, functionName, paramList.size());
       getCurrentScope().addFunctionDescriptor(signature, false);
       funcDecl = new FunctionDecl(signature, paramList, funcBody);
+      removeCurrentScope();
       return funcDecl;
     }
 }
 
-CreateFunctionStatement FunctionCreation() throws ParseException:
-{
-  CreateFunctionStatement cfs = null;
-  FunctionSignature signature;
-  String dataverse;
-  String functionName;
-  boolean ifNotExists = false;
-  List<VarIdentifier> paramList = new ArrayList<VarIdentifier>();
-  String functionBody;
-  VarIdentifier var = null;
-  createNewScope();
-  Expression functionBodyExpr;
-  Token beginPos;
-  Token endPos;
-  Pair<Identifier,Identifier> nameComponents=null;
-}
-{
-    {
-      nameComponents = getDotSeparatedPair();
-   	  dataverse = nameComponents.first != null ? nameComponents.first.getValue() : defaultDataverse;
-   	  functionName= nameComponents.second.getValue();
-	}
-	
-	(
-      "if not exists"
-       {
-         ifNotExists = true;
-       }
-    )?
-	
-    <LEFTPAREN> (<VARIABLE>
-    {
-      var = new VarIdentifier();
-      var.setValue(getToken(0).toString());
-      paramList.add(var);
-      getCurrentScope().addNewVarSymbolToScope(var);
-    }
-    ("," <VARIABLE>
-    {
-      var = new VarIdentifier();
-      var.setValue(getToken(0).toString());
-      paramList.add(var);
-      getCurrentScope().addNewVarSymbolToScope(var);
-    })*)? <RIGHTPAREN>  "{"
-          {
-          beginPos = getToken(0);
-          } 
-          functionBodyExpr = Expression() 
-          "}" 
-          {
-            endPos = getToken(0);
-            functionBody = extractFragment(beginPos.beginLine, beginPos.beginColumn, endPos.beginLine, endPos.beginColumn);
-          }
-          (";")?
-    {
-      signature = new FunctionSignature(dataverse, functionName, paramList.size());
-      getCurrentScope().addFunctionDescriptor(signature, false);
-      cfs = new CreateFunctionStatement(signature, paramList, functionBody, ifNotExists);
-      return cfs;
-    }
-}
 
-
-
-Query Query()throws ParseException:
+Query Query() throws ParseException:
 {
   Query query = new Query();
   Expression expr;
 }
 {
-    expr = Expression()
-    (";")?
+  expr = Expression()
     {
       query.setBody(expr);
       query.setVarCounter(getVarCounter());
@@ -1500,8 +1179,7 @@
   	    op.addOperand(operand);
 	    op.setCurrentop(true);  	      	     
   	  }  
-  	  Token t = getToken(0);
-      op.addOperator(t.toString());
+      op.addOperator(token.image);
 	}
 
 	operand = AndExpr()
@@ -1532,8 +1210,7 @@
   	    op.addOperand(operand);
 	    op.setCurrentop(true);  	      	     
   	  }  
-  	  Token t = getToken(0);
-      op.addOperator(t.toString());
+      op.addOperator(token.image);
 	}
 
 	operand = RelExpr()
@@ -1580,9 +1257,8 @@
   	      op.addOperand(operand, broadcast);
           op.setCurrentop(true);
           broadcast = false;
-  	    }   
-  	    Token t = getToken(0);
-        op.addOperator(t.toString());
+  	    }
+        op.addOperator(token.image);
 	  }
 	  
  	  operand = AddExpr()
@@ -1620,9 +1296,8 @@
   	    op = new OperatorExpr();
         op.addOperand(operand);  	    
         op.setCurrentop(true);        
-  	  } 
-  	  Token t = getToken(0);
-	  ((OperatorExpr)op).addOperator(t.toString());
+  	  }
+	  ((OperatorExpr)op).addOperator(token.image);
 	}
 
 	operand = MultExpr()
@@ -1650,9 +1325,8 @@
   	    op = new OperatorExpr();
         op.addOperand(operand);
         op.setCurrentop(true);          	    
-  	  } 
-  	  Token t = getToken(0);
-	  op.addOperator(t.toString());
+  	  }
+	  op.addOperator(token.image);
 	}
 	operand = UnionExpr()
 	{
@@ -1694,11 +1368,10 @@
 {
 	(( "+"|"-") 
 	{
-	  	uexpr = new UnaryExpr(); 
-		Token t = getToken(0);
-		if("+".equals(t.toString()))
+	  	uexpr = new UnaryExpr();
+		if("+".equals(token.image))
 			((UnaryExpr)uexpr).setSign(Sign.POSITIVE);
-		else if("-".equals(t.toString()))
+		else if("-".equals(token.image))
 			((UnaryExpr)uexpr).setSign(Sign.NEGATIVE);
 		else 
 			throw new ParseException();
@@ -1717,73 +1390,39 @@
 	}
 }
 
-Expression ValueExpr() throws ParseException:
-{
-  Expression expr;
-}
-{
-  expr = FieldOrIndexAccessor()
-  {
-    return expr;
-  }
-}
-
-
-Expression FieldOrIndexAccessor()throws ParseException:
+Expression ValueExpr()throws ParseException:
 {
   Expression expr = null;
   Identifier ident = null;
   AbstractAccessor fa = null;
   int index;
-
 }
 {
-	( expr = PrimaryExpr()
-
-	)
-
-
-	(
-	(
-	  	ident = Field()
+  expr = PrimaryExpr() ( ident = Field()
 	{
-		  if(fa == null)
-		  	fa = new FieldAccessor(expr, ident);
-		  else
-		  	fa = new FieldAccessor(fa, ident);
-	}
-	)
-	| (
-		index = Index()
-		{
-		  if(fa == null)
-		  	fa = new IndexAccessor(expr, index);
-		  else
-		  	fa = new IndexAccessor(fa, index);
-		}
-	) 
-	)*
-
-	
-  	{
- 	  return fa==null?expr:fa;
- 	}
+	  fa = (fa == null ? new FieldAccessor(expr, ident) 
+                       : new FieldAccessor(fa, ident));
+    }
+  | index = Index()
+    {
+      fa = (fa == null ? new IndexAccessor(expr, index)
+                       : new IndexAccessor(fa, index));
+     }
+  )*
+    {
+      return fa == null ? expr : fa;
+    }
 }
 
 Identifier Field() throws ParseException:
 {
-  Identifier ident = null;
-
+  String ident = null;
 }
 {
-  "." < IDENTIFIER >
-  	{
-    
-  	ident = new Identifier();
-	ident.setValue(getToken(0).toString());
-
-	  return ident;
-	}
+  "." ident = Identifier()
+    {
+      return new Identifier(ident);
+    }
 }
 
 int Index() throws ParseException:
@@ -1828,75 +1467,63 @@
   Expression expr = null;
 }
 {
-  //Literal | VariableRef | ListConstructor | RecordConstructor | FunctionCallExpr | DatasetAccessExpression | ParenthesizedExpression
-	(
-	  expr =Literal() 
-	   | expr = FunctionCallExpr()
-	   | expr = DatasetAccessExpression()
-	   | expr =VariableRef() 
-	   
+  ( LOOKAHEAD(2) 
+    expr = FunctionCallExpr()
+  | expr = Literal()
+  | expr = DatasetAccessExpression()
+  | expr = VariableRef() 
     {
       if(((VariableExpr)expr).getIsNewVar() == true)
-      	throw new ParseException("can't find variable " + ((VariableExpr)expr).getVar());
+        throw new ParseException("can't find variable " + ((VariableExpr)expr).getVar());
     }
-    	   | expr = ListConstructor()
-	   | expr = RecordConstructor()
-	   | expr = ParenthesizedExpression()
-	)
- 	{
- 	  return expr;
- 	}
+  | expr = ListConstructor()
+  | expr = RecordConstructor()
+  | expr = ParenthesizedExpression()
+  )
+    {
+      return expr;
+    }
 }
 
 Expression Literal() throws ParseException:
 {
-
   LiteralExpr lit = new LiteralExpr();
-  Token t;
+  String str = null;
 }
 {
-(
-      <STRING_LITERAL>
-	{
-	  t= getToken(0);
-	  lit.setValue( new StringLiteral(removeQuotesAndEscapes(t.image)));
-	}
-    
- 	 | <INTEGER_LITERAL>
+  ( str = StringLiteral()
     {
-      t= getToken(0);
-	  try {
-	      lit.setValue(new IntegerLiteral(new Integer(t.image)));
-	  } catch(NumberFormatException ex) {
-	      lit.setValue(new LongIntegerLiteral(new Long(t.image)));
-	  }
-	}
-     | < FLOAT_LITERAL >
+      lit.setValue(new StringLiteral(str));
+    }
+  | <INTEGER_LITERAL>
     {
-      t= getToken(0);
-      lit.setValue(new FloatLiteral(new Float(t.image)));
-    }     
-	 | < DOUBLE_LITERAL >
+      try {
+        lit.setValue(new IntegerLiteral(new Integer(token.image)));
+      } catch(NumberFormatException ex) {
+        lit.setValue(new LongIntegerLiteral(new Long(token.image)));
+      }
+    }
+  | <FLOAT_LITERAL>
     {
-      t= getToken(0);
-	  lit.setValue(new DoubleLiteral(new Double(t.image)));
-	}	  
- 	 | <NULL>
-	{
-	  t= getToken(0);
-	  lit.setValue(NullLiteral.INSTANCE);
-	}
-   	 | <TRUE>
-	{
-	  t= getToken(0);
-	  lit.setValue(TrueLiteral.INSTANCE);
-	}   	 
-   	 | <FALSE>
-	{
-	  t= getToken(0);
-	  lit.setValue(FalseLiteral.INSTANCE);
-	}
-)
+      lit.setValue(new FloatLiteral(new Float(token.image)));
+    }
+  | <DOUBLE_LITERAL>
+    {
+      lit.setValue(new DoubleLiteral(new Double(token.image)));
+    }
+  | <NULL>
+    {
+      lit.setValue(NullLiteral.INSTANCE);
+    }
+  | <TRUE>
+    {
+      lit.setValue(TrueLiteral.INSTANCE);
+    }      
+  | <FALSE>
+    {
+      lit.setValue(FalseLiteral.INSTANCE);
+    }
+  )
     {
       return lit;
     }
@@ -1907,13 +1534,11 @@
 {
 	VariableExpr varExp = new VariableExpr();
 	VarIdentifier var = new VarIdentifier();
-	Token t;
 }
 {
-      <VARIABLE>
+  <VARIABLE>
     {
-     t = getToken(0);//get current token
-     String varName = t.toString(); 
+     String varName = token.image; 
      Identifier ident = lookupSymbol(varName);
      if (isInForbiddenScopes(varName)) {
        throw new ParseException("Inside limit clauses, it is disallowed to reference a variable having the same name as any variable bound in the same scope as the limit clause.");
@@ -1924,7 +1549,7 @@
      } else {
        varExp.setVar(var);     
      }
-     var.setValue(t.toString());        
+     var.setValue(varName);        
      return varExp;
     }
 }
@@ -1934,18 +1559,16 @@
 {
 	VariableExpr varExp = new VariableExpr();
 	VarIdentifier var = new VarIdentifier();
-	Token t;
 }
 {
-      <VARIABLE>
+  <VARIABLE>
     {
-     t = getToken(0);//get current token
-     Identifier ident = lookupSymbol(t.toString());
+     Identifier ident = lookupSymbol(token.image);
      if(ident != null) { // exist such ident
        varExp.setIsNewVar(false);
      }  
      varExp.setVar(var);     
-     var.setValue(t.toString());        
+     var.setValue(token.image);        
      return varExp;
     }
 }
@@ -2050,36 +1673,42 @@
   List<Expression> argList = new ArrayList<Expression>();
   Expression tmp;
   int arity = 0;
+  Pair<Identifier,Identifier> funcId = null;
   String funcName;
   String dataverse;
-  String hint=null;
-  String id1=null;
-  String id2=null;
+  String hint = null;
+  String id1 = null;
+  String id2 = null;
 }
 {  
-    
-    <IDENTIFIER> { dataverse = defaultDataverse; funcName = token.image;} ("." <IDENTIFIER> { dataverse = funcName; funcName = token.image;})?
+  funcId = FunctionOrTypeName()
     {
-       hint = getHint(token);
+      dataverse = funcId.first.getValue();
+      funcName = funcId.second.getValue();
+      hint = getHint(token);
     }
-     <LEFTPAREN> (tmp = Expression()
-     {
-       argList.add(tmp);
-       arity ++;
-     } ("," tmp = Expression() { argList.add(tmp); arity++; })*)? <RIGHTPAREN>
-
-     {
-       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;
-     }
+  <LEFTPAREN> (tmp = Expression()
+    {
+      argList.add(tmp);
+      arity ++;
+    }
+  ("," tmp = Expression()
+    {
+      argList.add(tmp);
+      arity++;
+    }
+  )*)? <RIGHTPAREN>
+    {
+      FunctionSignature signature = lookupFunctionSignature(dataverse, funcName, arity);
+      if (signature == null) {
+        signature = new FunctionSignature(dataverse, funcName, arity);
+      }
+      callExpr = new CallExpr(signature,argList);
+      if (hint != null && hint.startsWith(INDEXED_NESTED_LOOP_JOIN_HINT)) {
+        callExpr.addHint(IndexedNLJoinExpressionAnnotation.INSTANCE);
+      }
+      return callExpr;
+    }
 }
 
 Expression DatasetAccessExpression() throws ParseException:
@@ -2088,27 +1717,45 @@
   List<Expression> argList = new ArrayList<Expression>();
   String funcName;
   String dataverse;
+  String arg1 = null;
+  String arg2 = null;
   LiteralExpr ds;
-  LiteralExpr dvds;
   Expression nameArg;
   int arity = 0;
 }
 {  
-    <DATASET> {dataverse = MetadataConstants.METADATA_DATAVERSE_NAME; funcName = getToken(0).toString();}
-    (
-    (<IDENTIFIER> {ds = new LiteralExpr(); ds.setValue( new StringLiteral(token.image) ); argList.add(ds); arity ++;} ("." <IDENTIFIER> { dvds = new LiteralExpr(); dvds.setValue(new StringLiteral(ds.getValue()+"."+token.image)); argList.remove(0); argList.add(dvds);})? ) |  
-    (<LEFTPAREN> nameArg = Expression() {argList.add(nameArg); arity ++;} ("," nameArg = Expression() { argList.add(nameArg); arity++; })* <RIGHTPAREN>)
-    )  
-     
-     {
-       FunctionSignature signature = lookupFunctionSignature(dataverse, funcName.toString(), arity);
-       if(signature == null)
-             {
-                signature = new FunctionSignature(dataverse, funcName.toString(), arity);
-             }
-       callExpr = new CallExpr(signature,argList);
-       return callExpr;
+  <DATASET>
+    {
+      dataverse = MetadataConstants.METADATA_DATAVERSE_NAME;
+      funcName = token.image;
+    }
+  ( ( arg1 = Identifier() ( "." arg2 = Identifier() )? ) 
+    {
+      String name = arg2 == null ? arg1 : arg1 + "." + arg2;
+      ds = new LiteralExpr();
+      ds.setValue( new StringLiteral(name) );
+      argList.add(ds);
+      arity ++;
+    }
+  | ( <LEFTPAREN> nameArg = Expression()
+    {
+      argList.add(nameArg);
+      arity ++;
+    }
+  ( "," nameArg = Expression()
+    {
+      argList.add(nameArg);
+      arity++;
+    }
+  )* <RIGHTPAREN> ) )  
+   {
+     FunctionSignature signature = lookupFunctionSignature(dataverse, funcName, arity);
+     if (signature == null) {
+       signature = new FunctionSignature(dataverse, funcName, arity);
      }
+     callExpr = new CallExpr(signature,argList);
+     return callExpr;
+   }
 }
 
 Expression ParenthesizedExpression() throws ParseException:
@@ -2554,7 +2201,7 @@
 <DEFAULT>
 TOKEN :
 {
-	<VARIABLE : "$" <IDENTIFIER> >
+	<VARIABLE : "$" (<LETTER>)+ (<LETTER> | <DIGIT> | "_")* >
 }
 
 SKIP:
diff --git a/asterix-common/.gitignore b/asterix-common/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/asterix-common/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfo.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfo.java
new file mode 100644
index 0000000..bcb8068
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/AsterixAppContextInfo.java
@@ -0,0 +1,97 @@
+/*
+ * 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.common.api;
+
+import java.util.logging.Logger;
+
+import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
+import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.hyracks.api.application.ICCApplicationContext;
+
+/*
+ * Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
+ * instances that are accessed from the NCs. In addition an instance of ICCApplicationContext 
+ * is stored for access by the CC.
+ */
+public class AsterixAppContextInfo implements IAsterixContextInfo, IAsterixPropertiesProvider {
+
+    private static AsterixAppContextInfo INSTANCE;
+
+    private final ICCApplicationContext appCtx;
+
+    private AsterixCompilerProperties compilerProperties;
+    private AsterixExternalProperties externalProperties;
+    private AsterixMetadataProperties metadataProperties;
+    private AsterixStorageProperties storageProperties;
+    private AsterixTransactionProperties txnProperties;
+
+    public static void initialize(ICCApplicationContext ccAppCtx) throws AsterixException {
+        if (INSTANCE == null) {
+            INSTANCE = new AsterixAppContextInfo(ccAppCtx);
+        }
+        AsterixPropertiesAccessor propertiesAccessor = new AsterixPropertiesAccessor();
+        INSTANCE.compilerProperties = new AsterixCompilerProperties(propertiesAccessor);
+        INSTANCE.externalProperties = new AsterixExternalProperties(propertiesAccessor);
+        INSTANCE.metadataProperties = new AsterixMetadataProperties(propertiesAccessor);
+        INSTANCE.storageProperties = new AsterixStorageProperties(propertiesAccessor);
+        INSTANCE.txnProperties = new AsterixTransactionProperties(propertiesAccessor);
+
+        Logger.getLogger("edu.uci.ics").setLevel(INSTANCE.externalProperties.getLogLevel());
+    }
+
+    private AsterixAppContextInfo(ICCApplicationContext ccAppCtx) {
+        this.appCtx = ccAppCtx;
+    }
+
+    public static AsterixAppContextInfo getInstance() {
+        return INSTANCE;
+    }
+
+    @Override
+    public ICCApplicationContext getCCApplicationContext() {
+        return appCtx;
+    }
+
+    @Override
+    public AsterixStorageProperties getStorageProperties() {
+        return storageProperties;
+    }
+
+    @Override
+    public AsterixTransactionProperties getTransactionProperties() {
+        return txnProperties;
+    }
+
+    @Override
+    public AsterixCompilerProperties getCompilerProperties() {
+        return compilerProperties;
+    }
+
+    @Override
+    public AsterixMetadataProperties getMetadataProperties() {
+        return metadataProperties;
+    }
+
+    @Override
+    public AsterixExternalProperties getExternalProperties() {
+        return externalProperties;
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
index 699346a..7892cc8 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/api/IAsterixAppRuntimeContext.java
@@ -2,7 +2,13 @@
 
 import java.io.IOException;
 
+import edu.uci.ics.asterix.common.config.AsterixCompilerProperties;
+import edu.uci.ics.asterix.common.config.AsterixExternalProperties;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
+import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
@@ -36,10 +42,13 @@
 
     public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory();
 
-    public void initialize() throws IOException, ACIDException;
+    public void initialize() throws IOException, ACIDException, AsterixException;
 
     public void setShuttingdown(boolean b);
 
     public void deinitialize() throws HyracksDataException;
+    
+    public double getBloomFilterFalsePositiveRate();
+    
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AbstractAsterixProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AbstractAsterixProperties.java
new file mode 100644
index 0000000..4968f40
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AbstractAsterixProperties.java
@@ -0,0 +1,9 @@
+package edu.uci.ics.asterix.common.config;
+
+public abstract class AbstractAsterixProperties {
+    protected final AsterixPropertiesAccessor accessor;
+
+    public AbstractAsterixProperties(AsterixPropertiesAccessor accessor) {
+        this.accessor = accessor;
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixCompilerProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixCompilerProperties.java
new file mode 100644
index 0000000..b1767b9
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixCompilerProperties.java
@@ -0,0 +1,31 @@
+package edu.uci.ics.asterix.common.config;
+
+public class AsterixCompilerProperties extends AbstractAsterixProperties {
+    private static final String COMPILER_SORTMEMORY_KEY = "compiler.sortmemory";
+    private static final long COMPILER_SORTMEMORY_DEFAULT = (512 << 20); // 512MB
+
+    private static final String COMPILER_JOINMEMORY_KEY = "compiler.joinmemory";
+    private static final long COMPILER_JOINMEMORY_DEFAULT = (512 << 20); // 512MB
+
+    private static final String COMPILER_FRAMESIZE_KEY = "compiler.framesize";
+    private static int COMPILER_FRAMESIZE_DEFAULT = (32 << 10); // 32KB
+
+    public AsterixCompilerProperties(AsterixPropertiesAccessor accessor) {
+        super(accessor);
+    }
+
+    public long getSortMemorySize() {
+        return accessor.getProperty(COMPILER_SORTMEMORY_KEY, COMPILER_SORTMEMORY_DEFAULT,
+                PropertyInterpreters.getLongPropertyInterpreter());
+    }
+
+    public long getJoinMemorySize() {
+        return accessor.getProperty(COMPILER_JOINMEMORY_KEY, COMPILER_JOINMEMORY_DEFAULT,
+                PropertyInterpreters.getLongPropertyInterpreter());
+    }
+
+    public int getFrameSize() {
+        return accessor.getProperty(COMPILER_FRAMESIZE_KEY, COMPILER_FRAMESIZE_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
new file mode 100644
index 0000000..cf38932
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixExternalProperties.java
@@ -0,0 +1,34 @@
+package edu.uci.ics.asterix.common.config;
+
+import java.util.logging.Level;
+
+public class AsterixExternalProperties extends AbstractAsterixProperties {
+
+    private static final String EXTERNAL_WEBPORT_KEY = "web.port";
+    private static int EXTERNAL_WEBPORT_DEFAULT = 19001;
+
+    private static final String EXTERNAL_LOGLEVEL_KEY = "log.level";
+    private static Level EXTERNAL_LOGLEVEL_DEFAULT = Level.INFO;
+
+    private static final String EXTERNAL_APISERVER_KEY = "api.port";
+    private static int EXTERNAL_APISERVER_DEFAULT = 19101;
+
+    public AsterixExternalProperties(AsterixPropertiesAccessor accessor) {
+        super(accessor);
+    }
+
+    public int getWebInterfacePort() {
+        return accessor.getProperty(EXTERNAL_WEBPORT_KEY, EXTERNAL_WEBPORT_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getAPIServerPort() {
+        return accessor.getProperty(EXTERNAL_APISERVER_KEY, EXTERNAL_APISERVER_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public Level getLogLevel() {
+        return accessor.getProperty(EXTERNAL_LOGLEVEL_KEY, EXTERNAL_LOGLEVEL_DEFAULT,
+                PropertyInterpreters.getLevelPropertyInterpreter());
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
new file mode 100644
index 0000000..6d47e78
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixMetadataProperties.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.asterix.common.config;
+
+import java.util.Map;
+import java.util.Set;
+
+public class AsterixMetadataProperties extends AbstractAsterixProperties {
+
+    public AsterixMetadataProperties(AsterixPropertiesAccessor accessor) {
+        super(accessor);
+    }
+
+    public String getMetadataNodeName() {
+        return accessor.getMetadataNodeName();
+    }
+
+    public String getMetadataStore() {
+        return accessor.getMetadataStore();
+    }
+
+    public Map<String, String[]> getStores() {
+        return accessor.getStores();
+    }
+
+    public Set<String> getNodeNames() {
+        return accessor.getNodeNames();
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixProperties.java
deleted file mode 100644
index 1dc464b..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixProperties.java
+++ /dev/null
@@ -1,252 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package edu.uci.ics.asterix.common.config;
-
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.InputStream;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.Unmarshaller;
-
-import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
-import edu.uci.ics.asterix.common.configuration.Property;
-import edu.uci.ics.asterix.common.configuration.Store;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-
-/**
- * Holder for Asterix properties values typically set as Java Properties.
- * Intended to live in the AsterixStateProxy so it can be accessed remotely.
- */
-public class AsterixProperties implements Serializable {
-
-    private static final long serialVersionUID = 1L;
-    private static String metadataNodeName;
-    private static HashSet<String> nodeNames;
-    private static Map<String, String[]> stores;
-    private static Map<String, String> asterixConfigurationParams;
-
-    public static AsterixProperties INSTANCE = new AsterixProperties();
-
-    public static class AsterixConfigurationKeys {
-
-        // JVM parameters for each Node Contoller (NC)
-        public static final String NC_JAVA_OPTS = "nc_java_opts";
-        public static final String NC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
-
-        // JVM parameters for the Cluster Contoller (CC)
-        public static final String CC_JAVA_OPTS = "cc_java_opts";
-        public static final String CC_JAVA_OPTS_DEFAULT = "-Xmx1024m";
-
-        public static final String SIZE_MEMORY_COMPONENT = "size_memory_component";
-        public static final String SIZE_MEMORY_COMPONENT_DEFAULT = "512m";
-
-        public static final String TOTAL_SIZE_MEMORY_COMPONENT = "total_size_memory_component";
-        public static final String TOTAL_SIZE_MEMORY_COMPONENT_DEFAULT = "512m";
-
-        public static final String LOG_BUFFER_NUM_PAGES = "log_buffer_num_pages";
-        public static final String LOG_BUFFER_NUM_PAGES_DEFAULT = "8";
-
-        public static final String LOG_BUFFER_PAGE_SIZE = "log_buffer_page_size";
-        public static final String LOG_BUFFER_PAGE_SIZE_DEFAULT = "131072";
-
-        public static final String LOG_PARTITION_SIZE = "log_partition_size";
-        public static final String LOG_PARTITION_SIZE_DEFAULT = "2147483648";
-
-        public static final String GROUP_COMMIT_INTERVAL = "group_commit_interval";
-        public static final String GROUP_COMMIT_INTERVAL_DEFAULT = "200ms";
-
-        public static final String SORT_OP_MEMORY = "sort_op_memory";
-        public static final String SORT_OP_MEMORY_DEFAULT = "512m";
-
-        public static final String JOIN_OP_MEMORY = "join_op_memory";
-        public static final String JOIN_OP_MEMORY_DEFAULT = "512m";
-
-        public static final String WEB_INTERFACE_PORT = "web_interface_port";
-        public static final String WEB_INTERFACE_PORT_DEFAULT = "19001";
-
-        public static final String NUM_PAGES_BUFFER_CACHE = "num_pages_buffer_cache";
-        public static final String NUM_PAGES_BUFFER_CACHE_DEFAULT = "1000";
-
-        public static final String LOG_LEVEL = "log_level";
-        public static final String LOG_LEVEL_DEFAULT = "INFO";
-
-        public static final String LSN_THRESHOLD = "lsn_threshold";
-        public static final String LSN_THRESHOLD_DEFAULT = "64m";
-
-        public static final String CHECKPOINT_TERMS_IN_SECS = "checkpoint_terms_in_secs";
-        public static final String CHECKPOINT_TERMS_IN_SECS_DEFAULT = "120";
-
-        public static final String ESCALATE_THRSHOLD_ENTITY_TO_DATASET = "escalate_threshold_entity_to_dataset";
-        public static final String ESCALATE_THRSHOLD_ENTITY_TO_DATASET_DEFAULT = "8";
-
-        public static final String SHRINK_TIMER_THRESHOLD = "shrink_timer_threshold";
-        public static final String SHRINK_TIMER_THRESHOLD_DEFAULT = "120000";
-
-    }
-
-    private AsterixProperties() {
-        try {
-            String fileName = System.getProperty(GlobalConfig.CONFIG_FILE_PROPERTY);
-            if (fileName == null) {
-                fileName = GlobalConfig.DEFAULT_CONFIG_FILE_NAME;
-            }
-            InputStream is = this.getClass().getClassLoader().getResourceAsStream(fileName);
-            if (is == null) {
-                try {
-                    fileName = GlobalConfig.DEFAULT_CONFIG_FILE_NAME;
-                    is = new FileInputStream(fileName);
-                } catch (FileNotFoundException fnf) {
-                    throw new AsterixException("Could not find the configuration file " + fileName);
-                }
-            }
-
-            JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
-            Unmarshaller unmarshaller = ctx.createUnmarshaller();
-            AsterixConfiguration asterixConfiguration = (AsterixConfiguration) unmarshaller.unmarshal(is);
-            metadataNodeName = asterixConfiguration.getMetadataNode();
-            stores = new HashMap<String, String[]>();
-            List<Store> configuredStores = asterixConfiguration.getStore();
-            nodeNames = new HashSet<String>();
-            for (Store store : configuredStores) {
-                String trimmedStoreDirs = store.getStoreDirs().trim();
-                stores.put(store.getNcId(), trimmedStoreDirs.split(","));
-                nodeNames.add(store.getNcId());
-            }
-            asterixConfigurationParams = new HashMap<String, String>();
-            for (Property p : asterixConfiguration.getProperty()) {
-                asterixConfigurationParams.put(p.getName(), p.getValue());
-            }
-
-            initializeLogLevel(getProperty(AsterixConfigurationKeys.LOG_LEVEL));
-        } catch (Exception e) {
-            throw new IllegalStateException(e);
-        }
-    }
-
-    public String getMetadataNodeName() {
-        return metadataNodeName;
-    }
-
-    public String getMetadataStore() {
-        return stores.get(metadataNodeName)[0];
-    }
-
-    public Map<String, String[]> getStores() {
-        return stores;
-    }
-
-    public HashSet<String> getNodeNames() {
-        return nodeNames;
-    }
-
-    public String getProperty(String property) {
-        String propValue = asterixConfigurationParams.get(property);
-        if (propValue == null) {
-            switch (property) {
-                case AsterixConfigurationKeys.NC_JAVA_OPTS:
-                    propValue = AsterixConfigurationKeys.NC_JAVA_OPTS_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.CC_JAVA_OPTS:
-                    propValue = AsterixConfigurationKeys.CC_JAVA_OPTS_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.SIZE_MEMORY_COMPONENT:
-                    propValue = AsterixConfigurationKeys.SIZE_MEMORY_COMPONENT_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.TOTAL_SIZE_MEMORY_COMPONENT:
-                    propValue = AsterixConfigurationKeys.TOTAL_SIZE_MEMORY_COMPONENT_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.LOG_BUFFER_NUM_PAGES:
-                    propValue = AsterixConfigurationKeys.LOG_BUFFER_NUM_PAGES_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.LOG_BUFFER_PAGE_SIZE:
-                    propValue = AsterixConfigurationKeys.LOG_BUFFER_PAGE_SIZE_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.LOG_PARTITION_SIZE:
-                    propValue = AsterixConfigurationKeys.LOG_PARTITION_SIZE_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.GROUP_COMMIT_INTERVAL:
-                    propValue = AsterixConfigurationKeys.GROUP_COMMIT_INTERVAL_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.SORT_OP_MEMORY:
-                    propValue = AsterixConfigurationKeys.SORT_OP_MEMORY_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.JOIN_OP_MEMORY:
-                    propValue = AsterixConfigurationKeys.JOIN_OP_MEMORY_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.WEB_INTERFACE_PORT:
-                    propValue = AsterixConfigurationKeys.WEB_INTERFACE_PORT_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.NUM_PAGES_BUFFER_CACHE:
-                    propValue = AsterixConfigurationKeys.NUM_PAGES_BUFFER_CACHE_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.LOG_LEVEL:
-                    propValue = AsterixConfigurationKeys.LOG_LEVEL_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.LSN_THRESHOLD:
-                    propValue = AsterixConfigurationKeys.LSN_THRESHOLD_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.CHECKPOINT_TERMS_IN_SECS:
-                    propValue = AsterixConfigurationKeys.CHECKPOINT_TERMS_IN_SECS_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.ESCALATE_THRSHOLD_ENTITY_TO_DATASET:
-                    propValue = AsterixConfigurationKeys.ESCALATE_THRSHOLD_ENTITY_TO_DATASET_DEFAULT;
-                    break;
-                case AsterixConfigurationKeys.SHRINK_TIMER_THRESHOLD:
-                    propValue = AsterixConfigurationKeys.SHRINK_TIMER_THRESHOLD_DEFAULT;
-                    break;
-            }
-        }
-        return propValue;
-    }
-
-    private void initializeLogLevel(String configuredLogLevel) {
-        Level level = null;
-        switch (configuredLogLevel.toLowerCase()) {
-            case "info":
-                level = Level.INFO;
-                break;
-            case "fine":
-                level = Level.FINE;
-                break;
-            case "finer":
-                level = Level.FINER;
-                break;
-            case "finest":
-                level = Level.FINEST;
-                break;
-            case "severe":
-                level = Level.SEVERE;
-                break;
-            case "off":
-                level = Level.OFF;
-                break;
-            case "warning":
-                level = Level.WARNING;
-                break;
-            default:
-                level = Level.ALL;
-        }
-        Logger.getLogger(".").setLevel(level);
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
new file mode 100644
index 0000000..7b2f2a6
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixPropertiesAccessor.java
@@ -0,0 +1,105 @@
+package edu.uci.ics.asterix.common.config;
+
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.InputStream;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+
+import edu.uci.ics.asterix.common.configuration.AsterixConfiguration;
+import edu.uci.ics.asterix.common.configuration.Property;
+import edu.uci.ics.asterix.common.configuration.Store;
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+
+public class AsterixPropertiesAccessor {
+    private static final Logger LOGGER = Logger.getLogger(AsterixPropertiesAccessor.class.getName());
+
+    private final String metadataNodeName;
+    private final Set<String> nodeNames;
+    private final Map<String, String[]> stores;
+    private final Map<String, Property> asterixConfigurationParams;
+
+    public AsterixPropertiesAccessor() throws AsterixException {
+        String fileName = System.getProperty(GlobalConfig.CONFIG_FILE_PROPERTY);
+        if (fileName == null) {
+            fileName = GlobalConfig.DEFAULT_CONFIG_FILE_NAME;
+        }
+        InputStream is = this.getClass().getClassLoader().getResourceAsStream(fileName);
+        if (is == null) {
+            try {
+                fileName = GlobalConfig.DEFAULT_CONFIG_FILE_NAME;
+                is = new FileInputStream(fileName);
+            } catch (FileNotFoundException fnf) {
+                throw new AsterixException("Could not find configuration file " + fileName);
+            }
+        }
+
+        AsterixConfiguration asterixConfiguration = null;
+        try {
+            JAXBContext ctx = JAXBContext.newInstance(AsterixConfiguration.class);
+            Unmarshaller unmarshaller = ctx.createUnmarshaller();
+            asterixConfiguration = (AsterixConfiguration) unmarshaller.unmarshal(is);
+        } catch (JAXBException e) {
+            throw new AsterixException("Failed to read configuration file " + fileName);
+        }
+        metadataNodeName = asterixConfiguration.getMetadataNode();
+        stores = new HashMap<String, String[]>();
+        List<Store> configuredStores = asterixConfiguration.getStore();
+        nodeNames = new HashSet<String>();
+        for (Store store : configuredStores) {
+            String trimmedStoreDirs = store.getStoreDirs().trim();
+            stores.put(store.getNcId(), trimmedStoreDirs.split(","));
+            nodeNames.add(store.getNcId());
+        }
+        asterixConfigurationParams = new HashMap<String, Property>();
+        for (Property p : asterixConfiguration.getProperty()) {
+            asterixConfigurationParams.put(p.getName(), p);
+        }
+    }
+
+    public String getMetadataNodeName() {
+        return metadataNodeName;
+    }
+
+    public String getMetadataStore() {
+        return stores.get(metadataNodeName)[0];
+    }
+
+    public Map<String, String[]> getStores() {
+        return stores;
+    }
+
+    public Set<String> getNodeNames() {
+        return nodeNames;
+    }
+
+    public <T> T getProperty(String property, T defaultValue, IPropertyInterpreter<T> interpreter) {
+        Property p = asterixConfigurationParams.get(property);
+        if (p == null) {
+            return defaultValue;
+        }
+
+        try {
+            return interpreter.interpret(p);
+        } catch (IllegalArgumentException e) {
+            logConfigurationError(p, defaultValue);
+            throw e;
+        }
+    }
+
+    private <T> void logConfigurationError(Property p, T defaultValue) {
+        if (LOGGER.isLoggable(Level.SEVERE)) {
+            LOGGER.severe("Invalid property value '" + p.getValue() + "' for property '" + p.getName()
+                    + "'.\n See the description: \n" + p.getDescription() + "\nDefault = " + defaultValue);
+        }
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
new file mode 100644
index 0000000..d34e4ac
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixStorageProperties.java
@@ -0,0 +1,72 @@
+package edu.uci.ics.asterix.common.config;
+
+public class AsterixStorageProperties extends AbstractAsterixProperties {
+
+    private static final String STORAGE_BUFFERCACHE_PAGESIZE_KEY = "storage.buffercache.pagesize";
+    private static int STORAGE_BUFFERCACHE_PAGESIZE_DEFAULT = (32 << 10); // 32KB
+
+    private static final String STORAGE_BUFFERCACHE_NUMPAGES_KEY = "storage.buffercache.numpages";
+    private static final int STORAGE_BUFFERCACHE_NUMPAGES_DEFAULT = 1024;
+
+    private static final String STORAGE_BUFFERCACHE_MAXOPENFILES_KEY = "storage.buffercache.maxopenfiles";
+    private static int STORAGE_BUFFERCACHE_MAXOPENFILES_DEFAULT = Integer.MAX_VALUE;
+
+    private static final String STORAGE_MEMORYCOMPONENT_PAGESIZE_KEY = "storage.memorycomponent.pagesize";
+    private static final int STORAGE_MEMORYCOMPONENT_PAGESIZE_DEFAULT = (32 << 10); // 32KB
+
+    private static final String STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY = "storage.memorycomponent.numpages";
+    private static final int STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT = 4096; // ... so 128MB components
+
+    private static final String STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY = "storage.memorycomponent.globalbudget";
+    private static final long STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT = (1 << 30); // 1GB
+
+    private static final String STORAGE_LSM_MERGETHRESHOLD_KEY = "storage.lsm.mergethreshold";
+    private static int STORAGE_LSM_MERGETHRESHOLD_DEFAULT = 3;
+
+    private static final String STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY = "storage.lsm.bloomfilter.falsepositiverate";
+    private static double STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_DEFAULT = 0.01;
+
+    public AsterixStorageProperties(AsterixPropertiesAccessor accessor) {
+        super(accessor);
+    }
+
+    public int getBufferCachePageSize() {
+        return accessor.getProperty(STORAGE_BUFFERCACHE_PAGESIZE_KEY, STORAGE_BUFFERCACHE_PAGESIZE_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getBufferCacheNumPages() {
+        return accessor.getProperty(STORAGE_BUFFERCACHE_NUMPAGES_KEY, STORAGE_BUFFERCACHE_NUMPAGES_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getBufferCacheMaxOpenFiles() {
+        return accessor.getProperty(STORAGE_BUFFERCACHE_MAXOPENFILES_KEY, STORAGE_BUFFERCACHE_MAXOPENFILES_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getMemoryComponentPageSize() {
+        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_PAGESIZE_KEY, STORAGE_MEMORYCOMPONENT_PAGESIZE_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getMemoryComponentNumPages() {
+        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_NUMPAGES_KEY, STORAGE_MEMORYCOMPONENT_NUMPAGES_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public long getMemoryComponentGlobalBudget() {
+        return accessor.getProperty(STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_KEY,
+                STORAGE_MEMORYCOMPONENT_GLOBALBUDGET_DEFAULT, PropertyInterpreters.getLongPropertyInterpreter());
+    }
+
+    public int getLSMIndexMergeThreshold() {
+        return accessor.getProperty(STORAGE_LSM_MERGETHRESHOLD_KEY, STORAGE_LSM_MERGETHRESHOLD_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public double getBloomFilterFalsePositiveRate() {
+        return accessor.getProperty(STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_KEY,
+                STORAGE_LSM_BLOOMFILTER_FALSEPOSITIVERATE_DEFAULT, PropertyInterpreters.getDoublePropertyInterpreter());
+    }
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
new file mode 100644
index 0000000..d97e53b
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/AsterixTransactionProperties.java
@@ -0,0 +1,72 @@
+package edu.uci.ics.asterix.common.config;
+
+public class AsterixTransactionProperties extends AbstractAsterixProperties {
+    private static final String TXN_LOG_BUFFER_NUMPAGES_KEY = "txn.log.buffer.numpages";
+    private static int TXN_LOG_BUFFER_NUMPAGES_DEFAULT = 8;
+
+    private static final String TXN_LOG_BUFFER_PAGESIZE_KEY = "txn.log.buffer.pagesize";
+    private static final int TXN_LOG_BUFFER_PAGESIZE_DEFAULT = (128 << 10); // 128KB
+
+    private static final String TXN_LOG_PARTITIONSIZE_KEY = "txn.log.partitionsize";
+    private static final long TXN_LOG_PARTITIONSIZE_DEFAULT = (2 << 30); // 2GB
+
+    private static final String TXN_LOG_GROUPCOMMITINTERVAL_KEY = "txn.log.groupcommitinterval";
+    private static int TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT = 200; // 200ms
+
+    private static final String TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY = "txn.log.checkpoint.lsnthreshold";
+    private static final int TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT = (64 << 20); // 64M
+
+    private static final String TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY = "txn.log.checkpoint.pollfrequency";
+    private static int TXN_LOG_CHECKPOINT_POLLFREQUENCY_DEFAULT = 120; // 120s
+
+    private static final String TXN_LOCK_ESCALATIONTHRESHOLD_KEY = "txn.lock.escalationthreshold";
+    private static int TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT = 1000;
+
+    private static final String TXN_LOCK_SHRINKTIMER_KEY = "txn.lock.shrinktimer";
+    private static int TXN_LOCK_SHRINKTIMER_DEFAULT = 120000; // 2m
+
+    public AsterixTransactionProperties(AsterixPropertiesAccessor accessor) {
+        super(accessor);
+    }
+
+    public int getLogBufferNumPages() {
+        return accessor.getProperty(TXN_LOG_BUFFER_NUMPAGES_KEY, TXN_LOG_BUFFER_NUMPAGES_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getLogBufferPageSize() {
+        return accessor.getProperty(TXN_LOG_BUFFER_PAGESIZE_KEY, TXN_LOG_BUFFER_PAGESIZE_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public long getLogPartitionSize() {
+        return accessor.getProperty(TXN_LOG_PARTITIONSIZE_KEY, TXN_LOG_PARTITIONSIZE_DEFAULT,
+                PropertyInterpreters.getLongPropertyInterpreter());
+    }
+
+    public int getGroupCommitInterval() {
+        return accessor.getProperty(TXN_LOG_GROUPCOMMITINTERVAL_KEY, TXN_LOG_GROUPCOMMITINTERVAL_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getCheckpointLSNThreshold() {
+        return accessor.getProperty(TXN_LOG_CHECKPOINT_LSNTHRESHOLD_KEY, TXN_LOG_CHECKPOINT_LSNTHRESHOLD_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getCheckpointPollFrequency() {
+        return accessor.getProperty(TXN_LOG_CHECKPOINT_POLLFREQUENCY_KEY, TXN_LOG_CHECKPOINT_POLLFREQUENCY_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getEntityToDatasetLockEscalationThreshold() {
+        return accessor.getProperty(TXN_LOCK_ESCALATIONTHRESHOLD_KEY, TXN_LOCK_ESCALATIONTHRESHOLD_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+    public int getLockManagerShrinkTimer() {
+        return accessor.getProperty(TXN_LOCK_SHRINKTIMER_KEY, TXN_LOCK_SHRINKTIMER_DEFAULT,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
index ebce6fe..3948e69 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/GlobalConfig.java
@@ -3,61 +3,43 @@
 import java.util.logging.Logger;
 
 public class GlobalConfig {
-	public static final boolean DEBUG = true;
+    public static final boolean DEBUG = true;
 
-	public static final String ASTERIX_LOGGER_NAME = "edu.uci.ics.asterix";
+    public static final String ASTERIX_LOGGER_NAME = "edu.uci.ics.asterix";
 
-	public static final Logger ASTERIX_LOGGER = Logger
-			.getLogger(ASTERIX_LOGGER_NAME);
+    public static final Logger ASTERIX_LOGGER = Logger.getLogger(ASTERIX_LOGGER_NAME);
 
-	public static final String ASTERIX_LOGFILE_PATTERN = "%t/asterix.log";
+    public static final String DEFAULT_CONFIG_FILE_NAME = "asterix-configuration.xml";
 
-	public static final String DEFAULT_CONFIG_FILE_NAME = "asterix-configuration.xml";
+    public static final String CONFIG_FILE_PROPERTY = "AsterixConfigFileName";
 
-	public static final String TEST_CONFIG_FILE_NAME = "src/main/resources/asterix-configuration.xml";
+    public static final String WEB_SERVER_PORT_PROPERTY = "AsterixWebServerPort";
 
-	public static final String CONFIG_FILE_PROPERTY = "AsterixConfigFileName";
+    public static final String JSON_API_SERVER_PORT_PROPERTY = "AsterixJSONAPIServerPort";
 
-	public static final String WEB_SERVER_PORT_PROPERTY = "AsterixWebServerPort";
+    public static final int DEFAULT_FRAME_SIZE = 32768;
 
-	public static final String JSON_API_SERVER_PORT_PROPERTY = "AsterixJSONAPIServerPort";
+    public static final String FRAME_SIZE_PROPERTY = "FrameSize";
 
-	public static final String BUFFER_CACHE_PAGE_SIZE_PROPERTY = "BufferCachePageSize";
+    public static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
 
-	public static final String BUFFER_CACHE_NUM_PAGES_PROPERTY = "BufferCacheNumPages";
+    public static int DEFAULT_INPUT_DATA_COLUMN = 0;
 
-	public static final int DEFAULT_BUFFER_CACHE_NUM_PAGES = 4096;
-
-	public static final int DEFAULT_FRAME_SIZE = 32768;
-
-	public static final String FRAME_SIZE_PROPERTY = "FrameSize";
-
-	public static final float DEFAULT_BTREE_FILL_FACTOR = 1.00f;
-
-	public static int DEFAULT_INPUT_DATA_COLUMN = 0;
-
-	public static int DEFAULT_INDEX_MEM_PAGE_SIZE = 32768;
-
-	public static int DEFAULT_INDEX_MEM_NUM_PAGES = 1000;
-
-	public static int getFrameSize() {
-		int frameSize = GlobalConfig.DEFAULT_FRAME_SIZE;
-		String frameSizeStr = System
-				.getProperty(GlobalConfig.FRAME_SIZE_PROPERTY);
-		if (frameSizeStr != null) {
-			int fz = -1;
-			try {
-				fz = Integer.parseInt(frameSizeStr);
-			} catch (NumberFormatException nfe) {
-				GlobalConfig.ASTERIX_LOGGER
-						.warning("Wrong frame size size argument. Picking default value ("
-								+ GlobalConfig.DEFAULT_FRAME_SIZE
-								+ ") instead.\n");
-			}
-			if (fz >= 0) {
-				frameSize = fz;
-			}
-		}
-		return frameSize;
-	}
+    public static int getFrameSize() {
+        int frameSize = GlobalConfig.DEFAULT_FRAME_SIZE;
+        String frameSizeStr = System.getProperty(GlobalConfig.FRAME_SIZE_PROPERTY);
+        if (frameSizeStr != null) {
+            int fz = -1;
+            try {
+                fz = Integer.parseInt(frameSizeStr);
+            } catch (NumberFormatException nfe) {
+                GlobalConfig.ASTERIX_LOGGER.warning("Wrong frame size size argument. Picking default value ("
+                        + GlobalConfig.DEFAULT_FRAME_SIZE + ") instead.\n");
+            }
+            if (fz >= 0) {
+                frameSize = fz;
+            }
+        }
+        return frameSize;
+    }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IAsterixPropertiesProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IAsterixPropertiesProvider.java
new file mode 100644
index 0000000..e42c827
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IAsterixPropertiesProvider.java
@@ -0,0 +1,14 @@
+package edu.uci.ics.asterix.common.config;
+
+
+public interface IAsterixPropertiesProvider {
+    public AsterixStorageProperties getStorageProperties();
+
+    public AsterixTransactionProperties getTransactionProperties();
+
+    public AsterixCompilerProperties getCompilerProperties();
+
+    public AsterixMetadataProperties getMetadataProperties();
+
+    public AsterixExternalProperties getExternalProperties();
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IPropertyInterpreter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IPropertyInterpreter.java
new file mode 100644
index 0000000..2cabbe5
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/IPropertyInterpreter.java
@@ -0,0 +1,7 @@
+package edu.uci.ics.asterix.common.config;
+
+import edu.uci.ics.asterix.common.configuration.Property;
+
+public interface IPropertyInterpreter<T> {
+    public T interpret(Property p) throws IllegalArgumentException;
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/PropertyInterpreters.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/PropertyInterpreters.java
new file mode 100644
index 0000000..9ff5dee
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/config/PropertyInterpreters.java
@@ -0,0 +1,71 @@
+package edu.uci.ics.asterix.common.config;
+
+import java.util.logging.Level;
+
+import edu.uci.ics.asterix.common.configuration.Property;
+
+public class PropertyInterpreters {
+
+    public static IPropertyInterpreter<Integer> getIntegerPropertyInterpreter() {
+        return new IPropertyInterpreter<Integer>() {
+
+            @Override
+            public Integer interpret(Property p) throws IllegalArgumentException {
+                try {
+                    return Integer.parseInt(p.getValue());
+                } catch (NumberFormatException e) {
+                    throw new IllegalArgumentException(e);
+                }
+            }
+        };
+    }
+
+    public static IPropertyInterpreter<Long> getLongPropertyInterpreter() {
+        return new IPropertyInterpreter<Long>() {
+
+            @Override
+            public Long interpret(Property p) throws IllegalArgumentException {
+                try {
+                    return Long.parseLong(p.getValue());
+                } catch (NumberFormatException e) {
+                    throw new IllegalArgumentException(e);
+                }
+            }
+        };
+    }
+
+    public static IPropertyInterpreter<Level> getLevelPropertyInterpreter() {
+        return new IPropertyInterpreter<Level>() {
+
+            @Override
+            public Level interpret(Property p) throws IllegalArgumentException {
+                return Level.parse(p.getValue());
+            }
+        };
+    }
+
+    public static IPropertyInterpreter<String> getStringPropertyInterpreter() {
+        return new IPropertyInterpreter<String>() {
+
+            @Override
+            public String interpret(Property p) throws IllegalArgumentException {
+                return p.getValue();
+            }
+        };
+    }
+
+    public static IPropertyInterpreter<Double> getDoublePropertyInterpreter() {
+        return new IPropertyInterpreter<Double>() {
+
+            @Override
+            public Double interpret(Property p) throws IllegalArgumentException {
+                try {
+                    return Double.parseDouble(p.getValue());
+                } catch (NumberFormatException e) {
+                    throw new IllegalArgumentException(e);
+                }
+            }
+        };
+    }
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java
index 188593c..563b2cf 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/functions/FunctionSignature.java
@@ -5,10 +5,10 @@
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 
 public class FunctionSignature implements Serializable {
-    private final String namespace;
-    private final String name;
-    private final int arity;
-    private final String rep;
+    private  String namespace;
+    private  String name;
+    private  int arity;
+    private  String rep;
 
     public FunctionSignature(String namespace, String name, int arity) {
         this.namespace = namespace;
@@ -50,4 +50,16 @@
         return arity;
     }
 
+    public void setNamespace(String namespace) {
+        this.namespace = namespace;
+    }
+
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    public void setArity(int arity) {
+        this.arity = arity;
+    }
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
index 8c46e73..0e80817 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
@@ -19,6 +19,9 @@
 import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 /**
  * Represent a buffer that is backed by a physical file. Provider custom APIs
@@ -27,22 +30,32 @@
 public class FileBasedBuffer extends Buffer implements IFileBasedBuffer {
 
     private String filePath;
-    private long nextWritePosition;
     private FileChannel fileChannel;
     private RandomAccessFile raf;
-    private int size;
+    private int bufferSize;
 
-    public FileBasedBuffer(String filePath, long offset, int size) throws IOException {
+    private int bufferLastFlushOffset;
+    private int bufferNextWriteOffset;
+    private final int diskSectorSize;
+
+    private final ReadWriteLock latch;
+    private final AtomicInteger referenceCount;
+
+    public FileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
         this.filePath = filePath;
-        this.nextWritePosition = offset;
-        buffer = ByteBuffer.allocate(size);
+        buffer = ByteBuffer.allocate(bufferSize);
         raf = new RandomAccessFile(new File(filePath), "rw");
-        raf.seek(offset);
         fileChannel = raf.getChannel();
+        fileChannel.position(offset);
         fileChannel.read(buffer);
         buffer.position(0);
-        this.size = size;
-        buffer.limit(size);
+        this.bufferSize = bufferSize;
+        buffer.limit(bufferSize);
+        bufferLastFlushOffset = 0;
+        bufferNextWriteOffset = 0;
+        this.diskSectorSize = diskSectorSize;
+        latch = new ReentrantReadWriteLock(true);
+        referenceCount = new AtomicInteger(0);
     }
 
     public String getFilePath() {
@@ -53,17 +66,9 @@
         this.filePath = filePath;
     }
 
-    public long getOffset() {
-        return nextWritePosition;
-    }
-
-    public void setOffset(long offset) {
-        this.nextWritePosition = offset;
-    }
-
     @Override
     public int getSize() {
-        return buffer.limit();
+        return bufferSize;
     }
 
     public void clear() {
@@ -72,11 +77,18 @@
 
     @Override
     public void flush() throws IOException {
-        buffer.position(0);
-        buffer.limit(size);
+        //flush
+        int pos = bufferLastFlushOffset;
+        int limit = (((bufferNextWriteOffset - 1) / diskSectorSize) + 1) * diskSectorSize;
+        buffer.position(pos);
+        buffer.limit(limit);
         fileChannel.write(buffer);
         fileChannel.force(true);
-        erase();
+
+        //update variables
+        bufferLastFlushOffset = limit;
+        bufferNextWriteOffset = limit;
+        buffer.limit(bufferSize);
     }
 
     @Override
@@ -124,45 +136,110 @@
      * starting at offset.
      */
     @Override
-    public void reset(String filePath, long nextWritePosition, int size) throws IOException {
+    public void reset(String filePath, long diskNextWriteOffset, int bufferSize) throws IOException {
         if (!filePath.equals(this.filePath)) {
             raf.close();//required?
             fileChannel.close();
             raf = new RandomAccessFile(filePath, "rw");
             this.filePath = filePath;
         }
-        this.nextWritePosition = nextWritePosition;
-        raf.seek(nextWritePosition);
         fileChannel = raf.getChannel();
+        fileChannel.position(diskNextWriteOffset);
         erase();
         buffer.position(0);
-        buffer.limit(size);
-        this.size = size;
+        buffer.limit(bufferSize);
+        this.bufferSize = bufferSize;
+
+        bufferLastFlushOffset = 0;
+        bufferNextWriteOffset = 0;
     }
-    
+
     @Override
     public void close() throws IOException {
         fileChannel.close();
     }
-    
+
     @Override
-    public void open(String filePath, long offset, int size) throws IOException {
+    public void open(String filePath, long offset, int bufferSize) throws IOException {
         raf = new RandomAccessFile(filePath, "rw");
-        this.nextWritePosition = offset;
         fileChannel = raf.getChannel();
         fileChannel.position(offset);
         erase();
         buffer.position(0);
-        buffer.limit(size);
-        this.size = size;
+        buffer.limit(bufferSize);
+        this.bufferSize = bufferSize;
+        bufferLastFlushOffset = 0;
+        bufferNextWriteOffset = 0;
     }
 
-    public long getNextWritePosition() {
-        return nextWritePosition;
+    @Override
+    public long getDiskNextWriteOffset() throws IOException {
+        return fileChannel.position();
     }
 
-    public void setNextWritePosition(long nextWritePosition) {
-        this.nextWritePosition = nextWritePosition;
+    @Override
+    public void setDiskNextWriteOffset(long offset) throws IOException {
+        fileChannel.position(offset);
     }
 
+    @Override
+    public int getBufferLastFlushOffset() {
+        return bufferLastFlushOffset;
+    }
+
+    @Override
+    public void setBufferLastFlushOffset(int offset) {
+        this.bufferLastFlushOffset = offset;
+    }
+
+    @Override
+    public int getBufferNextWriteOffset() {
+        synchronized (fileChannel) {
+            return bufferNextWriteOffset;
+        }
+    }
+
+    @Override
+    public void setBufferNextWriteOffset(int offset) {
+        synchronized (fileChannel) {
+            if (bufferNextWriteOffset < offset) {
+                bufferNextWriteOffset = offset;
+            }
+        }
+    }
+
+    @Override
+    public void acquireWriteLatch() {
+        latch.writeLock().lock();
+    }
+
+    @Override
+    public void releaseWriteLatch() {
+        latch.writeLock().unlock();
+    }
+
+    @Override
+    public void acquireReadLatch() {
+        latch.readLock().lock();
+    }
+
+    @Override
+    public void releaseReadLatch() {
+        latch.readLock().unlock();
+    }
+
+    @Override
+    public void incRefCnt() {
+        referenceCount.incrementAndGet();
+    }
+    
+    @Override
+    public void decRefCnt() {
+        referenceCount.decrementAndGet();
+    }
+    
+    @Override
+    public int getRefCnt() {
+        return referenceCount.get();
+    }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
index 238f018..2ba0a08 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
@@ -38,8 +38,8 @@
         return (new File(path)).mkdir();
     }
 
-    public static IFileBasedBuffer getFileBasedBuffer(String filePath, long offset, int size) throws IOException {
-        IFileBasedBuffer fileBasedBuffer = new FileBasedBuffer(filePath, offset, size);
+    public static IFileBasedBuffer getFileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
+        IFileBasedBuffer fileBasedBuffer = new FileBasedBuffer(filePath, offset, bufferSize, diskSectorSize);
         return fileBasedBuffer;
     }
 
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
index 7321062..f762b16 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IAsterixAppRuntimeContextProvider.java
@@ -26,24 +26,24 @@
     public ILSMMergePolicy getLSMMergePolicy();
 
     public ILSMOperationTrackerFactory getLSMBTreeOperationTrackerFactory();
-    
+
     public ILSMOperationTrackerFactory getLSMRTreeOperationTrackerFactory();
-    
+
     public ILSMOperationTrackerFactory getLSMInvertedIndexOperationTrackerFactory();
-    
+
     public ILSMIOOperationCallbackProvider getLSMBTreeIOOperationCallbackProvider();
-    
+
     public ILSMIOOperationCallbackProvider getLSMRTreeIOOperationCallbackProvider();
-    
+
     public ILSMIOOperationCallbackProvider getLSMInvertedIndexIOOperationCallbackProvider();
-    
+
     public ILSMIOOperationCallbackProvider getNoOpIOOperationCallbackProvider();
-    
+
     public ILSMIOOperationScheduler getLSMIOScheduler();
 
     public ILocalResourceRepository getLocalResourceRepository();
 
     public ResourceIdFactory getResourceIdFactory();
-    
+
     public IIOManager getIOManager();
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
index 76199a3..5093ca0 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
@@ -31,12 +31,34 @@
      */
     public void reset(String filePath, long offset, int size) throws IOException;
 
-    public long getNextWritePosition();
+    public long getDiskNextWriteOffset() throws IOException;
 
-    public void setNextWritePosition(long writePosition);
+    public void setDiskNextWriteOffset(long writePosition) throws IOException;
 
     public void close() throws IOException;
     
     public void open(String filePath, long offset, int size) throws IOException;
 
+    public int getBufferLastFlushOffset();
+
+    public void setBufferLastFlushOffset(int offset);
+
+    public int getBufferNextWriteOffset();
+
+    public void setBufferNextWriteOffset(int offset);
+    
+    public void acquireWriteLatch();
+
+    public void releaseWriteLatch();
+
+    public void acquireReadLatch();
+
+    public void releaseReadLatch();
+    
+    public void incRefCnt();
+    
+    public void decRefCnt();
+    
+    public int getRefCnt();
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index d02c508..9470d17 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -50,17 +50,8 @@
             ACIDException;
 
     /**
-     * Provides a cursor for retrieving logs that satisfy a given ILogFilter
-     * instance. Log records are retrieved in increasing order of lsn
-     * 
-     * @param logFilter
-     *            specifies the filtering criteria for the retrieved logs
-     * @return LogCursor an iterator for the retrieved logs
-     * @throws ACIDException
-     */
-    public ILogCursor readLog(ILogFilter logFilter) throws ACIDException;
-
-    /**
+     * @param logicalLogLocator
+     *            TODO
      * @param logicalLogLocator
      *            TODO
      * @param PhysicalLogLocator
@@ -70,15 +61,6 @@
     public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
 
     /**
-     * Flushes the log records up to the lsn represented by the
-     * logicalLogLocator
-     * 
-     * @param logicalLogLocator
-     * @throws ACIDException
-     */
-    public void flushLog(LogicalLogLocator logicalLogLocator) throws ACIDException;
-
-    /**
      * Retrieves the configuration parameters of the ILogManager
      * 
      * @return LogManagerProperties: the configuration parameters for the
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
index 6e851e8..13fc722 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
@@ -23,41 +23,44 @@
 
 public interface ILogRecordHelper {
 
-    byte getLogType(LogicalLogLocator logicalLogLocator);
+    public byte getLogType(LogicalLogLocator logicalLogLocator);
 
-    int getJobId(LogicalLogLocator logicalLogLocator);
+    public int getJobId(LogicalLogLocator logicalLogLocator);
 
-    int getDatasetId(LogicalLogLocator logicalLogLocator);
+    public int getDatasetId(LogicalLogLocator logicalLogLocator);
 
-    int getPKHashValue(LogicalLogLocator logicalLogLocator);
+    public int getPKHashValue(LogicalLogLocator logicalLogLocator);
 
-    PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
+    public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
 
-    boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
+    public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
 
-    long getResourceId(LogicalLogLocator logicalLogLocator);
+    public long getResourceId(LogicalLogLocator logicalLogLocator);
 
-    byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
+    public byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
 
-    int getLogContentSize(LogicalLogLocator logicalLogLocater);
+    public int getLogContentSize(LogicalLogLocator logicalLogLocater);
 
-    long getLogChecksum(LogicalLogLocator logicalLogLocator);
+    public long getLogChecksum(LogicalLogLocator logicalLogLocator);
 
-    int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
+    public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
 
-    int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
+    public int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
 
-    String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
+    public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
 
-    void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context, int datasetId,
-            int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId, int logRecordSize);
+    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
+            int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
+            int logRecordSize);
 
-    boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
+    public boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
 
-    int getLogRecordSize(byte logType, int logBodySize);
+    public int getLogRecordSize(byte logType, int logBodySize);
 
-    int getLogHeaderSize(byte logType);
+    public int getLogHeaderSize(byte logType);
 
-    int getLogChecksumSize();
+    public int getLogChecksumSize();
 
-}
+    public int getCommitLogSize();
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
index fa02e31..4dc943c 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
@@ -28,6 +28,7 @@
     public static final String NUM_LOG_PAGES_KEY = "num_log_pages";
     public static final String LOG_FILE_PREFIX_KEY = "log_file_prefix";
     public static final String GROUP_COMMIT_WAIT_PERIOD_KEY = "group_commit_wait_period";
+    public static final String DISK_SECTOR_SIZE_KEY = "disk_sector_size";
 
     private static final int DEFAULT_LOG_PAGE_SIZE = 128 * 1024; //128KB
     private static final int DEFAULT_NUM_LOG_PAGES = 8;
@@ -35,6 +36,7 @@
     private static final long DEFAULT_GROUP_COMMIT_WAIT_PERIOD = 200; // time in millisec.
     private static final String DEFAULT_LOG_FILE_PREFIX = "asterix_transaction_log";
     private static final String DEFAULT_LOG_DIRECTORY = "asterix_logs/";
+    private static final int DEFAULT_DISK_SECTOR_SIZE = 4096;
 
     // follow the naming convention <logFilePrefix>_<number> where number starts from 0
     private final String logFilePrefix;
@@ -51,6 +53,8 @@
     private final int logBufferSize;
     // maximum size of each log file
     private final long logPartitionSize;
+    // default disk sector size
+    private final int diskSectorSize;
 
     public LogManagerProperties(Properties properties, String nodeId) {
         this.logDirKey = new String(nodeId + LOG_DIR_SUFFIX_KEY);
@@ -66,6 +70,8 @@
         this.logBufferSize = logPageSize * numLogPages;
         //make sure that the log partition size is the multiple of log buffer size.
         this.logPartitionSize = (logPartitionSize / logBufferSize) * logBufferSize;
+        this.diskSectorSize = Integer.parseInt(properties.getProperty(DISK_SECTOR_SIZE_KEY, ""
+                + DEFAULT_DISK_SECTOR_SIZE));
     }
 
     public long getLogPartitionSize() {
@@ -99,6 +105,10 @@
     public String getLogDirKey() {
         return logDirKey;
     }
+    
+    public int getDiskSectorSize() {
+        return diskSectorSize;
+    }
 
     public String toString() {
         StringBuilder builder = new StringBuilder();
@@ -108,6 +118,7 @@
         builder.append("num_log_pages : " + numLogPages + FileUtil.lineSeparator);
         builder.append("log_partition_size : " + logPartitionSize + FileUtil.lineSeparator);
         builder.append("group_commit_wait_period : " + groupCommitWaitPeriod + FileUtil.lineSeparator);
+        builder.append("disk_sector_size : " + diskSectorSize + FileUtil.lineSeparator);
         return builder.toString();
     }
 }
diff --git a/asterix-doc/pom.xml b/asterix-doc/pom.xml
new file mode 100644
index 0000000..d987e5f
--- /dev/null
+++ b/asterix-doc/pom.xml
@@ -0,0 +1,21 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+	<modelVersion>4.0.0</modelVersion>
+	<parent>
+		<artifactId>asterix</artifactId>
+		<groupId>edu.uci.ics.asterix</groupId>
+		<version>0.0.6-SNAPSHOT</version>
+	</parent>
+	<artifactId>asterix-doc</artifactId>
+	<build>
+		<plugins>
+			<plugin>
+				<groupId>org.apache.maven.plugins</groupId>
+				<artifactId>maven-site-plugin</artifactId>
+				<version>3.3</version>
+				<configuration>
+				  <generateReports>false</generateReports>
+				</configuration>
+			</plugin>
+		</plugins>
+	</build>
+</project>
diff --git a/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md b/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md
new file mode 100644
index 0000000..7e49a0f
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md
@@ -0,0 +1,199 @@
+# Accessing External Data in AsterixDB #
+
+## Introduction ##
+Data that needs to be processed by ASTERIX could be residing outside ASTERIX storage. Examples include data files on a distributed file system such as HDFS or on the local file system of a machine that is part of an ASTERIX cluster.  For ASTERIX to process such data, end-user may create a regular dataset in ASTERIX (a.k.a. internal dataset) and load the dataset with the data. ASTERIX supports ''external datasets'' so that it is not necessary to “load” all data prior to using it. This also avoids creating multiple copies of data and the need to keep the copies in sync.
+
+### Adapter for an External Dataset ###
+External data is accessed using wrappers (adapters in ASTERIX) that abstract away the mechanism of connecting with an external service, receiving data and transforming the data into ADM records that are understood by ASTERIX. ASTERIX comes with built-in adapters for common storage systems such as HDFS or the local file system.
+
+### Creating an External Dataset ###
+
+As an example we consider the Lineitem dataset from [TPCH schema](http://www.openlinksw.com/dataspace/doc/dav/wiki/Main/VOSTPCHLinkedData/tpch.sql).
+
+We assume that you have successfully created an ASTERIX instance following the instructions at [Installing Asterix Using Managix](InstallingAsterixUsingManagix.html).
+_For constructing an example, we assume a single machine setup._
+
+Similar to a regular dataset, an external dataset has an associated datatype.  We shall first create the datatype associated with each record in Lineitem data.
+Paste the following in the query textbox on the webpage at http://127.0.0.1 and hit 'Execute'.
+
+
+        create dataverse ExternalFileDemo;
+        use dataverse ExternalFileDemo;
+        
+        create type LineitemType as closed {
+          l_orderkey:int32,
+          l_partkey: int32,
+          l_suppkey: int32,
+          l_linenumber: int32,
+          l_quantity: double,
+          l_extendedprice: double,
+          l_discount: double,
+          l_tax: double,
+          l_returnflag: string,
+          l_linestatus: string,
+          l_shipdate: string,
+          l_commitdate: string,
+          l_receiptdate: string,
+          l_shipinstruct: string,
+          l_shipmode: string,
+          l_comment: string}
+
+
+We describe here two scenarios.
+
+#### 1) Data file resides on the local file system of a host####
+Prerequisite: The host is a  part of the ASTERIX cluster.
+
+Earlier, we assumed a single machine ASTERIX setup. To satisfy the prerequisite, log-in to the machine running ASTERIX.
+
+ * Download the [data file](https://code.google.com/p/asterixdb/downloads/detail?name=lineitem.tbl&amp;can=2&amp;q=) to an appropriate location. We denote this location by SOURCE_PATH.
+
+ASTERIX provides a built-in adapter for data residing on the local file system. The adapter is referred by its alias- 'localfs'. We create an external dataset named Lineitem and use the 'localfs' adapter.
+
+
+        create external dataset Lineitem(LineitemType)
+        using localfs
+
+Above, the definition is not complete as we need to provide a set of parameters that are specific to the source file.
+
+<table>
+<tr>
+  <td> Parameter </td>
+  <td> Description </td>
+</tr>
+<tr>
+  <td> path </td>
+  <td> A fully qualified path of the form <tt>host://&lt;absolute path&gt;</tt>. 
+  Use a comma separated list if there are multiple files. 
+  E.g. <tt>host1://&lt;absolute path&gt;</tt>, <tt>host2://&lt;absolute path&gt;</tt> and so forth. </td>
+</tr>
+<tr>
+  <td> format </td>
+  <td> The format for the content. Use 'adm' for data in ADM (ASTERIX Data Model) or <a href="http://www.json.org/">JSON</a> format. Use 'delimited-text' if fields are separted by . </td></tr>
+<tr><td>delimiter</td><td>The delimiting character in the source file if format is 'delimited text'</td></tr>
+</table>
+
+As we are using a single single machine ASTERIX instance, we use 127.0.0.1 as host in the path parameter.
+We *complete the create dataset statement* as follows.
+
+
+        use dataverse ExternalFileDemo;
+        
+        create external dataset Lineitem(LineitemType)
+        using localfs
+        (("path"="127.0.0.1://SOURCE_PATH"),
+        ("format"="delimited-text"),
+        ("delimiter"="|"));
+
+
+Please substitute SOURCE_PATH with the absolute path to the source file on the local file system.
+
+#### Common source of error ####
+
+An incorrect value for the path parameter will give the following exception message when the dataset is used in a query.
+
+        edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException: edu.uci.ics.hyracks.api.exceptions.HyracksDataException: edu.uci.ics.hyracks.api.exceptions.HyracksDataException: Job failed.
+
+
+Verify the correctness of the path parameter provided to the localfs adapter. Note that the path parameter must be an absolute path to the data file. For e.g. if you saved your file in your home directory (assume it to be /home/joe), then the path value should be
+
+        127.0.0.1:///home/joe/lineitem.tbl.
+
+
+In your web-browser, navigate to 127.0.0.1 and paste the above to the query text box. Finally hit 'Execute'.
+
+Next we move over to the the section [Writing Queries against an External Dataset](#Writing_Queries_against_an_External_Dataset) and try a sample query against the external dataset.
+
+#### 2) Data file resides on an HDFS instance ####
+Pre-requisite: It is required that the Namenode and atleast one of the HDFS Datanodes are reachable from the hosts that form the ASTERIX cluster.  ASTERIX provides a built-in adapter for data residing on HDFS. The HDFS adapter is referred (in AQL) by its alias - 'hdfs'. We create an external dataset named Lineitem and associate the HDFS adapter with it.
+
+
+        create external dataset Lineitem(LineitemType)
+        using hdfs
+
+
+The above statement is *not complete* as we need to provide a set of parameters specific to the HDFS instance and the source file.
+These parameters are described below.
+
+<table>
+<tr>
+  <td> Parameter </td>
+  <td> Description </td>
+</tr>
+<tr>
+  <td> hdfs </td>
+  <td> The HDFS URL </td>
+</tr>
+<tr>
+  <td> path </td>
+  <td> The absolute path to the source HDFS file. Use a comma separated list if there are multiple files. </td></tr>
+<tr>
+  <td> input-format </td>
+  <td> The associated input format. Use 'text-input-format' for textual data or 'sequence-input-format' for binary data (sequence files). </td>
+</tr>
+<tr>
+  <td> format </td>
+  <td> The format for the content. Use 'adm' for data in ADM (ASTERIX Data Model) or 
+  <a href="http://www.json.org/">JSON</a> format and use 'delimited-text' for delimited data 
+  that has fields separated by a delimiting character. </td>
+</tr>
+<tr>
+  <td> delimiter </td>
+  <td> The delimiting character in the source file if format is 'delimited text' </td>
+</tr>
+</table>
+
+*Difference between 'input-format' and 'format'*
+
+*input-format*: File stored under HDFS have an associated storage format  For example, TextInputFormat represents plain text files.  SequenceFileInputFormat indicates binary compressed file. The parameter 'input-format' is used to distinguish between these two kind of files.
+
+*format*:
+The parameter 'format' refers to the type of the data contained in the file. For example data contained in a file could be in json, ADM format or could be delimited-text with fields separated by a delimiting character.
+
+As an example. consider the [data file](https://code.google.com/p/asterixdb/downloads/detail?name=lineitem.tbl&amp;can=2&amp;q=).  The file is a text file with each line representing a record. The fields in each record are separated by the '|' character.
+
+We assume the HDFS URL to be hdfs://host:port. We further assume that the example data file is copied to the HDFS at a path denoted by HDFS_PATH.
+
+The complete set of parameters for our example file are as follows. (("hdfs"="HDFS_URL",("path"="HDFS_PATH"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"))
+
+We modify the create external dataset statement as follows.
+
+
+        create external dataset Lineitem('LineitemType)
+        using hdfs
+        (("hdfs"="HDFS_URL"),("path"="HDFS_PATH"),("input-format"="text-input-format"),("format"="delimited-text"),("delimiter"="|"));
+
+
+Once you have copied the source data file to your HDFS instance, substitute the values of HDFS_URL and HDFS_PATH in the above statement. In your web-browser, navigate to http://127.0.0.1:19001 and execute the above statement with substituted values.
+
+You may now run the sample query in next section.
+
+## Writing Queries against an External Dataset ##
+You may  write AQL queries against an external dataset. Following is an example AQL query that applies a filter and returns an ordered result.
+
+
+        use dataverse ExternalFileDemo;
+        
+        for $c in dataset('Lineitem')
+        where $c.l_orderkey <= 3
+        order by $c.l_orderkey, $c.l_linenumber
+        return $c
+
+
+The expected result is:
+
+
+        { "l_orderkey": 1, "l_partkey": 156, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 17, "l_extendedprice": 17954.55d, "l_discount": 0.04d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-13", "l_commitdate": "1996-02-12", "l_receiptdate": "1996-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "egular courts above the" }
+        { "l_orderkey": 1, "l_partkey": 68, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36, "l_extendedprice": 34850.16d, "l_discount": 0.09d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-12", "l_commitdate": "1996-02-28", "l_receiptdate": "1996-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ly final dependencies: slyly bold " }
+        { "l_orderkey": 1, "l_partkey": 64, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 8, "l_extendedprice": 7712.48d, "l_discount": 0.1d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-29", "l_commitdate": "1996-03-05", "l_receiptdate": "1996-01-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "riously. regular, express dep" }
+        { "l_orderkey": 1, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28, "l_extendedprice": 25284.0d, "l_discount": 0.09d, "l_tax": 0.06d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lites. fluffily even de" }
+        { "l_orderkey": 1, "l_partkey": 25, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 24, "l_extendedprice": 22200.48d, "l_discount": 0.1d, "l_tax": 0.04d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-04-01", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " pending foxes. slyly re" }
+        { "l_orderkey": 1, "l_partkey": 16, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 32, "l_extendedprice": 29312.32d, "l_discount": 0.07d, "l_tax": 0.02d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-30", "l_commitdate": "1996-02-07", "l_receiptdate": "1996-02-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "arefully slyly ex" }
+        { "l_orderkey": 2, "l_partkey": 107, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 38, "l_extendedprice": 38269.8d, "l_discount": 0.0d, "l_tax": 0.05d, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-28", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-02-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ven requests. deposits breach a" }
+        { "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45, "l_extendedprice": 40725.0d, "l_discount": 0.06d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
+        { "l_orderkey": 3, "l_partkey": 20, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 49, "l_extendedprice": 45080.98d, "l_discount": 0.1d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-09", "l_commitdate": "1993-12-20", "l_receiptdate": "1993-11-24", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " unusual accounts. eve" }
+        { "l_orderkey": 3, "l_partkey": 129, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 27, "l_extendedprice": 27786.24d, "l_discount": 0.06d, "l_tax": 0.07d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-16", "l_commitdate": "1993-11-22", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "nal foxes wake. " }
+        { "l_orderkey": 3, "l_partkey": 30, "l_suppkey": 5, "l_linenumber": 4, "l_quantity": 2, "l_extendedprice": 1860.06d, "l_discount": 0.01d, "l_tax": 0.06d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-04", "l_commitdate": "1994-01-07", "l_receiptdate": "1994-01-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y. fluffily pending d" }
+        { "l_orderkey": 3, "l_partkey": 184, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 28, "l_extendedprice": 30357.04d, "l_discount": 0.04d, "l_tax": 0.0d, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-14", "l_commitdate": "1994-01-10", "l_receiptdate": "1994-01-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ages nag slyly pending" }
+        { "l_orderkey": 3, "l_partkey": 63, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 26, "l_extendedprice": 25039.56d, "l_discount": 0.1d, "l_tax": 0.02d, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-11-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "ges sleep after the caref" }
+
diff --git a/asterix-doc/src/site/markdown/AdmAql101.md b/asterix-doc/src/site/markdown/AdmAql101.md
new file mode 100644
index 0000000..ed4736e
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AdmAql101.md
@@ -0,0 +1,897 @@
+# AsterixDB 101: An ADM and AQL Primer #
+
+## Welcome to AsterixDB! ##
+This document introduces the main features of AsterixDB's data model (ADM) and query language (AQL) by example.
+The example is a simple scenario involving (synthetic) sample data modeled after data from the social domain.
+This document describes a set of sample ADM datasets, together with a set of illustrative AQL queries,
+to introduce you to the "AsterixDB user experience".
+The complete set of steps required to create and load a handful of sample datasets, along with runnable queries
+and the expected results for each query, are included.
+
+This document assumes that you are at least vaguely familiar with AsterixDB and why you might want to use it.
+Most importantly, it assumes you already have a running instance of AsterixDB and that you know how to query
+it using AsterixDB's basic web interface.
+For more information on these topics, you should go through the steps in 
+[Installing Asterix Using Managix](InstallingAsterixUsingManagix.html)
+before reading this document and make sure that you have a running AsterixDB instance ready to go.
+To get your feet wet, you should probably start with a simple local installation of AsterixDB on your favorite
+machine, accepting all of the default settings that Managix offers.
+Later you can graduate to trying AsterixDB on a cluster, its real intended home (since it targets Big Data).
+(Note: With the exception of specifying the correct locations where you put the source data for this example,
+there should no changes needed in your ADM or AQL statements to run the examples locally and/or to run them
+on a cluster when you are ready to take that step.)
+
+As you read through this document, you should try each step for yourself on your own AsterixDB instance.
+Once you have reached the end, you will be fully armed and dangerous, with all the basic AsterixDB knowledge
+that you'll need to start down the path of modeling, storing, and querying your own semistructured data.
+
+----
+## ADM: Modeling Semistructed Data in AsterixDB ##
+In this section you will learn all about modeling Big Data using
+ADM, the data model of the AsterixDB BDMS.
+
+### Dataverses, Datatypes, and Datasets ###
+The top-level organizing concept in the AsterixDB world is the _dataverse_.
+A dataverse---short for "data universe"---is a place (similar to a database in a relational DBMS) in which
+to create and manage the types, datasets, functions, and other artifacts for a given AsterixDB application.
+When you start using an AsterixDB instance for the first time, it starts out "empty"; it contains no data
+other than the AsterixDB system catalogs (which live in a special dataverse called the Metadata dataverse).
+To store your data in AsterixDB, you will first create a dataverse and then you use it for the _datatypes_
+and _datasets_ for managing your own data.
+A datatype tells AsterixDB what you know (or more accurately, what you want it to know) a priori about one
+of the kinds of data instances that you want AsterixDB to hold for you.
+A dataset is a collection of data instances of a datatype,
+and AsterixDB makes sure that the data instances that you put in it conform to its specified type.
+Since AsterixDB targets semistructured data, you can use _open_ datatypes and tell it as little or as
+much as you wish about your data up front; the more you tell it up front, the less information it will
+have to store repeatedly in the individual data instances that you give it.
+Instances of open datatypes are permitted to have additional content, beyond what the datatype says,
+as long as they at least contain the information prescribed by the datatype definition.
+Open typing allows data to vary from one instance to another and it leaves wiggle room for application
+evolution in terms of what might need to be stored in the future.
+If you want to restrict data instances in a dataset to have only what the datatype says, and nothing extra,
+you can define a _closed_ datatype for that dataset and AsterixDB will keep users from storing objects
+that have extra data in them.
+Datatypes are open by default unless you tell AsterixDB otherwise.
+Let's put these concepts to work
+
+Our little sample scenario involves hypothetical information about users of two popular social networks,
+Facebook and Twitter, and their messages.
+We'll start by defining a dataverse called "TinySocial" to hold our datatypes and datasets.
+The AsterixDB data model (ADM) is essentially a superset of JSON---it's what you get by extending
+JSON with more data types and additional data modeling constructs borrowed from object databases.
+The following is how we can create the TinySocial dataverse plus a set of ADM types for modeling
+Twitter users, their Tweets, Facebook users, their users' employment information, and their messages.
+(Note: Keep in mind that this is just a tiny and somewhat silly example intended for illustrating
+some of the key features of AsterixDB. :-))
+
+
+        drop dataverse TinySocial if exists;
+        create dataverse TinySocial;
+        use dataverse TinySocial;
+        
+        create type TwitterUserType as open {
+        	screen-name: string,
+        	lang: string,
+        	friends_count: int32,
+        	statuses_count: int32,
+        	name: string,
+        	followers_count: int32
+        }
+        
+        create type TweetMessageType as closed {
+        	tweetid: string,
+        	user: TwitterUserType,
+        	sender-location: point?,
+        	send-time: datetime,
+        	referred-topics: {{ string }},
+        	message-text: string
+        }
+        
+        create type EmploymentType as open {
+        	organization-name: string,
+        	start-date: date,
+        	end-date: date?
+        }
+        
+        create type FacebookUserType as closed {
+        	id: int32,
+        	alias: string,
+        	name: string,
+        	user-since: datetime,
+        	friend-ids: {{ int32 }},
+        	employment: [EmploymentType]
+        }
+        
+        create type FacebookMessageType as closed {
+        	message-id: int32,
+        	author-id: int32,
+        	in-response-to: int32?,
+        	sender-location: point?,
+        	message: string
+        }
+        
+
+
+The first three lines above tell AsterixDB to drop the old TinySocial dataverse, if one already
+exists, and then to create a brand new one and make it the focus of the statements that follow.
+The first type creation statement creates a datatype for holding information about Twitter users.
+It is a record type with a mix of integer and string data, very much like a (flat) relational tuple.
+The indicated fields are all mandatory, but because the type is open, additional fields are welcome.
+The second statement creates a datatype for Twitter messages; this shows how to specify a closed type.
+Interestingly (based on one of Twitter's APIs), each Twitter message actually embeds an instance of the
+sending user's information (current as of when the message was sent), so this is an example of a nested
+record in ADM.
+Twitter messages can optionally contain the sender's location, which is modeled via the sender-location
+field of spatial type _point_; the question mark following the field type indicates its optionality.
+An optional field is like a nullable field in SQL---it may be present or missing, but when it's present,
+its data type will conform to the datatype's specification.
+The send-time field illustrates the use of a temporal primitive type, _datetime_.
+Lastly, the referred-topics field illustrates another way that ADM is richer than the relational model;
+this field holds a bag (a.k.a. an unordered list) of strings.
+Since the overall datatype definition for Twitter messages says "closed", the fields that it lists are
+the only fields that instances of this type will be allowed to contain.
+The next two create type statements create a record type for holding information about one component of
+the employment history of a Facebook user and then a record type for holding the user information itself.
+The Facebook user type highlights a few additional ADM data model features.
+Its friend-ids field is a bag of integers, presumably the Facebook user ids for this user's friends,
+and its employment field is an ordered list of employment records.
+The final create type statement defines a type for handling the content of a Facebook message in our
+hypothetical social data storage scenario.
+
+Before going on, we need to once again emphasize the idea that AsterixDB is aimed at storing
+and querying not just Big Data, but Big _Semistructured_ Data.
+This means that most of the fields listed in the create type statements above could have been
+omitted without changing anything other than the resulting size of stored data instances on disk.
+AsterixDB stores its information about the fields defined a priori as separate metadata, whereas
+the information about other fields that are "just there" in instances of open datatypes is stored
+with each instance---making for more bits on disk and longer times for operations affected by
+data size (e.g., dataset scans).
+The only fields that _must_ be specified a priori are the primary key and any fields that you
+would like to build indexes on.
+(AsterixDB does not yet support auto-generated keys or indexes on the unspecified "open" fields
+of its data instances).
+
+### Creating Datasets and Indexes ###
+
+Now that we have defined our datatypes, we can move on and create datasets to store the actual data.
+(If we wanted to, we could even have several named datasets based on any one of these datatypes.)
+We can do this as follows, utilizing the DDL capabilities of AsterixDB.
+
+
+        
+        use dataverse TinySocial;
+        
+        create dataset FacebookUsers(FacebookUserType)
+        primary key id;
+        
+        create dataset FacebookMessages(FacebookMessageType)
+        primary key message-id;
+        
+        create dataset TwitterUsers(TwitterUserType)
+        primary key screen-name;
+        
+        create dataset TweetMessages(TweetMessageType)
+        primary key tweetid
+        hints(cardinality=100);
+        
+        create index fbUserSinceIdx on FacebookUsers(user-since);
+        create index fbAuthorIdx on FacebookMessages(author-id) type btree;
+        create index fbSenderLocIndex on FacebookMessages(sender-location) type rtree;
+        create index fbMessageIdx on FacebookMessages(message) type keyword;
+        
+        for $ds in dataset Metadata.Dataset return $ds;
+        for $ix in dataset Metadata.Index return $ix;
+        
+
+
+The ADM DDL statements above create four datasets for holding our social data in the TinySocial
+dataverse: FacebookUsers, FacebookMessages, TwitterUsers, and TweetMessages.
+The first statement creates the FacebookUsers data set.
+It specifies that this dataset will store data instances conforming to FacebookUserType and that
+it has a primary key which is the id field of each instance.
+The primary key information is used by AsterixDB to uniquely identify instances for the purpose
+of later lookup and for use in secondary indexes.
+Each AsterixDB dataset is stored (and indexed) in the form of a B+ tree on primary key;
+secondary indexes point to their indexed data by primary key.
+In AsterixDB clusters, the primary key is also used to hash-partition (a.k.a. shard) the
+dataset across the nodes of the cluster.
+The next three create dataset statements are similar.
+The last one illustrates an optional clause for providing useful hints to AsterixDB.
+In this case, the hint tells AsterixDB that the dataset definer is anticipating that the
+TweetMessages dataset will contain roughly 100 objects; knowing this can help AsterixDB
+to more efficiently manage and query this dataset.
+(AsterixDB does not yet gather and maintain data statistics; it will currently, abitrarily,
+assume a cardinality of one million objects per dataset in the absence of such an optional
+definition-time hint.)
+
+The create dataset statements above are followed by four more DDL statements, each of which
+creates a secondary index on a field of one of the datasets.
+The first one indexes the FacebookUsers dataset on its user-since field.
+This index will be a B+ tree index; its type is unspecified and _btree_ is the default type.
+The other three illustrate how you can explicitly specify the desired type of index.
+In addition to btree, _rtree_ and inverted _keyword_ indexes are supported by AsterixDB.
+Indexes can also have composite keys, and there are more advanced flavors of text indexing
+available as well (_fuzzy keyword_ and _ngram(k)_, where _k_ is the desired gram length).
+
+### Querying the Metadata Dataverse ###
+
+The last two statements above show how you can use queries in AQL to examine the AsterixDB
+system catalogs and tell what artifacts you have created.
+Just as relational DBMSs use their own tables to store their catalogs, AsterixDB uses
+its own datasets to persist descriptions of its datasets, datatypes, indexes, and so on.
+Running the first of the two queries above will list all of your newly created datasets,
+and it will also show you a full list of all the metadata datasets.
+(You can then explore from there on your own if you are curious)
+These last two queries also illustrate one other factoid worth knowing:
+AsterixDB allows queries to span dataverses by allowing the optional use
+of fully-qualified dataset names (i.e., _dataversename.datasetname_)
+to reference datasets that live in a dataverse other than the one that
+was named in the most recently executed _use dataverse_ directive.
+
+----
+## Loading Data Into AsterixDB ##
+Okay, so far so good---AsterixDB is now ready for data, so let's give it some data to store
+Our next task will be to load some sample data into the four datasets that we just defined.
+Here we will load a tiny set of records, defined in ADM format (a superset of JSON), into each dataset.
+In the boxes below you can see the actual data instances contained in each of the provided sample files.
+In order to load this data yourself, you should first store the four corresponding `.adm` files
+(whose URLs are indicated on top of each box below) into a filesystem directory accessible to your
+running AsterixDB instance.
+Take a few minutes to look carefully at each of the sample data sets.
+This will give you a better sense of the nature of the data that we are about to load and query.
+We should note that ADM format is a textual serialization of what AsterixDB will actually store;
+when persisted in AsterixDB, the data format will be binary and the data in the predefined fields
+of the data instances will be stored separately from their associated field name and type metadata.
+
+[Twitter Users](http://asterixdb.googlecode.com/files/twu.adm)
+
+        {"screen-name":"NathanGiesen@211","lang":"en","friends_count":18,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416}
+        {"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159}
+        {"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649}
+        {"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136}
+
+[Tweet Messages](http://asterixdb.googlecode.com/files/twm.adm)
+
+        {"tweetid":"1","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("47.44,80.65"),"send-time":datetime("2008-04-26T10:10:00"),"referred-topics":{{"t-mobile","customization"}},"message-text":" love t-mobile its customization is good:)"}
+        {"tweetid":"2","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("32.84,67.14"),"send-time":datetime("2010-05-13T10:10:00"),"referred-topics":{{"verizon","shortcut-menu"}},"message-text":" like verizon its shortcut-menu is awesome:)"}
+        {"tweetid":"3","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("29.72,75.8"),"send-time":datetime("2006-11-04T10:10:00"),"referred-topics":{{"motorola","speed"}},"message-text":" like motorola the speed is good:)"}
+        {"tweetid":"4","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("39.28,70.48"),"send-time":datetime("2011-12-26T10:10:00"),"referred-topics":{{"sprint","voice-command"}},"message-text":" like sprint the voice-command is mind-blowing:)"}
+        {"tweetid":"5","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("40.09,92.69"),"send-time":datetime("2006-08-04T10:10:00"),"referred-topics":{{"motorola","speed"}},"message-text":" can't stand motorola its speed is terrible:("}
+        {"tweetid":"6","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("47.51,83.99"),"send-time":datetime("2010-05-07T10:10:00"),"referred-topics":{{"iphone","voice-clarity"}},"message-text":" like iphone the voice-clarity is good:)"}
+        {"tweetid":"7","user":{"screen-name":"ChangEwing_573","lang":"en","friends_count":182,"statuses_count":394,"name":"Chang Ewing","followers_count":32136},"sender-location":point("36.21,72.6"),"send-time":datetime("2011-08-25T10:10:00"),"referred-topics":{{"samsung","platform"}},"message-text":" like samsung the platform is good"}
+        {"tweetid":"8","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("46.05,93.34"),"send-time":datetime("2005-10-14T10:10:00"),"referred-topics":{{"t-mobile","shortcut-menu"}},"message-text":" like t-mobile the shortcut-menu is awesome:)"}
+        {"tweetid":"9","user":{"screen-name":"NathanGiesen@211","lang":"en","friends_count":39339,"statuses_count":473,"name":"Nathan Giesen","followers_count":49416},"sender-location":point("36.86,74.62"),"send-time":datetime("2012-07-21T10:10:00"),"referred-topics":{{"verizon","voicemail-service"}},"message-text":" love verizon its voicemail-service is awesome"}
+        {"tweetid":"10","user":{"screen-name":"ColineGeyer@63","lang":"en","friends_count":121,"statuses_count":362,"name":"Coline Geyer","followers_count":17159},"sender-location":point("29.15,76.53"),"send-time":datetime("2008-01-26T10:10:00"),"referred-topics":{{"verizon","voice-clarity"}},"message-text":" hate verizon its voice-clarity is OMG:("}
+        {"tweetid":"11","user":{"screen-name":"NilaMilliron_tw","lang":"en","friends_count":445,"statuses_count":164,"name":"Nila Milliron","followers_count":22649},"sender-location":point("37.59,68.42"),"send-time":datetime("2008-03-09T10:10:00"),"referred-topics":{{"iphone","platform"}},"message-text":" can't stand iphone its platform is terrible"}
+        {"tweetid":"12","user":{"screen-name":"OliJackson_512","lang":"en","friends_count":445,"statuses_count":164,"name":"Oli Jackson","followers_count":22649},"sender-location":point("24.82,94.63"),"send-time":datetime("2010-02-13T10:10:00"),"referred-topics":{{"samsung","voice-command"}},"message-text":" like samsung the voice-command is amazing:)"}
+
+[Facebook Users](http://asterixdb.googlecode.com/files/fbu.adm)
+
+        {"id":1,"alias":"Margarita","name":"MargaritaStoddard","user-since":datetime("2012-08-20T10:10:00"),"friend-ids":{{2,3,6,10}},"employment":[{"organization-name":"Codetechno","start-date":date("2006-08-06")}]}
+        {"id":2,"alias":"Isbel","name":"IsbelDull","user-since":datetime("2011-01-22T10:10:00"),"friend-ids":{{1,4}},"employment":[{"organization-name":"Hexviafind","start-date":date("2010-04-27")}]}
+        {"id":3,"alias":"Emory","name":"EmoryUnk","user-since":datetime("2012-07-10T10:10:00"),"friend-ids":{{1,5,8,9}},"employment":[{"organization-name":"geomedia","start-date":date("2010-06-17"),"end-date":date("2010-01-26")}]}
+        {"id":4,"alias":"Nicholas","name":"NicholasStroh","user-since":datetime("2010-12-27T10:10:00"),"friend-ids":{{2}},"employment":[{"organization-name":"Zamcorporation","start-date":date("2010-06-08")}]}
+        {"id":5,"alias":"Von","name":"VonKemble","user-since":datetime("2010-01-05T10:10:00"),"friend-ids":{{3,6,10}},"employment":[{"organization-name":"Kongreen","start-date":date("2010-11-27")}]}
+        {"id":6,"alias":"Willis","name":"WillisWynne","user-since":datetime("2005-01-17T10:10:00"),"friend-ids":{{1,3,7}},"employment":[{"organization-name":"jaydax","start-date":date("2009-05-15")}]}
+        {"id":7,"alias":"Suzanna","name":"SuzannaTillson","user-since":datetime("2012-08-07T10:10:00"),"friend-ids":{{6}},"employment":[{"organization-name":"Labzatron","start-date":date("2011-04-19")}]}
+        {"id":8,"alias":"Nila","name":"NilaMilliron","user-since":datetime("2008-01-01T10:10:00"),"friend-ids":{{3}},"employment":[{"organization-name":"Plexlane","start-date":date("2010-02-28")}]}
+        {"id":9,"alias":"Woodrow","name":"WoodrowNehling","user-since":datetime("2005-09-20T10:10:00"),"friend-ids":{{3,10}},"employment":[{"organization-name":"Zuncan","start-date":date("2003-04-22"),"end-date":date("2009-12-13")}]}
+        {"id":10,"alias":"Bram","name":"BramHatch","user-since":datetime("2010-10-16T10:10:00"),"friend-ids":{{1,5,9}},"employment":[{"organization-name":"physcane","start-date":date("2007-06-05"),"end-date":date("2011-11-05")}]}
+
+[Facebook Messages](http://asterixdb.googlecode.com/files/fbm.adm)
+
+        {"message-id":1,"author-id":3,"in-response-to":2,"sender-location":point("47.16,77.75"),"message":" love sprint its shortcut-menu is awesome:)"}
+        {"message-id":2,"author-id":1,"in-response-to":4,"sender-location":point("41.66,80.87"),"message":" dislike iphone its touch-screen is horrible"}
+        {"message-id":3,"author-id":2,"in-response-to":4,"sender-location":point("48.09,81.01"),"message":" like samsung the plan is amazing"}
+        {"message-id":4,"author-id":1,"in-response-to":2,"sender-location":point("37.73,97.04"),"message":" can't stand at&t the network is horrible:("}
+        {"message-id":5,"author-id":6,"in-response-to":2,"sender-location":point("34.7,90.76"),"message":" love sprint the customization is mind-blowing"}
+        {"message-id":6,"author-id":2,"in-response-to":1,"sender-location":point("31.5,75.56"),"message":" like t-mobile its platform is mind-blowing"}
+        {"message-id":7,"author-id":5,"in-response-to":15,"sender-location":point("32.91,85.05"),"message":" dislike sprint the speed is horrible"}
+        {"message-id":8,"author-id":1,"in-response-to":11,"sender-location":point("40.33,80.87"),"message":" like verizon the 3G is awesome:)"}
+        {"message-id":9,"author-id":3,"in-response-to":12,"sender-location":point("34.45,96.48"),"message":" love verizon its wireless is good"}
+        {"message-id":10,"author-id":1,"in-response-to":12,"sender-location":point("42.5,70.01"),"message":" can't stand motorola the touch-screen is terrible"}
+        {"message-id":11,"author-id":1,"in-response-to":1,"sender-location":point("38.97,77.49"),"message":" can't stand at&t its plan is terrible"}
+        {"message-id":12,"author-id":10,"in-response-to":6,"sender-location":point("42.26,77.76"),"message":" can't stand t-mobile its voicemail-service is OMG:("}
+        {"message-id":13,"author-id":10,"in-response-to":4,"sender-location":point("42.77,78.92"),"message":" dislike iphone the voice-command is bad:("}
+        {"message-id":14,"author-id":9,"in-response-to":12,"sender-location":point("41.33,85.28"),"message":" love at&t its 3G is good:)"}
+        {"message-id":15,"author-id":7,"in-response-to":11,"sender-location":point("44.47,67.11"),"message":" like iphone the voicemail-service is awesome"}
+
+
+It's loading time! We can use AQL _load_ statements to populate our datasets with the sample records shown above.
+The following shows how loading can be done for data stored in `.adm` files in your local filesystem.
+*Note:* You _MUST_ replace the `<Host Name>` and `<Absolute File Path>` placeholders in each load
+statement below with valid values based on the host IP address (or host name) for the machine and
+directory that you have downloaded the provided `.adm` files to.
+As you do so, be very, very careful to retain the two slashes in the load statements, i.e.,
+do not delete the two slashes that appear in front of the absolute path to your `.adm` files.
+(This will lead to a three-slash character sequence at the start of each load statement's file
+input path specification.)
+
+
+        use dataverse TinySocial;
+        
+        load dataset FacebookUsers using localfs
+        (("path"="<Host Name>://<Absolute File Path>/fbu.adm"),("format"="adm"));
+        
+        load dataset FacebookMessages using localfs
+        (("path"="<Host Name>://<Absolute File Path>/fbm.adm"),("format"="adm"));
+        
+        load dataset TwitterUsers using localfs
+        (("path"="<Host Name>://<Absolute File Path>/twu.adm"),("format"="adm"));
+        
+        load dataset TweetMessages using localfs
+        (("path"="<Host Name>://<Absolute File Path>/twm.adm"),("format"="adm"));
+
+
+----
+## AQL: Querying Your AsterixDB Data ##
+Congratulations! You now have sample social data stored (and indexed) in AsterixDB.
+(You are part of an elite and adventurous group of individuals. :-))
+Now that you have successfully loaded the provided sample data into the datasets that we defined,
+you can start running queries against them.
+
+The query language for AsterixDB is AQL---the Asterix Query Language.
+AQL is loosely based on XQuery, the language developed and standardized in the early to mid 2000's
+by the World Wide Web Consortium (W3C) for querying semistructured data stored in their XML format.
+We have tossed all of the "XML cruft" out of their language but retained many of its core ideas.
+We did this because its design was developed over a period of years by a diverse committee of smart
+and experienced language designers, including "SQL people", "functional programming people", and
+"XML people", all of whom were focused on how to design a new query language that operates well over
+semistructured data.
+(We decided to stand on their shoulders instead of starting from scratch and revisiting many of the
+same issues.)
+Note that AQL is not SQL and not based on SQL: In other words, AsterixDB is fully "NoSQL compliant". :-)
+
+In this section we introduce AQL via a set of example queries, along with their expected results,
+based on the data above, to help you get started.
+Many of the most important features of AQL are presented in this set of representative queries.
+You can find a BNF description of the current AQL grammar at [wiki:AsterixDBGrammar], and someday
+in the not-too-distant future we will also provide a complete reference manual for the language.
+In the meantime, this will get you started down the path of using AsterixDB.
+A more complete list of the supported AsterixDB primitive types and built-in functions can be
+found at [AsterixDataTypesAndFunctions](AsterixDataTypesAndFunctions.html).
+
+AQL is an expression language.
+Even the expression 1+1 is a valid AQL query that evaluates to 2.
+(Try it for yourself!
+Okay, maybe that's _not_ the best use of a 512-node shared-nothing compute cluster.)
+Most useful AQL queries will be based on the _FLWOR_ (pronounced "flower") expression structure
+that AQL has borrowed from XQuery ((http://en.wikipedia.org/wiki/FLWOR)).
+The FLWOR expression syntax supports both the incremental binding (_for_) of variables to ADM data
+instances in a dataset (or in the result of any AQL expression, actually) and the full binding (_let_)
+of variables to entire intermediate results in a fashion similar to temporary views in the SQL world.
+FLWOR is an acronym that is short for _for_-_let_-_where_-_order by_-_return_,
+naming five of the most frequently used clauses from the syntax of a full AQL query.
+AQL also includes _group by_ and _limit_ clauses, as you will see shortly.
+Roughly speaking, for SQL afficiandos, the _for_ clause in AQL is like the _from_ clause in SQL,
+the _return_ clause in AQL is like the _select_ clause in SQL (but appears at the end instead of
+the beginning of a query), the _let_ clause in AQL is like SQL's _with_ clause, and the _where_
+and _order by_ clauses in both languages are similar.
+
+Enough talk!
+Let's go ahead and try writing some queries and see about learning AQL by example.
+
+### Query 0-A - Exact-Match Lookup ###
+For our first query, let's find a Facebook user based on his or her user id.
+Suppose the user we want is the user whose id is 8:
+
+
+        use dataverse TinySocial;
+        
+        for $user in dataset FacebookUsers
+        where $user.id = 8
+        return $user;
+
+The query's _for_ clause  binds the variable `$user` incrementally to the data instances residing in
+the dataset named FacebookUsers.
+Its _where_ clause selects only those bindings having a user id of interest, filtering out the rest.
+The _return_ clause returns the (entire) data instance for each binding that satisfies the predicate.
+Since this dataset is indexed on user id (its primary key), this query will be done via a quick index lookup.
+
+The expected result for our sample data is as follows:
+
+        { "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28"), "end-date": null } ] }
+
+
+### Query 0-B - Range Scan ###
+AQL, like SQL, supports a variety of different predicates.
+For example, for our next query, let's find the Facebook users whose ids are in the range between 2 and 4:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset FacebookUsers
+        where $user.id >= 2 and $user.id <= 4
+        return $user;
+
+This query's expected result, also evaluable using the primary index on user id, is:
+
+        { "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27"), "end-date": null } ] }
+        { "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+        { "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08"), "end-date": null } ] }
+
+
+### Query 1 - Other Query Filters ###
+AQL can do range queries on any data type that supports the appropriate set of comparators.
+As an example, this next query retrieves the Facebook users who joined between July 22, 2010 and July 29, 2012:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset FacebookUsers
+        where $user.user-since >= datetime('2010-07-22T00:00:00')
+          and $user.user-since <= datetime('2012-07-29T23:59:59')
+        return $user;
+
+The expected result for this query, also an indexable query, is as follows:
+
+        { "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27"), "end-date": null } ] }
+        { "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+        { "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08"), "end-date": null } ] }
+        { "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
+
+
+### Query 2-A - Equijoin ###
+In addition to simply binding variables to data instances and returning them "whole",
+an AQL query can construct new ADM instances to return based on combinations of its variable bindings.
+This gives AQL the power to do joins much like those done using multi-table _from_ clauses in SQL.
+For example, suppose we wanted a list of all Facebook users paired with their associated messages,
+with the list enumerating the author name and the message text associated with each Facebook message.
+We could do this as follows in AQL:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset FacebookUsers
+        for $message in dataset FacebookMessages
+        where $message.author-id = $user.id
+        return {
+        "uname": $user.name,
+        "message": $message.message
+        };
+
+The result of this query is a sequence of new ADM instances, one for each author/message pair.
+Each instance in the result will be an ADM record containing two fields, "uname" and "message",
+containing the user's name and the message text, respectively, for each author/message pair.
+(Note that "uname" and "message" are both simple AQL expressions themselves---so in the most
+general case, even the resulting field names can be computed as part of the query, making AQL
+a very powerful tool for slicing and dicing semistructured data.)
+
+The expected result of this example AQL join query for our sample data set is:
+
+        { "uname": "MargaritaStoddard", "message": " dislike iphone its touch-screen is horrible" }
+        { "uname": "MargaritaStoddard", "message": " can't stand at&t the network is horrible:(" }
+        { "uname": "MargaritaStoddard", "message": " like verizon the 3G is awesome:)" }
+        { "uname": "MargaritaStoddard", "message": " can't stand motorola the touch-screen is terrible" }
+        { "uname": "MargaritaStoddard", "message": " can't stand at&t its plan is terrible" }
+        { "uname": "IsbelDull", "message": " like samsung the plan is amazing" }
+        { "uname": "IsbelDull", "message": " like t-mobile its platform is mind-blowing" }
+        { "uname": "EmoryUnk", "message": " love sprint its shortcut-menu is awesome:)" }
+        { "uname": "EmoryUnk", "message": " love verizon its wireless is good" }
+        { "uname": "VonKemble", "message": " dislike sprint the speed is horrible" }
+        { "uname": "WillisWynne", "message": " love sprint the customization is mind-blowing" }
+        { "uname": "SuzannaTillson", "message": " like iphone the voicemail-service is awesome" }
+        { "uname": "WoodrowNehling", "message": " love at&t its 3G is good:)" }
+        { "uname": "BramHatch", "message": " can't stand t-mobile its voicemail-service is OMG:(" }
+        { "uname": "BramHatch", "message": " dislike iphone the voice-command is bad:(" }
+
+
+### Query 2-B - Index join ###
+By default, AsterixDB evaluates equijoin queries using hash-based join methods that work
+well for doing ad hoc joins of very large data sets
+([http://en.wikipedia.org/wiki/Hash_join](http://en.wikipedia.org/wiki/Hash_join)).
+On a cluster, hash partitioning is employed as AsterixDB's divide-and-conquer strategy for
+computing large parallel joins.
+AsterixDB includes other join methods, but in the absence of data statistics and selectivity
+estimates, it doesn't (yet) have the know-how to intelligently choose among its alternatives.
+We therefore asked ourselves the classic question---WWOD?---What Would Oracle Do?---and in the
+interim, AQL includes a clunky (but useful) hint-based mechanism for addressing the occasional
+need to suggest to AsterixDB which join method it should use for a particular AQL query.
+
+The following query is similar to Query 2-A but includes a suggestion to AsterixDB that it
+should consider employing an index-based nested-loop join technique to process the query:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset FacebookUsers
+        for $message in dataset FacebookMessages
+        where $message.author-id /*+ indexnl */  = $user.id
+        return {
+        "uname": $user.name,
+        "message": $message.message
+        };
+        
+
+The expected result is (of course) the same as before, modulo the order of the instances.
+Result ordering is (intentionally) undefined in AQL in the absence of an _order by_ clause.
+The query result for our sample data in this case is:
+
+        { "uname": "EmoryUnk", "message": " love sprint its shortcut-menu is awesome:)" }
+        { "uname": "MargaritaStoddard", "message": " dislike iphone its touch-screen is horrible" }
+        { "uname": "IsbelDull", "message": " like samsung the plan is amazing" }
+        { "uname": "MargaritaStoddard", "message": " can't stand at&t the network is horrible:(" }
+        { "uname": "WillisWynne", "message": " love sprint the customization is mind-blowing" }
+        { "uname": "IsbelDull", "message": " like t-mobile its platform is mind-blowing" }
+        { "uname": "VonKemble", "message": " dislike sprint the speed is horrible" }
+        { "uname": "MargaritaStoddard", "message": " like verizon the 3G is awesome:)" }
+        { "uname": "EmoryUnk", "message": " love verizon its wireless is good" }
+        { "uname": "MargaritaStoddard", "message": " can't stand motorola the touch-screen is terrible" }
+        { "uname": "MargaritaStoddard", "message": " can't stand at&t its plan is terrible" }
+        { "uname": "BramHatch", "message": " can't stand t-mobile its voicemail-service is OMG:(" }
+        { "uname": "BramHatch", "message": " dislike iphone the voice-command is bad:(" }
+        { "uname": "WoodrowNehling", "message": " love at&t its 3G is good:)" }
+        { "uname": "SuzannaTillson", "message": " like iphone the voicemail-service is awesome" }
+
+
+(It is worth knowing, with respect to influencing AsterixDB's query evaluation, that nested _for_
+clauses---a.k.a. joins--- are currently evaluated with the "outer" clause probing the data of the "inner"
+clause.)
+
+### Query 3 - Nested Outer Join ###
+In order to support joins between tables with missing/dangling join tuples, the designers of SQL ended
+up shoe-horning a subset of the relational algebra into SQL's _from_ clause syntax---and providing a
+variety of join types there for users to choose from.
+Left outer joins are particularly important in SQL, e.g., to print a summary of customers and orders,
+grouped by customer, without omitting those customers who haven't placed any orders yet.
+
+The AQL language supports nesting, both of queries and of query results, and the combination allows for
+an arguably cleaner/more natural approach to such queries.
+As an example, supposed we wanted, for each Facebook user, to produce a record that has his/her name
+plus a list of the messages written by that user.
+In SQL, this would involve a left outer join between users and messages, grouping by user, and having
+the user name repeated along side each message.
+In AQL, this sort of use case can be handled (more naturally) as follows:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset FacebookUsers
+        return {
+        "uname": $user.name,
+        "messages": for $message in dataset FacebookMessages
+        		where $message.author-id = $user.id
+        		return $message.message
+        };
+
+This AQL query binds the variable `$user` to the data instances in FacebookUsers;
+for each user, it constructs a result record containing a "uname" field with the user's
+name and a "messages" field with a nested collection of all messages for that user.
+The nested collection for each user is specified by using a correlated subquery.
+(Note: While it looks like nested loops could be involved in computing the result,
+AsterixDB recogizes the equivalence of such a query to an outerjoin, and it will
+use an efficient hash-based strategy when actually computing the query's result.)
+
+Here is this example query's expected output:
+
+        { "uname": "MargaritaStoddard", "messages": [ " dislike iphone its touch-screen is horrible", " can't stand at&t the network is horrible:(", " like verizon the 3G is awesome:)", " can't stand motorola the touch-screen is terrible", " can't stand at&t its plan is terrible" ] }
+        { "uname": "IsbelDull", "messages": [ " like samsung the plan is amazing", " like t-mobile its platform is mind-blowing" ] }
+        { "uname": "EmoryUnk", "messages": [ " love sprint its shortcut-menu is awesome:)", " love verizon its wireless is good" ] }
+        { "uname": "NicholasStroh", "messages": [  ] }
+        { "uname": "VonKemble", "messages": [ " dislike sprint the speed is horrible" ] }
+        { "uname": "WillisWynne", "messages": [ " love sprint the customization is mind-blowing" ] }
+        { "uname": "SuzannaTillson", "messages": [ " like iphone the voicemail-service is awesome" ] }
+        { "uname": "NilaMilliron", "messages": [  ] }
+        { "uname": "WoodrowNehling", "messages": [ " love at&t its 3G is good:)" ] }
+        { "uname": "BramHatch", "messages": [ " dislike iphone the voice-command is bad:(", " can't stand t-mobile its voicemail-service is OMG:(" ] }
+
+
+### Query 4 - Theta Join ###
+Not all joins are expressible as equijoins and computable using equijoin-oriented algorithms.
+The join predicates for some use cases involve predicates with functions; AsterixDB supports the
+expression of such queries and will still evaluate them as best it can using nested loop based
+techniques (and broadcast joins in the parallel case).
+
+As an example of such a use case, suppose that we wanted, for each tweet T, to find all of the
+other tweets that originated from within a circle of radius of 1 surrounding tweet T's location.
+In AQL, this can be specified in a manner similar to the previous query using one of the built-in
+functions on the spatial data type instead of id equality in the correlated query's _where_ clause:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset TweetMessages
+        return {
+        "message": $t.message-text,
+        "nearby-messages": for $t2 in dataset TweetMessages
+        			where spatial-distance($t.sender-location, $t2.sender-location) <= 1
+        			return { "msgtxt":$t2.message-text}
+        };
+
+Here is the expected result for this query:
+
+        { "message": " love t-mobile its customization is good:)", "nearby-messages": [ { "msgtxt": " love t-mobile its customization is good:)" } ] }
+        { "message": " hate verizon its voice-clarity is OMG:(", "nearby-messages": [ { "msgtxt": " like motorola the speed is good:)" }, { "msgtxt": " hate verizon its voice-clarity is OMG:(" } ] }
+        { "message": " can't stand iphone its platform is terrible", "nearby-messages": [ { "msgtxt": " can't stand iphone its platform is terrible" } ] }
+        { "message": " like samsung the voice-command is amazing:)", "nearby-messages": [ { "msgtxt": " like samsung the voice-command is amazing:)" } ] }
+        { "message": " like verizon its shortcut-menu is awesome:)", "nearby-messages": [ { "msgtxt": " like verizon its shortcut-menu is awesome:)" } ] }
+        { "message": " like motorola the speed is good:)", "nearby-messages": [ { "msgtxt": " hate verizon its voice-clarity is OMG:(" }, { "msgtxt": " like motorola the speed is good:)" } ] }
+        { "message": " like sprint the voice-command is mind-blowing:)", "nearby-messages": [ { "msgtxt": " like sprint the voice-command is mind-blowing:)" } ] }
+        { "message": " can't stand motorola its speed is terrible:(", "nearby-messages": [ { "msgtxt": " can't stand motorola its speed is terrible:(" } ] }
+        { "message": " like iphone the voice-clarity is good:)", "nearby-messages": [ { "msgtxt": " like iphone the voice-clarity is good:)" } ] }
+        { "message": " like samsung the platform is good", "nearby-messages": [ { "msgtxt": " like samsung the platform is good" } ] }
+        { "message": " like t-mobile the shortcut-menu is awesome:)", "nearby-messages": [ { "msgtxt": " like t-mobile the shortcut-menu is awesome:)" } ] }
+        { "message": " love verizon its voicemail-service is awesome", "nearby-messages": [ { "msgtxt": " love verizon its voicemail-service is awesome" } ] }
+
+
+### Query 5 - Fuzzy Join ###
+As another example of a non-equijoin use case, we could ask AsterixDB to find, for each Facebook user,
+all Twitter users with names "similar" to their name.
+AsterixDB supports a variety of "fuzzy match" functions for use with textual and set-based data.
+As one example, we could choose to use edit distance with a threshold of 3 as the definition of name
+similarity, in which case we could write the following query using AQL's operator-based syntax (~=)
+for testing whether or not two values are similar:
+
+        use dataverse TinySocial;
+        
+        set simfunction "edit-distance";
+        set simthreshold "3";
+        
+        for $fbu in dataset FacebookUsers
+        return {
+            "id": $fbu.id,
+            "name": $fbu.name,
+            "similar-users": for $t in dataset TweetMessages
+        			let $tu := $t.user
+        			where $tu.name ~= $fbu.name
+        			return {
+        			"twitter-screenname": $tu.screen-name,
+        			"twitter-name": $tu.name
+        			}
+        };
+
+The expected result for this query against our sample data is:
+
+        { "id": 1, "name": "MargaritaStoddard", "similar-users": [  ] }
+        { "id": 2, "name": "IsbelDull", "similar-users": [  ] }
+        { "id": 3, "name": "EmoryUnk", "similar-users": [  ] }
+        { "id": 4, "name": "NicholasStroh", "similar-users": [  ] }
+        { "id": 5, "name": "VonKemble", "similar-users": [  ] }
+        { "id": 6, "name": "WillisWynne", "similar-users": [  ] }
+        { "id": 7, "name": "SuzannaTillson", "similar-users": [  ] }
+        { "id": 8, "name": "NilaMilliron", "similar-users": [ { "twitter-screenname": "NilaMilliron_tw", "twitter-name": "Nila Milliron" } ] }
+        { "id": 9, "name": "WoodrowNehling", "similar-users": [  ] }
+        { "id": 10, "name": "BramHatch", "similar-users": [  ] }
+
+
+### Query 6 - Existential Quantification ###
+The expressive power of AQL includes support for queries involving "some" (existentially quantified)
+and "all" (universally quantified) query semantics.
+As an example of an existential AQL query, here we show a query to list the Facebook users who are currently employed.
+Such employees will have an employment history containing a record with a null end-date value, which leads us to the
+following AQL query:
+
+        use dataverse TinySocial;
+        
+        for $fbu in dataset FacebookUsers
+        where (some $e in $fbu.employment satisfies is-null($e.end-date))
+        return $fbu;
+
+The expected result in this case is:
+
+        { "id": 1, "alias": "Margarita", "name": "MargaritaStoddard", "user-since": datetime("2012-08-20T10:10:00.000Z"), "friend-ids": {{ 2, 3, 6, 10 }}, "employment": [ { "organization-name": "Codetechno", "start-date": date("2006-08-06"), "end-date": null } ] }
+        { "id": 2, "alias": "Isbel", "name": "IsbelDull", "user-since": datetime("2011-01-22T10:10:00.000Z"), "friend-ids": {{ 1, 4 }}, "employment": [ { "organization-name": "Hexviafind", "start-date": date("2010-04-27"), "end-date": null } ] }
+        { "id": 4, "alias": "Nicholas", "name": "NicholasStroh", "user-since": datetime("2010-12-27T10:10:00.000Z"), "friend-ids": {{ 2 }}, "employment": [ { "organization-name": "Zamcorporation", "start-date": date("2010-06-08"), "end-date": null } ] }
+        { "id": 5, "alias": "Von", "name": "VonKemble", "user-since": datetime("2010-01-05T10:10:00.000Z"), "friend-ids": {{ 3, 6, 10 }}, "employment": [ { "organization-name": "Kongreen", "start-date": date("2010-11-27"), "end-date": null } ] }
+        { "id": 6, "alias": "Willis", "name": "WillisWynne", "user-since": datetime("2005-01-17T10:10:00.000Z"), "friend-ids": {{ 1, 3, 7 }}, "employment": [ { "organization-name": "jaydax", "start-date": date("2009-05-15"), "end-date": null } ] }
+        { "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000Z"), "friend-ids": {{ 6 }}, "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19"), "end-date": null } ] }
+        { "id": 8, "alias": "Nila", "name": "NilaMilliron", "user-since": datetime("2008-01-01T10:10:00.000Z"), "friend-ids": {{ 3 }}, "employment": [ { "organization-name": "Plexlane", "start-date": date("2010-02-28"), "end-date": null } ] }
+
+
+### Query 7 - Universal Quantification ###
+As an example of a universal AQL query, here we show a query to list the Facebook users who are currently unemployed.
+Such employees will have an employment history containing no records with null end-date values, leading us to the
+following AQL query:
+
+        use dataverse TinySocial;
+        
+        for $fbu in dataset FacebookUsers
+        where (every $e in $fbu.employment satisfies not(is-null($e.end-date)))
+        return $fbu;
+
+Here is the expected result for our sample data:
+
+        { "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }}, "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ] }
+        { "id": 9, "alias": "Woodrow", "name": "WoodrowNehling", "user-since": datetime("2005-09-20T10:10:00.000Z"), "friend-ids": {{ 3, 10 }}, "employment": [ { "organization-name": "Zuncan", "start-date": date("2003-04-22"), "end-date": date("2009-12-13") } ] }
+        { "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }}, "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ] }
+
+
+### Query 8 - Simple Aggregation ###
+Like SQL, the AQL language of AsterixDB provides support for computing aggregates over large amounts of data.
+As a very simple example, the following AQL query computes the total number of Facebook users:
+
+        use dataverse TinySocial;
+        
+        count(for $fbu in dataset FacebookUsers return $fbu);
+
+In AQL, aggregate functions can be applied to arbitrary subquery results; in this case, the count function
+is applied to the result of a query that enumerates the Facebook users.  The expected result here is:
+
+        10
+
+
+
+### Query 9-A - Grouping and Aggregation ###
+Also like SQL, AQL supports grouped aggregation.
+For every Twitter user, the following group-by/aggregate query counts the number of tweets sent by that user:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset TweetMessages
+        group by $uid := $t.user.screen-name with $t
+        return {
+        "user": $uid,
+        "count": count($t)
+        };
+
+The _for_ clause incrementally binds $t to tweets, and the _group by_ clause groups the tweets by its
+issuer's Twitter screen-name.
+Unlike SQL, where data is tabular---flat---the data model underlying AQL allows for nesting.
+Thus, following the _group by_ clause, the _return_ clause in this query sees a sequence of $t groups,
+with each such group having an associated $uid variable value (i.e., the tweeting user's screen name).
+In the context of the return clause, due to "... with $t ...", $uid is bound to the tweeter's id and $t
+is bound to the _set_ of tweets issued by that tweeter.
+The return clause constructs a result record containing the tweeter's user id and the count of the items
+in the associated tweet set.
+The query result will contain one such record per screen name.
+This query also illustrates another feature of AQL; notice that each user's screen name is accessed via a
+path syntax that traverses each tweet's nested record structure.
+
+Here is the expected result for this query over the sample data:
+
+        { "user": "ChangEwing_573", "count": 1 }
+        { "user": "ColineGeyer@63", "count": 3 }
+        { "user": "NathanGiesen@211", "count": 6 }
+        { "user": "NilaMilliron_tw", "count": 1 }
+        { "user": "OliJackson_512", "count": 1 }
+
+
+
+### Query 9-B - (Hash-Based) Grouping and Aggregation ###
+As for joins, AsterixDB has multiple evaluation strategies available for processing grouped aggregate queries.
+For grouped aggregation, the system knows how to employ both sort-based and hash-based aggregation methods,
+with sort-based methods being used by default and a hint being available to suggest that a different approach
+be used in processing a particular AQL query.
+
+The following query is similar to Query 9-A, but adds a hash-based aggregation hint:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset TweetMessages
+        /*+ hash*/
+        group by $uid := $t.user.screen-name with $t
+        return {
+        "user": $uid,
+        "count": count($t)
+        };
+
+Here is the expected result:
+
+        { "user": "OliJackson_512", "count": 1 }
+        { "user": "ColineGeyer@63", "count": 3 }
+        { "user": "NathanGiesen@211", "count": 6 }
+        { "user": "NilaMilliron_tw", "count": 1 }
+        { "user": "ChangEwing_573", "count": 1 }
+
+
+
+### Query 10 - Grouping and Limits ###
+In some use cases it is not necessary to compute the entire answer to a query.
+In some cases, just having the first _N_ or top _N_ results is sufficient.
+This is expressible in AQL using the _limit_ clause combined with the _order by_ clause.
+
+The following AQL  query returns the top 3 Twitter users based on who has issued the most tweets:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset TweetMessages
+        group by $uid := $t.user.screen-name with $t
+        let $c := count($t)
+        order by $c desc
+        limit 3
+        return {
+        	"user": $uid,
+        	"count": $c
+        };
+
+The expected result for this query is:
+
+        { "user": "NathanGiesen@211", "count": 6 }
+        { "user": "ColineGeyer@63", "count": 3 }
+        { "user": "NilaMilliron_tw", "count": 1 }
+
+
+### Query 11 - Left Outer Fuzzy Join ###
+As a last example of AQL and its query power, the following query, for each tweet,
+finds all of the tweets that are similar based on the topics that they refer to:
+
+        use dataverse TinySocial;
+        
+        set simfunction "jaccard";
+        set simthreshold "0.3";
+        
+        for $t in dataset TweetMessages
+        return {
+            "tweet": $t,
+            "similar-tweets": for $t2 in dataset TweetMessages
+        			where  $t2.referred-topics ~= $t.referred-topics
+        			and $t2.tweetid != $t.tweetid
+        			return $t2.referred-topics
+        };
+
+This query illustrates several things worth knowing in order to write fuzzy queries in AQL.
+First, as mentioned earlier, AQL offers an operator-based syntax for seeing whether two values are "similar" to one another or not.
+Second, recall that the referred-topics field of records of datatype TweetMessageType is a bag of strings.
+This query sets the context for its similarity join by requesting that Jaccard-based similarity semantics
+([http://en.wikipedia.org/wiki/Jaccard_index](http://en.wikipedia.org/wiki/Jaccard_index))
+be used for the query's similarity operator and that a similarity index of 0.3 be used as its similarity threshold.
+
+The expected result for this fuzzy join query is:
+
+        { "tweet": { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " love t-mobile its customization is good:)" }, "similar-tweets": [ {{ "t-mobile", "shortcut-menu" }} ] }
+        { "tweet": { "tweetid": "10", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("29.15,76.53"), "send-time": datetime("2008-01-26T10:10:00.000Z"), "referred-topics": {{ "verizon", "voice-clarity" }}, "message-text": " hate verizon its voice-clarity is OMG:(" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "verizon", "voicemail-service" }}, {{ "verizon", "shortcut-menu" }} ] }
+        { "tweet": { "tweetid": "11", "user": { "screen-name": "NilaMilliron_tw", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Nila Milliron", "followers_count": 22649 }, "sender-location": point("37.59,68.42"), "send-time": datetime("2008-03-09T10:10:00.000Z"), "referred-topics": {{ "iphone", "platform" }}, "message-text": " can't stand iphone its platform is terrible" }, "similar-tweets": [ {{ "iphone", "voice-clarity" }}, {{ "samsung", "platform" }} ] }
+        { "tweet": { "tweetid": "12", "user": { "screen-name": "OliJackson_512", "lang": "en", "friends_count": 445, "statuses_count": 164, "name": "Oli Jackson", "followers_count": 22649 }, "sender-location": point("24.82,94.63"), "send-time": datetime("2010-02-13T10:10:00.000Z"), "referred-topics": {{ "samsung", "voice-command" }}, "message-text": " like samsung the voice-command is amazing:)" }, "similar-tweets": [ {{ "samsung", "platform" }}, {{ "sprint", "voice-command" }} ] }
+        { "tweet": { "tweetid": "2", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("32.84,67.14"), "send-time": datetime("2010-05-13T10:10:00.000Z"), "referred-topics": {{ "verizon", "shortcut-menu" }}, "message-text": " like verizon its shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "verizon", "voicemail-service" }}, {{ "verizon", "voice-clarity" }}, {{ "t-mobile", "shortcut-menu" }} ] }
+        { "tweet": { "tweetid": "3", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("29.72,75.8"), "send-time": datetime("2006-11-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " like motorola the speed is good:)" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+        { "tweet": { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }, "similar-tweets": [ {{ "samsung", "voice-command" }} ] }
+        { "tweet": { "tweetid": "5", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("40.09,92.69"), "send-time": datetime("2006-08-04T10:10:00.000Z"), "referred-topics": {{ "motorola", "speed" }}, "message-text": " can't stand motorola its speed is terrible:(" }, "similar-tweets": [ {{ "motorola", "speed" }} ] }
+        { "tweet": { "tweetid": "6", "user": { "screen-name": "ColineGeyer@63", "lang": "en", "friends_count": 121, "statuses_count": 362, "name": "Coline Geyer", "followers_count": 17159 }, "sender-location": point("47.51,83.99"), "send-time": datetime("2010-05-07T10:10:00.000Z"), "referred-topics": {{ "iphone", "voice-clarity" }}, "message-text": " like iphone the voice-clarity is good:)" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "iphone", "platform" }} ] }
+        { "tweet": { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }, "similar-tweets": [ {{ "iphone", "platform" }}, {{ "samsung", "voice-command" }} ] }
+        { "tweet": { "tweetid": "8", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("46.05,93.34"), "send-time": datetime("2005-10-14T10:10:00.000Z"), "referred-topics": {{ "t-mobile", "shortcut-menu" }}, "message-text": " like t-mobile the shortcut-menu is awesome:)" }, "similar-tweets": [ {{ "t-mobile", "customization" }}, {{ "verizon", "shortcut-menu" }} ] }
+        { "tweet": { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }, "similar-tweets": [ {{ "verizon", "voice-clarity" }}, {{ "verizon", "shortcut-menu" }} ] }
+
+
+### Inserting New Data  ###
+In addition to loading and querying data, AsterixDB supports incremental additions to datasets via the AQL _insert_ statement.
+
+The following example adds a new tweet by user "NathanGiesen@211" to the TweetMessages dataset.
+(An astute reader may notice that this tweet was issued a half an hour after his last tweet, so his counts
+have all gone up in the interim, although he appears not to have moved in the last half hour.)
+
+        use dataverse TinySocial;
+        
+        insert into dataset TweetMessages
+        (
+           {"tweetid":"13",
+            "user":
+                {"screen-name":"NathanGiesen@211",
+                 "lang":"en",
+                 "friends_count":39345,
+                 "statuses_count":479,
+                 "name":"Nathan Giesen",
+                 "followers_count":49420
+                },
+            "sender-location":point("47.44,80.65"),
+            "send-time":datetime("2008-04-26T10:10:35"),
+            "referred-topics":{{"tweeting"}},
+            "message-text":"tweety tweet, my fellow tweeters!"
+           }
+        );
+
+In general, the data to be inserted may be specified using any valid AQL query expression.
+The insertion of a single object instance, as in this example, is just a special case where
+the query expression happens to be a record constructor involving only constants.
+
+### Deleting Existing Data  ###
+In addition to inserting new data, AsterixDB supports deletion from datasets via the AQL _delete_ statement.
+The statement supports "searched delete" semantics, and its
+_where_ clause can involve any valid XQuery expression.
+
+The following example deletes the tweet that we just added from user "NathanGiesen@211".  (Easy come, easy go. :-))
+
+        use dataverse TinySocial;
+        
+        delete $tm from dataset TweetMessages where $tm.tweetid = "13";
+
+It should be noted that one form of data change not yet supported by AsterixDB is in-place data modification (_update_).
+Currently, only insert and delete operations are supported; update is not.
+To achieve the effect of an update, two statements are currently needed---one to delete the old record from the
+dataset where it resides, and another to insert the new replacement record (with the same primary key but with
+different field values for some of the associated data content).
+
+## Further Help ##
+That's it  You are now armed and dangerous with respect to semistructured data management using AsterixDB.
+
+AsterixDB is a powerful new BDMS---Big Data Management System---that we hope may usher in a new era of much
+more declarative Big Data management.
+AsterixDB is powerful, so use it wisely, and remember: "With great power comes great responsibility..." :-)
+
+Please e-mail the AsterixDB user group
+(asterixdb-users (at) googlegroups.com)
+if you run into any problems or simply have further questions about the AsterixDB system, its features, or their proper use.
diff --git a/asterix-doc/src/site/markdown/AsterixAlphaRelease.md b/asterix-doc/src/site/markdown/AsterixAlphaRelease.md
new file mode 100644
index 0000000..ff9efb2
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AsterixAlphaRelease.md
@@ -0,0 +1,57 @@
+# AsterixDB: A Big Data Management System _(Alpha Release)_ #
+
+## What Is AsterixDB? ##
+
+Welcome to the new home of the AsterixDB Big Data Management System (BDMS).
+The AsterixDB BDMS is the result of about 3.5 years of R&D involving researchers at UC Irvine, UC Riverside, and UC San Diego.
+The AsterixDB code base now consists of roughly 250K lines of Java code that has been co-developed at UC Irvine and UC Riverside.
+
+Initiated in 2009, the NSF-sponsored ASTERIX project has been developing new technologies for ingesting, storing, managing, indexing, querying, and analyzing vast quantities of semi-structured information.
+The project has been combining ideas from three distinct areas---semi-structured data, parallel databases, and data-intensive computing (a.k.a. today's Big Data platforms)---in order to create a next-generation, open-source software platform that scales by running on large, shared-nothing commodity computing clusters.
+The ASTERIX effort has been targeting a wide range of semi-structured information, ranging from "data" use cases---where information is well-typed and highly regular---to "content" use cases---where data tends to be irregular, much of each datum may be textual, and the ultimate schema for the various data types involved may be hard to anticipate up front.
+The ASTERIX project has been addressing technical issues including highly scalable data storage and indexing, semi-structured query processing on very large clusters, and merging time-tested parallel database techniques with modern data-intensive computing techniques to support performant yet declarative solutions to the problem of storing and analyzing semi-structured information effectively.
+The first fruits of this labor have been captured in the AsterixDB system that is now being released in preliminary or "Alpha" release form.
+We are hoping that the arrival of AsterixDB will mark the beginning of the "BDMS era", and we hope that both the Big Data community and the database community will find the AsterixDB system to be interesting and useful for a much broader class of problems than can be addressed with any one of today's current Big Data platforms and related technologies (e.g., Hadoop, Pig, Hive, HBase, MongoDB, and so on).  One of our project mottos has been "one size fits a bunch"---at least that has been our aim.  For more information about the research effort that led to the birth of AsterixDB, please refer to our NSF project web site: [http://asterix.ics.uci.edu/](http://asterix.ics.uci.edu/).
+
+In a nutshell, AsterixDB is a full-function BDMS with a rich feature set that distinguishes it from pretty much any other Big Data platform that's out and available today.  We believe that its feature set makes it well-suited to modern needs such as web data warehousing and social data storage and analysis.  AsterixDB has:
+
+ * A semistructured NoSQL style data model (ADM) resulting from extending JSON with object database ideas
+ * An expressive and declarative query language (AQL) that supports a broad range of queries and analysis over semistructured data
+ * A parallel runtime query execution engine, Hyracks, that has been scale-tested on up to 1000+ cores and 500+ disks
+ * Partitioned LSM-based data storage and indexing to support efficient ingestion and management of semistructured data
+ * Support for query access to externally stored data (e.g., data in HDFS) as well as to data stored natively by AsterixDB
+ * A rich set of primitive data types, including spatial and temporal data in addition to integer, floating point, and textual data
+ * Secondary indexing options that include B+ trees, R trees, and inverted keyword (exact and fuzzy) index types
+ * Support for fuzzy and spatial queries as well as for more traditional parametric queries
+ * Basic transactional (concurrency and recovery) capabilities akin to those of a NoSQL store
+
+## Getting and Using AsterixDB ##
+
+You are most likely here because you are interested in getting your hands on AsterixDB---so you would like to know how to get it, how to set it up, and how to use it.
+Someday our plan is to have comprehensive documentation for AsterixDB and its data model (ADM) and query language (AQL) here on this wiki.
+For the Alpha release, we've got a start; for the Beta release a month or so from now, we will hopefully have much more.
+The following is a list of the wiki pages and supporting documents that we have available today:
+
+1. [InstallingAsterixUsingManagix](InstallingAsterixUsingManagix.html) :
+This is our installation guide, and it is where you should start.
+This document will tell you how to obtain, install, and manage instances of [AsterixDB](https://asterixdb.googlecode.com/files/asterix-installer-0.0.4-binary-assembly.zip), including both single-machine setup (for developers) as well as cluster installations (for deployment in its intended form).
+
+2. [AdmAql101](AdmAql101.html) :
+This is a first-timers introduction to the user model of the AsterixDB BDMS, by which we mean the view of AsterixDB as seen from the perspective of an "average user" or Big Data application developer.
+The AsterixDB user model consists of its data modeling features (ADM) and its query capabilities (AQL).
+This document presents a tiny "social data warehousing" example and uses it as a backdrop for describing, by example, the key features of AsterixDB.
+By working through this document, you will learn how to define the artifacts needed to manage data in AsterixDB, how to load data into the system, how to use most of the basic features of its query language, and how to insert and delete data dynamically.
+
+3. [AsterixDataTypesAndFunctions](AsterixDataTypesAndFunctions.html) :
+This is a reference document that catalogs the primitive data types and built-in functions available for use in AsterixDB schemas (in ADM) and queries (in AQL).
+
+4. [AQL Reference](AsterixQueryLanguageReference.html) :
+This is the AQL language reference manual.
+
+5. [AsterixDBRestAPI](AsterixDBRestAPI.html) :
+Access to data in an AsterixDB instance is provided via a REST-based API.
+This is a short document that describes the REST API entry points and their URL syntax.
+
+To all who have now come this far: Thanks for your interest in AsterixDB, and for kicking its tires in its Alpha form
+In addition to getting the system and trying it out, please sign up as a member of the AsterixDB user mailing list (asterixdb-users (at) googlegroups.com) so that you can contact us easily with your questions, issues, and other feedback.
+We want AsterixDB to be a "big hit" some day, and we are anxious to see what users do with it and to learn from that feedback what we should be working on most urgently in the next phase of the project.
diff --git a/asterix-doc/src/site/markdown/AsterixDBDataModel.md b/asterix-doc/src/site/markdown/AsterixDBDataModel.md
new file mode 100644
index 0000000..ffefd09
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AsterixDBDataModel.md
@@ -0,0 +1,291 @@
+# Asterix Data Model (ADM) #
+
+# Basic data types #
+
+An instance of Asterix data model (ADM) can be a _primitive type_ (`Int32`, `Int64`, `String`, `Float`, `Double`, `Date`, `Time`, `Datetime`, etc. or `NULL`) or a _derived type_.
+
+## Primitive Types ##
+
+### Boolean ###
+`Boolean` data type can have one of the two values: _*true*_ or _*false*_.
+
+ * Example:
+
+        let $t := true
+        let $f := false
+        return { "true": $t, "false": $f }
+
+
+ * The expected result is:
+
+        { "true": true, "false": false }
+
+
+
+### Int8 / Int16 / Int32 / Int64 ###
+Integer types using 8, 16, 32, or 64 bits. The ranges of these types are:
+
+- `Int8`: -127 to 127
+- `Int16`: -32767 to 32767
+- `Int32`: -2147483647 to 2147483647
+- `Int64`: -9223372036854775808 to 9223372036854775807
+
+ * Example:
+
+        let $v8 := int8("125")
+        let $v16 := int16("32765")
+        let $v32 := 294967295
+        let $v64 := int64("1700000000000000000")
+        return { "int8": $v8, "int16": $v16, "int32": $v32, "int64": $v64}
+
+
+ * The expected result is:
+
+        { "int8": 125i8, "int16": 32765i16, "int32": 294967295, "int64": 1700000000000000000i64 }
+
+
+### Float ###
+`Float` represents approximate numeric data values using 4 bytes. The range of a float value can be from 2^(-149) to (2-2^(-23)·2^(127) for both positive and negative. Beyond these ranges will get `INF` or `-INF`.
+
+ * Example:
+
+        let $v1 := float("NaN")
+        let $v2 := float("INF")
+        let $v3 := float("-INF")
+        let $v4 := float("-2013.5")
+        return { "v1": $v1, "v2": $v2, "v3": $v3, "v4": $v4 }
+
+
+ * The expected result is:
+
+        { "v1": NaNf, "v2": Infinityf, "v3": -Infinityf, "v4": -2013.5f }
+
+
+### Double ###
+`Double` represents approximate numeric data values using 8 bytes. The range of a double value can be from (2^(-1022)) to (2-2^(-52))·2^(1023) for both positive and negative. Beyond these ranges will get `INF` or `-INF`.
+
+ * Example:
+
+        let $v1 := double("NaN")
+        let $v2 := double("INF")
+        let $v3 := double("-INF")
+        let $v4 := double("-2013.593823748327284")
+        return { "v1": $v1, "v2": $v2, "v3": $v3, "v4": $v4 }
+
+
+ * The expected result is:
+
+        { "v1": NaNd, "v2": Infinityd, "v3": -Infinityd, "v4": -2013.5938237483274d }
+
+
+### String ###
+`String` represents a sequence of characters.
+
+ * Example:
+
+        let $v1 := string("This is a string.")
+        let $v2 := string("\"This is a quoted string\"")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": "This is a string.", "v2": "\"This is a quoted string\"" }
+
+
+### Point ###
+`Point` is the fundamental two-dimensional building block for spatial types. It consists of two `double` coordinates x and y.
+
+ * Example:
+
+        let $v1 := point("80.10d, -10E5")
+        let $v2 := point("5.10E-10d, -10E5")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": point("80.1,-1000000.0"), "v2": point("5.1E-10,-1000000.0") }
+
+
+### Line ###
+`Line` consists of two points that represent the start and the end points of a line segment.
+
+ * Example:
+
+        let $v1 := line("10.1234,11.1e-1 +10.2E-2,-11.22")
+        let $v2 := line("0.1234,-1.00e-10 +10.5E-2,-01.02")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": line("10.1234,1.11 0.102,-11.22"), "v2": line("0.1234,-1.0E-10 0.105,-1.02") }
+
+
+### Rectangle ###
+`Rectangle` consists of two points that represent the _*bottom left*_ and _*upper right*_ corners of a rectangle.
+
+ * Example:
+
+        let $v1 := rectangle("5.1,11.8 87.6,15.6548")
+        let $v2 := rectangle("0.1234,-1.00e-10 5.5487,0.48765")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": rectangle("5.1,11.8 87.6,15.6548"), "v2": rectangle("0.1234,-1.0E-10 5.5487,0.48765") }
+
+
+### Circle ###
+`Circle` consists of one `point` that represents the center of the circle and a radius of type `double`.
+
+ * Example:
+
+        let $v1 := circle("10.1234,11.1e-1 +10.2E-2")
+        let $v2 := circle("0.1234,-1.00e-10 +10.5E-2")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": circle("10.1234,1.11 0.102"), "v2": circle("0.1234,-1.0E-10 0.105") }
+
+
+### Polygon ###
+`Polygon` consists of _*n*_ points that represent the vertices of a _*simple closed*_ polygon.
+
+ * Example:
+
+        let $v1 := polygon("-1.2,+1.3e2 -2.14E+5,2.15 -3.5e+2,03.6 -4.6E-3,+4.81")
+        let $v2 := polygon("-1.0,+10.5e2 -02.15E+50,2.5 -1.0,+3.3e3 -2.50E+05,20.15 +3.5e+2,03.6 -4.60E-3,+4.75 -2,+1.0e2 -2.00E+5,20.10 30.5,03.25 -4.33E-3,+4.75")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": polygon("-1.2,130.0 -214000.0,2.15 -350.0,3.6 -0.0046,4.81"), "v2": polygon("-1.0,1050.0 -2.15E50,2.5 -1.0,3300.0 -250000.0,20.15 350.0,3.6 -0.0046,4.75 -2.0,100.0 -200000.0,20.1 30.5,3.25 -0.00433,4.75") }
+
+
+### Date ###
+`Date` represents a time point along the Gregorian calendar system specified by the year, month and day. ASTERIX supports the date from `-9999-01-01` to `9999-12-31`.
+
+A date value can be represented in two formats, extended format and basic format.
+
+ * Extended format is represented as `[-]yyyy-mm-dd` for `year-month-day`. Each field should be padded if there are less digits than the format specified.
+ * Basic format is in the format of `[-]yyyymmdd`.
+
+ * Example:
+
+        let $v1 := date("2013-01-01")
+        let $v2 := date("-19700101")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": date("2013-01-01"), "v2": date("-1970-01-01") }
+
+
+### Time ###
+`Time` type describes the time within the range of a day. It is represented by three fields: hour, minute and second. Millisecond field is optional as the fraction of the second field. Its extended format is as `hh:mm:ss[.mmm]` and the basic format is `hhmmss[mmm]`. The value domain is from `00:00:00.000` to `23:59:59.999`.
+
+Timezone field is optional for a time value. Timezone is represented as `[+|-]hh:mm` for extended format or `[+|-]hhmm` for basic format. Note that the sign designators cannot be omitted. `Z` can also be used to represent the UTC local time. If no timezone information is given, it is UTC by default.
+
+ * Example:
+
+        let $v1 := time("12:12:12.039Z")
+        let $v2 := time("000000000-0800")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": time("12:12:12.039Z"), "v2": time("08:00:00.000Z") }
+
+
+### Datetime ###
+A `Datetime` value is a combination of an `Date` and `Time`, representing a fixed time point along the Gregorian calendar system. The value is among `-9999-01-01 00:00:00.000` and `9999-12-31 23:59:59.999`.
+
+A `Datetime` value is represented as a combination of the representation of its `Date` part and `Time` part, separated by a separator `T`. Either extended or basic format can be used, and the two parts should be the same format.
+
+Millisecond field and timezone field are optional, as specified in the `Time` type.
+
+ * Example:
+
+        let $v1 := datetime("2013-01-01T12:12:12.039Z")
+        let $v2 := datetime("-19700101T000000000-0800")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": datetime("2013-01-01T12:12:12.039Z"), "v2": datetime("-1970-01-01T08:00:00.000Z") }
+
+
+### Duration ###
+`Duration` represents a duration of time. A duration value is specified by integers on at least one of the following fields: year, month, day, hour, minute, second, and millisecond.
+
+A duration value is in the format of `[-]PnYnMnDTnHnMn.mmmS`. The millisecond part (as the fraction of the second field) is optional, and when no millisecond field is used, the decimal point should also be absent.
+
+Negative durations are also supported for the arithmetic operations between time instance types (`Date`, `Time` and `Datetime`), and is used to roll the time back for the given duration. For example `date("2012-01-01") + duration("-P3D")` will return `date("2011-12-29")`.
+
+Note that a canonical representation of the duration is always returned, regardless whether the duration is in the canonical representation or not from the user's input. More information about canonical representation can be found from [XPath dayTimeDuration Canonical Representation](http://www.w3.org/TR/xpath-functions/#canonical-dayTimeDuration) and [yearMonthDuration Canonical Representation](http://www.w3.org/TR/xpath-functions/#canonical-yearMonthDuration).
+
+ * Example:
+
+        let $v1 := duration("P100Y12MT12M")
+        let $v2 := duration("-PT20.943S")
+        return { "v1": $v1, "v2": $v2 }
+
+
+ * The expected result is:
+
+        { "v1": duration("P101YT12M"), "v2": duration("-PT20.943S") }
+
+
+### Interval ###
+`Interval` represents inclusive-exclusive ranges of time. It is defined by two time point values with the same temporal type(`Date`, `Time` or `Datetime`).
+
+ * Example:
+
+        let $v1 := interval-from-date(date("2013-01-01"), date("20130505"))
+        let $v2 := interval-from-time(time("00:01:01"), time("213901049+0800"))
+        let $v3 := interval-from-datetime(datetime("2013-01-01T00:01:01"), datetime("20130505T213901049+0800"))
+        return { "v1": $v1, "v2": $v2, "v3": $v3 }
+
+
+ * The expected result is:
+
+        { "v1": interval-date("2013-01-01, 2013-05-05"), "v2": interval-time("00:01:01.000Z, 13:39:01.049Z"), "v3": interval-datetime("2013-01-01T00:01:01.000Z, 2013-05-05T13:39:01.049Z") }
+
+
+## Derived Types ##
+
+### Record ###
+A `Record` contains a set of fields, where each field is described by its name and type. A record type is either open or closed. Open records can contain fields that are not part of the type definition, while closed records cannot. Syntactically, record constructors are surrounded by curly braces "{...}".
+
+An example would be
+
+
+        { "id": 213508, "name": "Alice Bob" }
+
+
+### OrderedList ###
+An `OrderedList` is a sequence of values for which the order is determined by creation or insertion. OrderedList constructors are denoted by brackets: "[...]".
+
+An example would be
+
+
+        ["alice", 123, "bob", null]
+
+
+### UnorderedList ###
+An `UnorderedList` is an unordered sequence of values, similar to bags in SQL. UnorderedList constructors are denoted by two opening flower braces followed by data and two closing flower braces, like "{{...}}".
+
+An example would be
+
+
+        {{"hello", 9328, "world", [1, 2, null]}}
+
diff --git a/asterix-doc/src/site/markdown/AsterixDBFunctions.md b/asterix-doc/src/site/markdown/AsterixDBFunctions.md
new file mode 100644
index 0000000..d47752c
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AsterixDBFunctions.md
@@ -0,0 +1,1910 @@
+# Asterix: Using Functions #
+Asterix provides rich support of various classes of functions to support operations on string, spatial, and temporal data.  This document explains how to use these functions.
+
+## String Functions ##
+### string-to-codepoint ###
+ * Syntax:
+
+        string-to-codepoint(string_expression)
+
+ * Converts the string `string_expression` to its code-based representation.
+ * Arguments:
+   * `string_expression` : A `String` that will be converted.
+ * Return Value:
+   * An `OrderedList` of the code points for the string `string_expression`.
+
+### codepoint-to-string ###
+ * Syntax:
+
+        codepoint-to-string(list_expression)
+
+ * Converts the ordered code-based representation `list_expression` to the corresponding string.
+ * Arguments:
+   * `list_expression` : An `OrderedList` of code-points.
+ * Return Value:
+   * A `String` representation of `list_expression`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $s := "Hello ASTERIX!"
+        let $l := string-to-codepoint($s)
+        let $ss := codepoint-to-string($l)
+        return {"codes": $l, "string": $ss}
+
+
+ * The expected result is:
+
+        { "codes": [ 72, 101, 108, 108, 111, 32, 65, 83, 84, 69, 82, 73, 88, 33 ], "string": "Hello ASTERIX!" }
+
+
+### contains ###
+ * Syntax:
+
+        contains(string_expression, string_pattern)
+
+ * Checks whether the string `string_expression` contains the string `string_pattern`
+ * Arguments:
+   * `string_expression` : A `String` that might contain the pattern.
+   * `string_pattern` : A target `String` that might be contained.
+ * Return Value:
+   * A `Boolean`, returns `true` if `string_expression` contains `string_pattern`, otherwise returns `false`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where contains($i.message, "phone")
+        return {"mid": $i.message-id, "message": $i.message}
+
+
+ * The expected result is:
+
+        { "mid": 2, "message": " dislike iphone its touch-screen is horrible" }
+        { "mid": 13, "message": " dislike iphone the voice-command is bad:(" }
+        { "mid": 15, "message": " like iphone the voicemail-service is awesome" }
+
+
+### len ###
+ * Syntax:
+
+        len(list_expression)
+
+ * Returns the length of the list `list_expression`.
+ * Arguments:
+   * `list_expression` : An `OrderedList`, `UnorderedList` or `NULL`, represents the list need to be checked.
+ * Return Value:
+   * An `Int32` that represents the length of `list_expression`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $l := ["ASTERIX", "Hyracks"]
+        return len($l)
+
+
+ * The expected result is:
+
+        2
+
+
+### like ###
+ * Syntax:
+
+        like(string_expression, string_pattern)
+
+ * Checks whether the string `string_expression` contains the string `string_pattern`.
+ * Arguments:
+   * `string_expression` : A `String` that might contain the pattern or `NULL`.
+   * `string_pattern` : A pattern `String` that might be contained or `NULL`.
+ * Return Value:
+   * A `Boolean`, returns `true` if `string_expression` contains the pattern `string_pattern`, otherwise returns `false`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where like($i.message, "%at&t%")
+        return $i.message
+
+
+ * The expected result is:
+
+        " can't stand at&t the network is horrible:("
+        " can't stand at&t its plan is terrible"
+        " love at&t its 3G is good:)"
+
+
+### starts-with ###
+ * Syntax:
+
+        starts-with(string_expression, string_pattern)
+
+ * Checks whether the string `string_expression` starts with the string `string_pattern`.
+ * Arguments:
+   * `string_expression` : A `String` that might start with the given string.
+   * `string_pattern` : A `String` that might be contained as the starting substring.
+ * Return Value:
+   * A `Boolean`, returns `true` if `string_expression` starts with the string `string_pattern`, otherwise returns `false`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where starts-with($i.message, " like")
+        return $i.message
+
+
+ * The expected result is:
+
+        " like samsung the plan is amazing"
+        " like t-mobile its platform is mind-blowing"
+        " like verizon the 3G is awesome:)"
+        " like iphone the voicemail-service is awesome"
+
+
+### ends-with ###
+ * Syntax:
+
+        ends-with(string_expression, string_pattern)
+
+ * Checks whether the string `string_expression` ends with the string `string_pattern`.
+ * Arguments:
+   * `string_expression` : A `String` that might end with the given string.
+   * `string_pattern` : A `String` that might be contained as the ending substring.
+ * Return Value:
+   * A `Boolean`, returns `true` if `string_expression` ends with the string `string_pattern`, otherwise returns `false`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where ends-with($i.message, ":)")
+        return $i.message
+
+
+ * The expected result is:
+
+        " love sprint its shortcut-menu is awesome:)"
+        " like verizon the 3G is awesome:)"
+        " love at&t its 3G is good:)"
+
+
+### string-concat ###
+ * Syntax:
+
+        string-concat(list_expression)
+
+ * Concatenates a list of strings `list_expression` into a single string.
+ * Arguments:
+   * `list_expression` : An `OrderedList` or `UnorderedList` of `String`s (could be `NULL`) to be concatenated.
+ * Return Value:
+   * Returns the concatenated `String` value.
+
+ * Example:
+
+        let $i := "ASTERIX"
+        let $j := " "
+        let $k := "ROCKS!"
+        return string-concat([$i, $j, $k])
+
+
+ * The expected result is:
+
+        "ASTERIX ROCKS!"
+
+
+### string-equal ###
+ * Syntax:
+
+        string-equal(string_expression1, string_expression2)
+
+ * Checks whether the strings `string_expression1` and `string_expression2` are equal.
+ * Arguments:
+   * `string_expression1` : A `String` to be compared.
+   * `string_expression2` : A `String` to be compared with.
+ * Return Value:
+   * A `Boolean`, returns `true` if `string_expression1` and `string_expression2` are equal, otherwise returns `false`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $i := "Android"
+        return {"Equal": string-equal($i, "Android"), "NotEqual": string-equal($i, "iphone")}
+
+
+ * The expected result is:
+
+        { "Equal": true, "NotEqual": false }
+
+
+### string-join ###
+ * Syntax:
+
+        string-join(list_expression, string_expression)
+
+ * Joins a list of strings `list_expression` with the given separator `string_expression` into a single string.
+ * Arguments:
+   * `list_expression` : An `OrderedList` or `UnorderedList` of `String`s (could be NULL) to be joined.
+   * `string_expression` : A separator `String` value.
+ * Return Value:
+   * Returns the joined `String`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $i := ["ASTERIX", "ROCKS~"]
+        return string-join($i, "!! ")
+
+
+ * The expected result is:
+
+        "ASTERIX!! ROCKS~"
+
+
+### lowercase ###
+ * Syntax:
+
+        lowercase(string_expression)
+
+ * Returns the lowercase of a given string `string_expression`.
+ * Arguments:
+   * `string_expression` : A `String` to be lowercased.
+ * Return Value:
+   * Returns the lowercased `String`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $i := "ASTERIX"
+        return lowercase($i)
+
+
+ * The expected result is:
+
+        asterix
+
+
+### matches ###
+ * Syntax:
+
+        matches(string_expression, string_pattern)
+
+ * Checks whether the strings `string_expression` matches the given pattern `string_pattern`.
+ * Arguments:
+   * `string_expression` : A `String` that might contain the pattern.
+   * `string_pattern` : A pattern `String` to be matched.
+ * Return Value:
+   * A `Boolean`, returns `true` if `string_expression` matches the pattern `string_pattern`, otherwise returns `false`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where matches($i.message, "dislike iphone")
+        return $i.message
+
+
+ * The expected result is:
+
+        " dislike iphone its touch-screen is horrible"
+        " dislike iphone the voice-command is bad:("
+
+
+### replace ###
+ * Syntax:
+
+        replace(string_expression, string_pattern, string_replacement)
+
+ * Checks whether the strings `string_expression` matches the given pattern `string_pattern`, and replace the matched pattern `string_pattern` with the new pattern `string_replacement`.
+ * Arguments:
+   * `string_expression` : A `String` that might contain the pattern.
+   * `string_pattern` : A pattern `String` to be matched.
+   * `string_replacement` : A pattern `String` to be used as the replacement.
+ * Return Value:
+   * Returns a `String` that is obtained after the replacements.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where matches($i.message, " like iphone")
+        return replace($i.message, " like iphone", "like android")
+
+
+ * The expected result is:
+
+        "like android the voicemail-service is awesome"
+
+
+### string-length ###
+ * Syntax:
+
+        string-length(string_expression)
+
+ * Returns the length of the string `string_expression`.
+ * Arguments:
+   * `string_expression` : A `String` or `NULL`, represents the string to be checked.
+ * Return Value:
+   * An `Int32` that represents the length of `string_expression`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        return {"mid": $i.message-id, "message-len": string-length($i.message)}
+
+
+ * The expected result is:
+
+        { "mid": 1, "message-len": 43 }
+        { "mid": 2, "message-len": 44 }
+        { "mid": 3, "message-len": 33 }
+        { "mid": 4, "message-len": 43 }
+        { "mid": 5, "message-len": 46 }
+        { "mid": 6, "message-len": 43 }
+        { "mid": 7, "message-len": 37 }
+        { "mid": 8, "message-len": 33 }
+        { "mid": 9, "message-len": 34 }
+        { "mid": 10, "message-len": 50 }
+        { "mid": 11, "message-len": 38 }
+        { "mid": 12, "message-len": 52 }
+        { "mid": 13, "message-len": 42 }
+        { "mid": 14, "message-len": 27 }
+        { "mid": 15, "message-len": 45 }
+
+
+### substring ###
+ * Syntax:
+
+        substring(string_expression, offset, length)
+
+ * Returns the substring from the given string `string_expression` based on the given start offset `offset`.
+ * Arguments:
+   * `string_expression` : A `String` as the string to be extracted.
+   * `offset` : An `Int32` as the starting offset of the substring in `string_expression`.
+   * `length` : (Optional) An `Int32` as the length of the substring.
+ * Return Value:
+   * A `String` that represents the substring.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where string-length($i.message) > 50
+        return substring($i.message, 50)
+
+
+ * The expected result is:
+
+        "G:("
+
+
+### substring-before ###
+ * Syntax:
+
+        substring-before(string_expression, string_pattern)
+
+ * Returns the substring from the given string `string_expression` before the given pattern `string_pattern`.
+ * Arguments:
+   * `string_expression` : A `String` as the string to be extracted.
+   * `string_pattern` : A `String` as the string pattern to be searched.
+ * Return Value:
+   * A `String` that represents the substring.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where contains($i.message, "iphone")
+        return substring-before($i.message, "iphone")
+
+
+ * The expected result is:
+
+        " dislike "
+        " dislike "
+        " like "
+
+
+### substring-after ###
+ * Syntax:
+
+        substring-after(string_expression, string_pattern)
+
+ * Returns the substring from the given string `string_expression` after the given pattern `string_pattern`.
+ * Arguments:
+   * `string_expression` : A `String` as the string to be extracted.
+   * `string_pattern` : A `String` as the string pattern to be searched.
+ * Return Value:
+   * A `String` that represents the substring.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookMessages')
+        where contains($i.message, "iphone")
+        return substring-after($i.message, "iphone")
+
+
+ * The expected result is:
+
+        " its touch-screen is horrible"
+        " the voice-command is bad:("
+        " the voicemail-service is awesome"
+
+
+## Spatial Functions ##
+### create-point ###
+ * Syntax:
+
+        create-point(latitude, longitude)
+
+ * Creates the primitive type `Point` using `latitude` and `longitude`.
+ * Arguments:
+   * `latitude` : A `Double` that represents the latitude.
+   * `longitude` : A `Double` that represents the longitude.
+ * Return Value:
+   * A `Point`, represents a spatial point created using the latitude and longitude provided in `latitude` and `longitude`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $c :=  create-point(30.0,70.0)
+        return {"point": $c}
+
+
+ * The expected result is:
+
+        { "point": point("30.0,70.0") }
+
+
+### create-line ###
+ * Syntax:
+
+        create-line(point_expression1, point_expression2)
+
+ * Creates the primitive type `Line` using `point_expression1` and `point_expression2`.
+ * Arguments:
+   * `point_expression1` : A `Point` that represents the start point of the line.
+   * `point_expression2` : A `Point` that represents the end point of the line.
+ * Return Value:
+   * A `Line`, represents a spatial line created using the points provided in `point_expression1` and `point_expression2`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $c :=  create-line(create-point(30.0,70.0), create-point(50.0,90.0))
+        return {"line": $c}
+
+
+ * The expected result is:
+
+        { "line": line("30.0,70.0 50.0,90.0") }
+
+
+### create-rectangle ###
+ * Syntax:
+
+        create-rectangle(point_expression1, point_expression2)
+
+ * Creates the primitive type Rectangle using `point_expression1` and `point_expression2`.
+ * Arguments:
+   * `point_expression1` : A `Point` that represents the lower-left point of the rectangle.
+   * `point_expression2` : A `Point` that represents the upper-right point of the rectangle.
+ * Return Value:
+   * A `Rectangle`, represents a spatial rectangle created using the points provided in `point_expression1` and `point_expression2`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $c :=  create-rectangle(create-point(30.0,70.0), create-point(50.0,90.0))
+        return {"rectangle": $c}
+
+
+ * The expected result is:
+
+        { "rectangle": rectangle("30.0,70.0 50.0,90.0") }
+
+
+### create-circle ###
+ * Syntax:
+
+        create-circle(point_expression, radius)
+
+ * Creates the primitive type `Circle` using `point_expression` and `radius`.
+ * Arguments:
+   * `point_expression` : A `Point` that represents the center of the circle.
+   * `radius` : A `Double` that represents the radius of the circle.
+ * Return Value:
+   * A `Circle`, represents a spatial circle created using the center point and the radius provided in `point_expression` and `radius`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $c :=  create-circle(create-point(30.0,70.0), 5.0)
+        return {"circle": $c}
+
+
+ * The expected result is:
+
+        { "circle": circle("30.0,70.0 5.0") }
+
+
+### create-polygon ###
+ * Syntax:
+
+        create-polygon(point_expression1, point_expression2, ..., point_expressionn)
+
+ * Creates the primitive type `Polygon` using unlimited number of arguments `point_expression1`, `point_expression2`, ..., `point_expressionn`.
+ * Arguments:
+   * `point_expression1`/.../`point_expressionn` : A `Point` that represents a vertex of the polygon.
+ * Return Value:
+   * A `Polygon`, represents a spatial simple polygon created using the points provided in `point_expression1`, `point_expression2`, ..., `point_expressionn`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $c :=  create-polygon(create-point(1.0,1.0), create-point(2.0,2.0), create-point(3.0,3.0), create-point(4.0,4.0))
+        return {"polygon": $c}
+
+
+ * The expected result is:
+
+        { "polygon": polygon("1.0,1.0 2.0,2.0 3.0,3.0 4.0,4.0") }
+
+
+### point ###
+ * Syntax:
+
+        point(string_expression)
+
+ * Constructor function for `Point` type by parsing a point string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a point value.
+ * Return Value:
+   * A `Point` value represented by the given string.
+
+ * Example:
+
+        
+        use dataverse TinySocial;
+        
+        let $c := point("55.05,-138.04")
+        return {"point": $c}
+
+
+ * The expected result is:
+
+        { "point": point("55.05,-138.04") }
+
+
+### line ###
+ * Syntax:
+
+        line(string_expression)
+
+ * Constructor function for `Line` type by parsing a line string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a line value.
+ * Return Value:
+   * A `Line` value represented by the given string.
+
+ * Example:
+
+        
+        use dataverse TinySocial;
+        
+        let $c := line("55.05,-138.04 13.54,-138.04")
+        return {"line": $c}
+
+
+ * The expected result is:
+
+        { "line": line("55.05,-138.04 13.54,-138.04") }
+
+
+### rectangle ###
+ * Syntax:
+
+        rectangle(string_expression)
+
+ * Constructor function for `Rectangle` type by parsing a rectangle string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a rectangle value.
+ * Return Value:
+   * A `Rectangle` value represented by the given string.
+
+ * Example:
+
+        
+        use dataverse TinySocial;
+        
+        let $c := rectangle("20.05,-125.0 40.67,-100.87")
+        return {"rectangle": $c}
+
+
+ * The expected result is:
+
+        { "rectangle": rectangle("20.05,-125.0 40.67,-100.87") }
+
+
+### circle ###
+ * Syntax:
+
+        circle(string_expression)
+
+ * Constructor function for `Circle` type by parsing a circle string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a circle value.
+ * Return Value:
+   * A `Circle` value represented by the given string.
+
+ * Example:
+
+        
+        use dataverse TinySocial;
+        
+        let $c := circle("55.05,-138.04 10.0")
+        return {"circle": $c}
+
+
+ * The expected result is:
+
+        { "circle": circle("55.05,-138.04 10.0") }
+
+
+### polygon ###
+ * Syntax:
+
+        polygon(string_expression)
+
+ * Constructor function for `Polygon` type by parsing a polygon string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a polygon value.
+ * Return Value:
+   * A `Polygon` value represented by the given string.
+
+ * Example:
+
+        
+        use dataverse TinySocial;
+        
+        let $c := polygon("55.05,-138.04 13.54,-138.04 13.54,-53.31 55.05,-53.31")
+        return {"polygon": $c}
+
+
+ * The expected result is:
+
+        { "polygon": polygon("55.05,-138.04 13.54,-138.04 13.54,-53.31 55.05,-53.31") }
+
+
+### get-x/get-y ###
+ * Syntax:
+
+        get-x(point_expression) or get-y(point_expression)
+
+ * Returns the x or y coordinates of a point `point_expression`.
+ * Arguments:
+   * `point_expression` : A `Point`.
+ * Return Value:
+   * A `Double`, represents the x or y coordinates of the point `point_expression`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $point := create-point(2.3,5.0)
+        return {"x-coordinate": get-x($point), "y-coordinate": get-y($point)}
+
+
+ * The expected result is:
+
+        { "x-coordinate": 2.3d, "y-coordinate": 5.0d }
+
+
+### get-points ###
+ * Syntax:
+
+        get-points(spatial_expression)
+
+ * Returns an ordered list of the points forming the spatial object `spatial_expression`.
+ * Arguments:
+   * `spatial_expression` : A `Point`, `Line`, `Rectangle`, `Circle`, or `Polygon`.
+ * Return Value:
+   * An `OrderedList` of the points forming the spatial object `spatial_expression`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $line := create-line(create-point(100.6,99.4), create-point(-72.0,-76.9))
+        let $rectangle := create-rectangle(create-point(9.2,49.0), create-point(77.8,111.1))
+        let $polygon := create-polygon(create-point(1.0,1.0), create-point(2.0,2.0), create-point(3.0,3.0), create-point(4.0,4.0))
+        let $line_list := get-points($line)
+        let $rectangle_list := get-points($rectangle)
+        let $polygon_list := get-points($polygon)
+        return {"line-first-point": $line_list[0], "line-second-point": $line_list[1], "rectangle-left-bottom-point": $rectangle_list[0], "rectangle-top-upper-point": $rectangle_list[1], "polygon-first-point": $polygon_list[0], "polygon-second-point": $polygon_list[1], "polygon-third-point": $polygon_list[2], "polygon-forth-point": $polygon_list[3]}
+
+
+ * The expected result is:
+
+        { "line-first-point": point("100.6,99.4"), "line-second-point": point("-72.0,-76.9"), "rectangle-left-bottom-point": point("9.2,49.0"), "rectangle-top-upper-point": point("77.8,111.1"), "polygon-first-point": point("1.0,1.0"), "polygon-second-point": point("2.0,2.0"), "polygon-third-point": point("3.0,3.0"), "polygon-forth-point": point("4.0,4.0") }
+
+
+### get-center/get-radius ###
+ * Syntax:
+
+        get-center(circle_expression) or get-radius(circle_expression)
+
+ * Returns the center and the radius of a circle `circle_expression`.
+ * Arguments:
+   * `circle_expression` : A `Circle`.
+ * Return Value:
+   * A `Point` or `Double`, represent the center or radius of the circle `circle_expression`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $circle := create-circle(create-point(6.0,3.0), 1.0)
+        return {"circle-radius": get-radius($circle), "circle-center": get-center($circle)}
+        
+
+
+ * The expected result is:
+
+        { "circle-radius": 1.0d, "circle-center": point("6.0,3.0") }
+
+
+
+### spatial-distance ###
+ * Syntax:
+
+        spatial-distance(point_expression1, point_expression2)
+
+ * Returns the euclidean distance between `point_expression1` and `point_expression2`.
+ * Arguments:
+   * `point_expression1` : A `Point`.
+   * `point_expression2` : A `Point`.
+ * Return Value:
+   * A `Double`, represents the euclidean distance between `point_expression1` and `point_expression2`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        let $d :=  spatial-distance($t.sender-location, create-point(30.0,70.0))
+        return {"point": $t.sender-location, "distance": $d}
+        
+
+
+ * The expected result is:
+
+        { "point": point("47.44,80.65"), "distance": 20.434678857275934d }
+        { "point": point("29.15,76.53"), "distance": 6.585089217315132d }
+        { "point": point("37.59,68.42"), "distance": 7.752709203884797d }
+        { "point": point("24.82,94.63"), "distance": 25.168816023007512d }
+        { "point": point("32.84,67.14"), "distance": 4.030533463451212d }
+        { "point": point("29.72,75.8"), "distance": 5.806754687430835d }
+        { "point": point("39.28,70.48"), "distance": 9.292405501268227d }
+        { "point": point("40.09,92.69"), "distance": 24.832321679617472d }
+        { "point": point("47.51,83.99"), "distance": 22.41250097601782d }
+        { "point": point("36.21,72.6"), "distance": 6.73231758015024d }
+        { "point": point("46.05,93.34"), "distance": 28.325926286707734d }
+        { "point": point("36.86,74.62"), "distance": 8.270671073135482d }
+
+
+### spatial-area ###
+ * Syntax:
+
+        spatial-distance(spatial_2d_expression)
+
+ * Returns the spatial area of `spatial_2d_expression`.
+ * Arguments:
+   * `spatial_2d_expression` : A `Rectangle`, `Circle`, or `Polygon`.
+ * Return Value:
+   * A `Double`, represents the area of `spatial_2d_expression`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $circleArea := spatial-area(create-circle(create-point(0.0,0.0), 5.0))
+        return {"Area":$circleArea}
+        
+
+
+ * The expected result is:
+
+        { "Area": 78.53981625d }
+
+
+### spatial-intersect ###
+ * Syntax:
+
+        spatial-intersect(spatial_expression1, spatial_expression2)
+
+ * Checks whether `@arg1` and `@arg2` spatially intersect each other.
+ * Arguments:
+   * `spatial_expression1` : A `Point`, `Line`, `Rectangle`, `Circle`, or `Polygon`.
+   * `spatial_expression2` : A `Point`, `Line`, `Rectangle`, `Circle`, or `Polygon`.
+ * Return Value:
+   * A `Boolean`, represents whether `spatial_expression1` and `spatial_expression2` spatially intersect each other.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        where spatial-intersect($t.sender-location, create-rectangle(create-point(30.0,70.0), create-point(40.0,80.0)))
+        return $t
+
+
+ * The expected result is:
+
+        { "tweetid": "4", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("39.28,70.48"), "send-time": datetime("2011-12-26T10:10:00.000Z"), "referred-topics": {{ "sprint", "voice-command" }}, "message-text": " like sprint the voice-command is mind-blowing:)" }
+        { "tweetid": "7", "user": { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136 }, "sender-location": point("36.21,72.6"), "send-time": datetime("2011-08-25T10:10:00.000Z"), "referred-topics": {{ "samsung", "platform" }}, "message-text": " like samsung the platform is good" }
+        { "tweetid": "9", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, "statuses_count": 473, "name": "Nathan Giesen", "followers_count": 49416 }, "sender-location": point("36.86,74.62"), "send-time": datetime("2012-07-21T10:10:00.000Z"), "referred-topics": {{ "verizon", "voicemail-service" }}, "message-text": " love verizon its voicemail-service is awesome" }
+
+
+### spatial-cell ###
+ * Syntax:
+
+        spatial-cell(point_expression1, point_expression2, x_increment, y_increment)
+
+ * Returns the grid cell that `point_expression1` belongs to.
+ * Arguments:
+   * `point_expression1` : A `Point`, represents the point of interest that its grid cell will be returned.
+   * `point_expression2` : A `Point`, represents the origin of the grid.
+   * `x_increment` : A `Double`, represents X increments.
+   * `y_increment` : A `Double`, represents Y increments.
+ * Return Value:
+   * A `Rectangle`, represents the grid cell that `point_expression1` belongs to.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        group by $c :=  spatial-cell($t.sender-location, create-point(20.0,50.0), 5.5, 6.0) with $t
+        let $num :=  count($t)
+        return { "cell": $c, "count": $num}
+
+
+ * The expected result is:
+
+        { "cell": rectangle("20.0,92.0 25.5,98.0"), "count": 1 }
+        { "cell": rectangle("25.5,74.0 31.0,80.0"), "count": 2 }
+        { "cell": rectangle("31.0,62.0 36.5,68.0"), "count": 1 }
+        { "cell": rectangle("31.0,68.0 36.5,74.0"), "count": 1 }
+        { "cell": rectangle("36.5,68.0 42.0,74.0"), "count": 2 }
+        { "cell": rectangle("36.5,74.0 42.0,80.0"), "count": 1 }
+        { "cell": rectangle("36.5,92.0 42.0,98.0"), "count": 1 }
+        { "cell": rectangle("42.0,80.0 47.5,86.0"), "count": 1 }
+        { "cell": rectangle("42.0,92.0 47.5,98.0"), "count": 1 }
+        { "cell": rectangle("47.5,80.0 53.0,86.0"), "count": 1 }
+
+
+
+
+## Similarity Functions ##
+
+AsterixDB supports queries with different similarity functions, including edit distance and Jaccard.
+
+### edit-distance ###
+ * Syntax:
+
+        edit-distance(expression1, expression2)
+
+ * Returns the [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) of `expression1` and `expression2`.
+ * Arguments:
+   * `expression1` : A `String` or a homogeneous `OrderedList` of a comparable item type.
+   * `expression2` : The same type as `expression1`.
+ * Return Value:
+   * An `Int32` that represents the edit-distance similarity of `expression1` and `expression2`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $ed := edit-distance($user.name, "Suzanna Tilson")
+        where $ed <= 2
+        return $user
+
+
+ * The expected result is:
+
+        {
+        "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000Z"), "friend-ids": {{ 6 }},
+        "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19"), "end-date": null } ]
+        }
+
+
+### edit-distance-check ###
+ * Syntax:
+
+        edit-distance-check(expression1, expression2, threshold)
+
+ * Checks whether `expression1` and `expression2` have a [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) `<= threshold`.  The “check” version of edit distance is faster than the "non-check" version because the former can detect whether two items satisfy a given similarity threshold using early-termination techniques, as opposed to computing their real distance. Although possible, it is not necessary for the user to write queries using the “check” versions explicitly, since a rewrite rule can perform an appropriate transformation from a “non-check” version to a “check” version.
+
+ * Arguments:
+   * `expression1` : A `String` or a homogeneous `OrderedList` of a comparable item type.
+   * `expression2` : The same type as `expression1`.
+   * `threshold` : An `Int32` that represents the distance threshold.
+ * Return Value:
+   * An `OrderedList` with two items:
+     * The first item contains a `Boolean` value representing whether `expression1` and `expression2` are similar.
+     * The second item contains an `Int32` that represents the edit distance of `expression1` and `expression2` if it is `<= `threshold`, or 0 otherwise.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $ed := edit-distance-check($user.name, "Suzanna Tilson", 2)
+        where $ed[0]
+        return $ed[1]
+
+
+ * The expected result is:
+
+        2
+
+
+### similarity-jaccard ###
+ * Syntax:
+
+        similarity-jaccard(list_expression1, list_expression2)
+
+ * Returns the [Jaccard similarity](http://en.wikipedia.org/wiki/Jaccard_index) of `list_expression1` and `list_expression2`.
+ * Arguments:
+   * `list_expression1` : An `UnorderedList` or `OrderedList`.
+   * `list_expression2` : An `UnorderedList` or `OrderedList`.
+ * Return Value:
+   * A `Float` that represents the Jaccard similarity of `list_expression1` and `list_expression2`.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $sim := similarity-jaccard($user.friend-ids, [1,5,9])
+        where $sim >= 0.6f
+        return $user
+
+
+ * The expected result is:
+
+        {
+        "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }},
+        "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ]
+        }
+        {
+        "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }},
+        "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ]
+        }
+
+
+### similarity-jaccard-check ###
+ * Syntax:
+
+        similarity-jaccard-check(list_expression1, list_expression2, threshold)
+
+ * Checks whether `list_expression1` and `list_expression2` have a [Jaccard similarity](http://en.wikipedia.org/wiki/Jaccard_index) `>= threshold`.  Again, the “check” version of Jaccard is faster than the "non-check" version.
+
+ * Arguments:
+   * `list_expression1` : An `UnorderedList` or `OrderedList`.
+   * `list_expression2` : An `UnorderedList` or `OrderedList`.
+   * `threshold` : A Float that represents the similarity threshold.
+ * Return Value:
+   * An `OrderedList` with two items:
+     * The first item contains a `Boolean` value representing whether `list_expression1` and `list_expression2` are similar.
+     * The second item contains a `Float` that represents the Jaccard similarity of `list_expression1` and `list_expression2` if it is >`= `threshold`, or 0 otherwise.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $sim := similarity-jaccard-check($user.friend-ids, [1,5,9], 0.6f)
+        where $sim[0]
+        return $sim[1]
+
+
+ * The expected result is:
+
+        0.75f
+        1.0f
+
+
+### Similarity Operator ~# ###
+ * "`~=`" is syntactic sugar for expressing a similarity condition with a given similarity threshold.
+ * The similarity function and threshold for "`~=`" are controlled via "set" directives.
+ * The "`~=`" operator returns a Boolean that represents whether the operands are similar.
+
+ * Example for Jaccard similarity:
+
+        use dataverse TinySocial;
+        
+        set simfunction "jaccard";
+        set simthreshold "0.6f";
+        
+        for $user in dataset('FacebookUsers')
+        where $user.friend-ids ~= [1,5,9]
+        return $user
+
+
+ * The expected result is:
+
+        {
+        "id": 3, "alias": "Emory", "name": "EmoryUnk", "user-since": datetime("2012-07-10T10:10:00.000Z"), "friend-ids": {{ 1, 5, 8, 9 }},
+        "employment": [ { "organization-name": "geomedia", "start-date": date("2010-06-17"), "end-date": date("2010-01-26") } ]
+        }
+        {
+        "id": 10, "alias": "Bram", "name": "BramHatch", "user-since": datetime("2010-10-16T10:10:00.000Z"), "friend-ids": {{ 1, 5, 9 }},
+        "employment": [ { "organization-name": "physcane", "start-date": date("2007-06-05"), "end-date": date("2011-11-05") } ]
+        }
+
+
+ * Example for edit-distance similarity:
+
+        use dataverse TinySocial;
+        
+        set simfunction "edit-distance";
+        set simthreshold "2";
+        
+        for $user in dataset('FacebookUsers')
+        where $user.name ~= "Suzanna Tilson"
+        return $user
+
+
+ * The expected output is:
+
+        {
+        "id": 7, "alias": "Suzanna", "name": "SuzannaTillson", "user-since": datetime("2012-08-07T10:10:00.000Z"), "friend-ids": {{ 6 }},
+        "employment": [ { "organization-name": "Labzatron", "start-date": date("2011-04-19"), "end-date": null } ]
+        }
+
+
+## Tokenizing Functions ##
+### word-tokens ###
+ * Syntax:
+
+        word-tokens(string_expression)
+
+ * Returns a list of word tokens of `string_expression`.
+ * Arguments:
+   * `string_expression` : A `String` that will be tokenized.
+ * Return Value:
+   * An `OrderedList` of `String` word tokens.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        let $tokens := word-tokens($t.message-text)
+        where $t.send-time >= datetime('2012-01-01T00:00:00')
+        return {
+        "tweetid": $t.tweetid,
+        "word-tokens": $tokens
+        }
+
+
+ * The expected result is:
+
+        { "tweetid": "9", "word-tokens": [ "love", "verizon", "its", "voicemail", "service", "is", "awesome" ] }
+
+
+### hashed-word-tokens ###
+ * Syntax:
+
+        hashed-word-tokens(string_expression)
+
+ * Returns a list of hashed word tokens of `string_expression`.
+ * Arguments:
+   * `string_expression` : A `String` that will be tokenized.
+ * Return Value:
+   * An `OrderedList` of Int32 hashed tokens.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        let $tokens := hashed-word-tokens($t.message-text)
+        where $t.send-time >= datetime('2012-01-01T00:00:00')
+        return {
+        "tweetid": $t.tweetid,
+        "hashed-word-tokens": $tokens
+        }
+
+
+ * The expected result is:
+
+        { "tweetid": "9", "hashed-word-tokens": [ -1217719622, -447857469, -1884722688, -325178649, 210976949, 285049676, 1916743959 ] }
+
+
+### counthashed-word-tokens ###
+ * Syntax:
+
+        counthashed-word-tokens(string_expression)
+
+ * Returns a list of hashed word tokens of `string_expression`. The hashing mechanism gives duplicate tokens different hash values, based on the occurrence count of that token.
+ * Arguments:
+   * `string_expression` : A `String` that will be tokenized.
+ * Return Value:
+   * An `OrderedList` of `Int32` hashed tokens.
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        let $tokens := counthashed-word-tokens($t.message-text)
+        where $t.send-time >= datetime('2012-01-01T00:00:00')
+        return {
+        "tweetid": $t.tweetid,
+        "counthashed-word-tokens": $tokens
+        }
+
+
+ * The expected result is:
+
+        { "tweetid": "9", "counthashed-word-tokens": [ -1217719622, -447857469, -1884722688, -325178649, 210976949, 285049676, 1916743959 ] }
+
+
+### gram-tokens ###
+ * Syntax:
+
+        gram-tokens(string_expression, gram_length, boolean_expression)
+
+ * Returns a list of gram tokens of `string_expression`, which can be obtained by scanning the characters using a sliding window of a fixed length.
+ * Arguments:
+   * `string_expression` : A `String` that will be tokenized.
+   * `gram_length` : An `Int32` as the length of grams.
+   * `boolean_expression` : A `Boolean` value to indicate whether to generate additional grams by pre- and postfixing `string_expression` with special characters.
+ * Return Value:
+   * An `OrderedList` of String gram tokens.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        let $tokens := gram-tokens($t.message-text, 3, true)
+        where $t.send-time >= datetime('2012-01-01T00:00:00')
+        return {
+        "tweetid": $t.tweetid,
+        "gram-tokens": $tokens
+        }
+
+
+ * The expected result is:
+
+        {
+        "tweetid": "9",
+        "gram-tokens": [ "## ", "# l", " lo", "lov", "ove", "ve ", "e v", " ve", "ver", "eri", "riz", "izo", "zon", "on ", "n i", " it", "its", "ts ", "s v", " vo", "voi", "oic", "ice",
+        "cem", "ema", "mai", "ail", "il-", "l-s", "-se", "ser", "erv", "rvi", "vic", "ice", "ce ", "e i", " is", "is ", "s a", " aw", "awe", "wes", "eso", "som", "ome", "me$", "e$$" ]
+        }
+
+
+### hashed-gram-tokens ###
+ * Syntax:
+
+        hashed-gram-tokens(string_expression, gram_length, boolean_expression)
+
+ * Returns a list of hashed gram tokens of `string_expression`.
+ * Arguments:
+   * `string_expression` : A `String` that will be tokenized.
+   * `gram_length` : An `Int32` as the length of grams.
+   * `boolean_expression` : A `Boolean` to indicate whether to generate additional grams by pre- and postfixing `string_expression` with special characters.
+ * Return Value:
+   * An `OrderedList` of `Int32` hashed gram tokens.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        let $tokens := hashed-gram-tokens($t.message-text, 3, true)
+        where $t.send-time >= datetime('2012-01-01T00:00:00')
+        return {
+        "tweetid": $t.tweetid,
+        "hashed-gram-tokens": $tokens
+        }
+
+
+ * The expected result is:
+
+        {
+        "tweetid": "9",
+        "hashed-gram-tokens": [ 40557178, -2002241593, 161665899, -856104603, -500544946, 693410611, 395674299, -1015235909, 1115608337, 1187999872, -31006095, -219180466, -1676061637,
+        1040194153, -1339307841, -1527110163, -1884722688, -179148713, -431014627, -1789789823, -1209719926, 684519765, -486734513, 1734740619, -1971673751, -932421915, -2064668066,
+        -937135958, -790946468, -69070309, 1561601454, 26169001, -160734571, 1330043462, -486734513, -18796768, -470303314, 113421364, 1615760212, 1688217556, 1223719184, 536568131,
+        1682609873, 2935161, -414769471, -1027490137, 1602276102, 1050490461 ]
+        }
+
+
+### counthashed-gram-tokens ###
+ * Syntax:
+
+        counthashed-gram-tokens(string_expression, gram_length, boolean_expression)
+
+ * Returns a list of hashed gram tokens of `string_expression`. The hashing mechanism gives duplicate tokens different hash values, based on the occurrence count of that token.
+ * Arguments:
+   * `string_expression` : A `String` that will be tokenized.
+   * `gram_length` : An `Int32`, length of grams to generate.
+   * `boolean_expression` : A `Boolean`, whether to generate additional grams by pre- and postfixing `string_expression` with special characters.
+ * Return Value:
+   * An `OrderedList` of `Int32` hashed gram tokens.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $t in dataset('TweetMessages')
+        let $tokens := counthashed-gram-tokens($t.message-text, 3, true)
+        where $t.send-time >= datetime('2012-01-01T00:00:00')
+        return {
+        "tweetid": $t.tweetid,
+        "counthashed-gram-tokens": $tokens
+        }
+
+
+ * The expected result is:
+
+        {
+        "tweetid": "9",
+        "counthashed-gram-tokens": [ 40557178, -2002241593, 161665899, -856104603, -500544946, 693410611, 395674299, -1015235909, 1115608337, 1187999872, -31006095, -219180466, -1676061637,
+        1040194153, -1339307841, -1527110163, -1884722688, -179148713, -431014627, -1789789823, -1209719926, 684519765, -486734513, 1734740619, -1971673751, -932421915, -2064668066, -937135958,
+        -790946468, -69070309, 1561601454, 26169001, -160734571, 1330043462, -486734512, -18796768, -470303314, 113421364, 1615760212, 1688217556, 1223719184, 536568131, 1682609873, 2935161,
+        -414769471, -1027490137, 1602276102, 1050490461 ]
+        }
+
+
+## Temporal Functions ##
+
+### date ###
+ * Syntax:
+
+        date(string_expression)
+
+ * Constructor function for `Date` type by parsing a date string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a date value.
+ * Return Value:
+   * A `Date` value represented by the given string.
+
+ * Example:
+
+        {
+        "date-extended": date("2013-04-01"),
+        "date-basic": date("20130401")
+        }
+
+
+ * The expected result is:
+
+        {
+        "date-extended": date("2013-04-01"),
+        "date-basic": date("2013-04-01")
+        }
+
+
+### time ###
+ * Syntax:
+
+        time(string_expression)
+
+ * Constructor function for `Time` type by parsing a time string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a time value.
+ * Return Value:
+   * A `Time` value represented by the given string.
+
+ * Example:
+
+        {
+        "time-extended": time("12:30:45.678+08:00"),
+        "time-basic": time("123045678+0800")
+        }
+
+
+ * The expected result is:
+
+        {
+        "time-extended": time("04:30:45.678Z"),
+        "time-basic": time("04:30:45.678Z")
+        }
+
+
+### datetime ###
+ * Syntax:
+
+        datetime(string_expression)
+
+ * Constructor function for `Datetime` type by parsing a datetime string `string_expression`
+ * Arguments:
+   * `string_expression` : The `String` value representing a datetime value.
+ * Return Value:
+   * A `Datetime` value represented by the given string.
+
+ * Example:
+
+        {
+        "datetime-extended": datetime("2013-04-01T12:30:45.678+08:00"),
+        "datetime-basic": datetime("20130401T123045678+0800")
+        }
+
+
+ * The expected result is:
+
+        {
+        "datetime-extended": datetime("2013-04-01T04:30:45.678Z"),
+        "datetime-basic": datetime("2013-04-01T04:30:45.678Z")
+        }
+
+
+### interval-from-date ###
+ * Syntax:
+
+        interval-from-date(string_expression1, string_expression2)
+
+ * Constructor function for `Interval` type by parsing two date strings.
+ * Arguments:
+   * `string_expression1` : The `String` value representing the starting date.
+   * `string_expression2` : The `String` value representing the ending date.
+ * Return Value:
+   * An `Interval` value between the two dates.
+
+ * Example:
+
+        {"date-interval": interval-from-date("2012-01-01", "2013-04-01")}
+
+
+ * The expected result is:
+
+        { "date-interval": interval-date("2012-01-01, 2013-04-01") }
+
+
+### interval-from-time ###
+ * Syntax:
+
+        interval-from-time(string_expression1, string_expression2)
+
+ * Constructor function for `Interval` type by parsing two time strings.
+ * Arguments:
+   * `string_expression1` : The `String` value representing the starting time.
+   * `string_expression2` : The `String` value representing the ending time.
+ * Return Value:
+   * An `Interval` value between the two times.
+
+ * Example:
+
+        {"time-interval": interval-from-time("12:23:34.456Z", "233445567+0800")}
+
+
+ * The expected result is:
+
+        { "time-interval": interval-time("12:23:34.456Z, 15:34:45.567Z") }
+
+
+### interval-from-datetime ###
+ * Syntax:
+
+        interval-from-datetime(string_expression1, string_expression2)
+
+ * Constructor function for `Interval` type by parsing two datetime strings.
+ * Arguments:
+   * `string_expression1` : The `String` value representing the starting datetime.
+   * `string_expression2` : The `String` value representing the ending datetime.
+ * Return Value:
+   * An `Interval` value between the two datetimes.
+
+ * Example:
+
+        {"datetime-interval": interval-from-datetime("2012-01-01T12:23:34.456+08:00", "20130401T153445567Z")}
+
+
+ * The expected result is:
+
+        { "datetime-interval": interval-datetime("2012-01-01T04:23:34.456Z, 2013-04-01T15:34:45.567Z") }
+
+
+### year/month/day/hour/minute/second/millisecond ###
+ * Syntax:
+
+        year/month/day/hour/minute/second/millisecond(temporal_expression)
+
+ * Accessors for accessing fields in a temporal value
+ * Arguments:
+   * `temporal_expression` : a temporal value represented as one of the following types: `Date`, `Datetime`, `Time`, `Duration`.
+ * Return Value:
+   * An `Int32` value representing the field to be extracted.
+
+ * Example:
+
+        let $c1 := date("2010-10-30")
+        let $c2 := datetime("1987-11-19T23:49:23.938")
+        let $c3 := time("12:23:34.930+07:00")
+        let $c4 := duration("P3Y73M632DT49H743M3948.94S")
+        
+        return {"year": year($c1), "month": month($c2), "day": day($c1), "hour": hour($c3), "min": minute($c4), "second": second($c2), "ms": millisecond($c4)}
+
+
+ * The expected result is:
+
+        { "year": 2010, "month": 11, "day": 30, "hour": 5, "min": 28, "second": 23, "ms": 94 }
+
+
+
+### add-date-duration ###
+ * Syntax:
+
+        add-date-duration(date_expression, duration_expression)
+
+ * Create a new date by adding the duration `duration_expression` to the given date `date_expression`.
+ * Arguments:
+   * `date_expression` : The `Date` value to be added onto.
+   * `duration_expression` : The `Duration` to be added.
+ * Return Value:
+   * A `Date` value represents the new date after being adjusted by the duration.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $startdate := date('2011-03-01')
+        for $i in dataset('TweetMessage')
+        where date-from-datetime($i.send-time) > $startdate
+        and date-from-datetime($i.send-time) < add-date-duration($startdate, duration('P2Y'))
+        return {"send-time": $i.send-time, "message": $i.message-text}
+
+
+ * The expected result is:
+
+        { "send-time": datetime("2011-12-26T10:10:00.000Z"), "message": " like sprint the voice-command is mind-blowing:)" }
+        { "send-time": datetime("2011-08-25T10:10:00.000Z"), "message": " like samsung the platform is good" }
+        { "send-time": datetime("2012-07-21T10:10:00.000Z"), "message": " love verizon its voicemail-service is awesome" }
+
+
+### add-datetime-duration ###
+ * Syntax:
+
+        add-date-duration(datetime_expression, duration_expression)
+
+ * Create a new datetime by adding the duration `duration_expression` to the given datetime `datetime_expression`.
+ * Arguments:
+   * `datetime_expression` : The `Datetime` value to be added onto.
+   * `duration_expression` : The `Duration` to be added.
+ * Return Value:
+   * A `Datetime` value represents the new datetime after being adjusted by the duration.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $startdt := datetime('2011-03-01T00:00:00')
+        for $i in dataset('TweetMessage')
+        where $i.send-time > $startdt and $i.send-time < add-datetime-duration($startdt, duration('P2Y'))
+        return {"send-time": $i.send-time, "message": $i.message-text}
+
+
+ * The expected result is:
+
+        { "send-time": datetime("2011-12-26T10:10:00.000Z"), "message": " like sprint the voice-command is mind-blowing:)" }
+        { "send-time": datetime("2011-08-25T10:10:00.000Z"), "message": " like samsung the platform is good" }
+        { "send-time": datetime("2012-07-21T10:10:00.000Z"), "message": " love verizon its voicemail-service is awesome" }
+
+
+### add-time-duration ###
+ * Syntax:
+
+        add-time-duration(time_expression, duration_expression)
+
+ * Create a new time by adding the duration `duration_expression` to the given time `time_expression`.
+ * Arguments:
+   * `time_expression` : The `Time` value to be added onto.
+   * `duration_expression` : The `Duration` to be added.
+ * Return Value:
+   * A `Time` value represents the new time after being adjusted by the duration.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $starttime := time('08:00:00')
+        for $i in dataset('TweetMessage')
+        where time-from-datetime($i.send-time) > $starttime and time-from-datetime($i.send-time) < add-time-duration($starttime, duration('PT5H'))
+        return {"send-time": $i.send-time, "message": $i.message-text}
+
+
+ * The expected result is:
+
+        { "send-time": datetime("2008-04-26T10:10:00.000Z"), "message": " love t-mobile its customization is good:)" }
+        { "send-time": datetime("2010-05-13T10:10:00.000Z"), "message": " like verizon its shortcut-menu is awesome:)" }
+        { "send-time": datetime("2006-11-04T10:10:00.000Z"), "message": " like motorola the speed is good:)" }
+        { "send-time": datetime("2011-12-26T10:10:00.000Z"), "message": " like sprint the voice-command is mind-blowing:)" }
+        { "send-time": datetime("2006-08-04T10:10:00.000Z"), "message": " can't stand motorola its speed is terrible:(" }
+        { "send-time": datetime("2010-05-07T10:10:00.000Z"), "message": " like iphone the voice-clarity is good:)" }
+        { "send-time": datetime("2011-08-25T10:10:00.000Z"), "message": " like samsung the platform is good" }
+        { "send-time": datetime("2005-10-14T10:10:00.000Z"), "message": " like t-mobile the shortcut-menu is awesome:)" }
+        { "send-time": datetime("2012-07-21T10:10:00.000Z"), "message": " love verizon its voicemail-service is awesome" }
+        { "send-time": datetime("2008-01-26T10:10:00.000Z"), "message": " hate verizon its voice-clarity is OMG:(" }
+        { "send-time": datetime("2008-03-09T10:10:00.000Z"), "message": " can't stand iphone its platform is terrible" }
+        { "send-time": datetime("2010-02-13T10:10:00.000Z"), "message": " like samsung the voice-command is amazing:)" }
+
+
+### adjust-datetime-for-timezone ###
+ * Syntax:
+
+        adjust-datetime-for-timezone(datetime_expression, string_expression)
+
+ * Adjust the given datetime `datetime_expression` by applying the timezone information `string_expression`
+ * Arguments:
+   * `datetime_expression` : A `Datetime` value to be adjusted.
+   * `string_expression` : A `String` representing the timezone information.
+ * Return Value:
+   * A `String` value represents the new datetime after being adjusted by the timezone information.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('TweetMessage')
+        return {"adjusted-send-time": adjust-datetime-for-timezone($i.send-time, "+08:00"), "message": $i.message-text}
+
+
+ * The expected result is:
+
+        { "adjusted-send-time": "2008-04-26T18:10:00.000+08:00", "message": " love t-mobile its customization is good:)" }
+        { "adjusted-send-time": "2010-05-13T18:10:00.000+08:00", "message": " like verizon its shortcut-menu is awesome:)" }
+        { "adjusted-send-time": "2006-11-04T18:10:00.000+08:00", "message": " like motorola the speed is good:)" }
+        { "adjusted-send-time": "2011-12-26T18:10:00.000+08:00", "message": " like sprint the voice-command is mind-blowing:)" }
+        { "adjusted-send-time": "2006-08-04T18:10:00.000+08:00", "message": " can't stand motorola its speed is terrible:(" }
+        { "adjusted-send-time": "2010-05-07T18:10:00.000+08:00", "message": " like iphone the voice-clarity is good:)" }
+        { "adjusted-send-time": "2011-08-25T18:10:00.000+08:00", "message": " like samsung the platform is good" }
+        { "adjusted-send-time": "2005-10-14T18:10:00.000+08:00", "message": " like t-mobile the shortcut-menu is awesome:)" }
+        { "adjusted-send-time": "2012-07-21T18:10:00.000+08:00", "message": " love verizon its voicemail-service is awesome" }
+        { "adjusted-send-time": "2008-01-26T18:10:00.000+08:00", "message": " hate verizon its voice-clarity is OMG:(" }
+        { "adjusted-send-time": "2008-03-09T18:10:00.000+08:00", "message": " can't stand iphone its platform is terrible" }
+        { "adjusted-send-time": "2010-02-13T18:10:00.000+08:00", "message": " like samsung the voice-command is amazing:)" }
+
+
+### adjust-time-for-timezone ###
+ * Syntax:
+
+        adjust-time-for-timezone(time_expression, string_expression)
+
+ * Adjust the given time `time_expression` by applying the timezone information `string_expression`
+ * Arguments:
+   * `time_expression` : A `Time` value to be adjusted.
+   * `string_expression` : A `String` representing the timezone information.
+ * Return Value:
+   * A `String` value represents the new time after being adjusted by the timezone information.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('TweetMessage')
+        return {"adjusted-send-time": adjust-time-for-timezone(time-from-datetime($i.send-time), "+08:00"), "message": $i.message-text}
+
+
+ * The expected result is:
+
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " love t-mobile its customization is good:)" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " like verizon its shortcut-menu is awesome:)" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " like motorola the speed is good:)" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " like sprint the voice-command is mind-blowing:)" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " can't stand motorola its speed is terrible:(" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " like iphone the voice-clarity is good:)" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " like samsung the platform is good" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " like t-mobile the shortcut-menu is awesome:)" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " love verizon its voicemail-service is awesome" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " hate verizon its voice-clarity is OMG:(" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " can't stand iphone its platform is terrible" }
+        { "adjusted-send-time": "18:10:00.000+08:00", "message": " like samsung the voice-command is amazing:)" }
+
+
+### calendar-duration-from-datetime ###
+ * Syntax:
+
+        calendar-duration-from-datetime(datetime_expression, duration_expression)
+
+ * Get a user-friendly representation of the duration `duration_expression` based on the given datetime `datetime_expression`
+ * Arguments:
+   * `datetime_expression` : A `Datetime` value to be used as the reference time point.
+   * `duration_expression` : A `Duration` value to be converted
+ * Return Value:
+   * A `Duration` value with the duration as `duration_expression` but with a user-friendly representation.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('TweetMessage')
+        where $i.send-time > datetime("2011-01-01T00:00:00")
+        return {"since-2011": subtract-datetime($i.send-time, datetime("2011-01-01T00:00:00")), "since-2011-user-friendly": calendar-duration-from-datetime($i.send-time, subtract-datetime($i.send-time, datetime("2011-01-01T00:00:00")))}
+
+
+ * The expected result is:
+
+        { "since-2011": duration("P359DT10H10M"), "since-2011-user-friendly": duration("P11M23DT10H10M") }
+        { "since-2011": duration("P236DT10H10M"), "since-2011-user-friendly": duration("P7M23DT10H10M") }
+        { "since-2011": duration("P567DT10H10M"), "since-2011-user-friendly": duration("P1Y6M18DT10H10M") }
+
+
+### calendar-duration-from-date ###
+ * Syntax:
+
+        calendar-duration-from-date(date_expression, duration_expression)
+
+ * Get a user-friendly representation of the duration `duration_expression` based on the given date `date_expression`
+ * Arguments:
+   * `date_expression` : A `Date` value to be used as the reference time point.
+   * `duration_expression` : A `Duration` value to be converted
+ * Return Value:
+   * A `Duration` value with the duration as `duration_expression` but with a user-friendly representation.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('TweetMessage')
+        where $i.send-time > datetime("2011-01-01T00:00:00")
+        return {"since-2011": subtract-datetime($i.send-time, datetime("2011-01-01T00:00:00")),
+        "since-2011-user-friendly": calendar-duration-from-date(date-from-datetime($i.send-time), subtract-datetime($i.send-time, datetime("2011-01-01T00:00:00")))}
+
+
+ * The expected result is:
+
+        { "since-2011": duration("P359DT10H10M"), "since-2011-user-friendly": duration("P11M23DT10H10M") }
+        { "since-2011": duration("P236DT10H10M"), "since-2011-user-friendly": duration("P7M23DT10H10M") }
+        { "since-2011": duration("P567DT10H10M"), "since-2011-user-friendly": duration("P1Y6M18DT10H10M") }
+
+
+### current-date ###
+ * Syntax:
+
+        current-date()
+
+ * Get the current date
+ * Arguments:None
+ * Return Value:
+   * A `Date` value of the date when the function is called.
+
+### current-time ###
+ * Syntax:
+
+        current-time()
+
+ * Get the current time
+ * Arguments:None
+ * Return Value:
+   * A `Time` value of the time when the function is called.
+
+### current-datetime ###
+ * Syntax:
+
+        current-datetime()
+
+ * Get the current datetime
+ * Arguments:None
+ * Return Value:
+   * A `Datetime` value of the datetime when the function is called.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        {"current-date": current-date(),
+        "current-time": current-time(),
+        "current-datetime": current-datetime()}
+
+
+ * The expected result is:
+
+        { "current-date": date("2013-04-06"),
+        "current-time": time("00:48:44.093Z"),
+        "current-datetime": datetime("2013-04-06T00:48:44.093Z") }
+
+
+### date-from-datetime ###
+ * Syntax:
+
+        date-from-datetime(datetime_expression)
+
+ * Get the date value from the given datetime value `datetime_expression`
+ * Arguments:
+   * `datetime_expression`: A `Datetime` value to be extracted from
+ * Return Value:
+   * A `Date` value from the datetime.
+
+### time-from-datetime ###
+ * Syntax:
+
+        time-from-datetime(datetime_expression)
+
+ * Get the time value from the given datetime value `datetime_expression`
+ * Arguments:
+   * `datetime_expression`: A `Datetime` value to be extracted from
+ * Return Value:
+   * A `Time` value from the datetime.
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('TweetMessage')
+        where $i.send-time > datetime("2011-01-01T00:00:00")
+        return {"send-date": date-from-datetime($i.send-time), "send-time": time-from-datetime($i.send-time)}
+
+
+ * The expected result is:
+
+        { "send-date": date("2011-12-26"), "send-time": time("10:10:00.000Z") }
+        { "send-date": date("2011-08-25"), "send-time": time("10:10:00.000Z") }
+        { "send-date": date("2012-07-21"), "send-time": time("10:10:00.000Z") }
+
+
+### date-from-unix-time-in-days ###
+ * Syntax:
+
+        date-from-unix-time-in-days(numeric_expression)
+
+ * Get date representing the time after `numeric_expression` days since 1970-01-01
+ * Arguments:
+   * `numeric_expression`: A `Int8`/`Int16`/`Int32` value representing the number of days
+ * Return Value:
+   * A `Date` value as the time after `numeric_expression` days since 1970-01-01
+
+### datetime-from-unix-time-in-ms ###
+ * Syntax:
+
+        datetime-from-unix-time-in-ms(numeric_expression)
+
+ * Get datetime representing the time after `numeric_expression` milliseconds since 1970-01-01T00:00:00Z
+ * Arguments:
+   * `numeric_expression`: A `Int8`/`Int16`/`Int32`/`Int64` value representing the number of milliseconds
+ * Return Value:
+   * A `Datetime` value as the time after `numeric_expression` milliseconds since 1970-01-01T00:00:00Z
+
+### time-from-unix-time-in-ms ###
+ * Syntax:
+
+        time-from-unix-time-in-ms(numeric_expression)
+
+ * Get time representing the time after `numeric_expression` milliseconds since 00:00:00.000Z
+ * Arguments:
+   * `numeric_expression`: A `Int8`/`Int16`/`Int32` value representing the number of milliseconds
+ * Return Value:
+   * A `Time` value as the time after `numeric_expression` milliseconds since 00:00:00.000Z
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        let $d := date-from-unix-time-in-days(15800)
+        let $dt := datetime-from-unix-time-in-ms(1365139700000)
+        let $t := time-from-unix-time-in-ms(3748)
+        return {"date": $d, "datetime": $dt, "time": $t}
+
+
+ * The expected result is:
+
+        { "date": date("2013-04-05"), "datetime": datetime("2013-04-05T05:28:20.000Z"), "time": time("00:00:03.748Z") }
+
+
+### subtract-date ###
+ * Syntax:
+
+        subtract-date(date_start, date_end)
+
+ * Get the duration between two dates `date_start` and `date_end`
+ * Arguments:
+   * `date_start`: the starting `Date`
+   * `date_end`: the ending `Date`
+ * Return Value:
+   * A `Duration` value between `date_start` and `date_end`
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookUser')
+        for $j in dataset('FacebookUser')
+        where $i.user-since < $j.user-since and $i.user-since > datetime("2012-01-01T00:00:00")
+        return {"id1": $i.id, "id2": $j.id, "diff": subtract-date(date-from-datetime($j.user-since), date-from-datetime($i.user-since))}
+
+
+ * The expected result is:
+
+        { "id1": 3, "id2": 1, "diff": duration("P41D") }
+        { "id1": 3, "id2": 7, "diff": duration("P28D") }
+        { "id1": 7, "id2": 1, "diff": duration("P13D") }
+
+
+### subtract-time ###
+ * Syntax:
+
+        subtract-time(time_start, time_end)
+
+ * Get the duration between two times `time_start` and `time_end`
+ * Arguments:
+   * `time_start`: the starting `Time`
+   * `time_end`: the ending `Time`
+ * Return Value:
+   * A `Duration` value between `time_start` and `time_end`
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookUser')
+        for $j in dataset('FacebookUser')
+        where $i.user-since < $j.user-since and $i.user-since > datetime("2012-01-01T00:00:00")
+        return {"id1": $i.id, "id2": $j.id, "diff": subtract-time(time-from-datetime($j.user-since), time("02:50:48.938"))}
+
+
+ * The expected result is:
+
+        { "id1": 3, "id2": 1, "diff": duration("PT7H19M11.62S") }
+        { "id1": 3, "id2": 7, "diff": duration("PT7H19M11.62S") }
+        { "id1": 7, "id2": 1, "diff": duration("PT7H19M11.62S") }
+
+
+### subtract-datetime ###
+ * Syntax:
+
+        subtract-datetime(datetime_start, datetime_end)
+
+ * Get the duration between two datetimes `datetime_start` and `datetime_end`
+ * Arguments:
+   * `datetime_start`: the starting `Datetime`
+   * `datetime_end`: the ending `Datetime`
+ * Return Value:
+   * A `Duration` value between `datetime_start` and `datetime_end`
+
+ * Example:
+
+        use dataverse TinySocial;
+        
+        for $i in dataset('FacebookUser')
+        for $j in dataset('FacebookUser')
+        where $i.user-since < $j.user-since and $i.user-since > datetime("2011-01-01T00:00:00")
+        return {"id1": $i.id, "id2": $j.id, "diff": subtract-datetime($j.user-since, $i.user-since)}
+
+
+ * The expected result is:
+
+        { "id1": 2, "id2": 1, "diff": duration("P576D") }
+        { "id1": 2, "id2": 3, "diff": duration("P535D") }
+        { "id1": 2, "id2": 7, "diff": duration("P563D") }
+        { "id1": 3, "id2": 1, "diff": duration("P41D") }
+        { "id1": 3, "id2": 7, "diff": duration("P28D") }
+        { "id1": 7, "id2": 1, "diff": duration("P13D") }
+
diff --git a/asterix-doc/src/site/markdown/AsterixDBRestAPI.md b/asterix-doc/src/site/markdown/AsterixDBRestAPI.md
new file mode 100644
index 0000000..80fbb11
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AsterixDBRestAPI.md
@@ -0,0 +1,285 @@
+# REST API to AsterixDB #
+
+## DDL API ##
+
+*End point for the data definition statements*
+
+Endpoint: _/ddl_
+
+Parameters:
+
+<table>
+<tr>
+  <td>Parameter</td>
+  <td>Description</td>
+  <td>Required?</td>
+</tr>
+<tr>
+  <td>ddl</td>
+  <td>String containing DDL statements to modify Metadata</td>
+  <td>Yes</td>
+</tr>
+</table>
+
+This call does not return any result. If the operations were successful, HTTP OK status code is returned.
+
+### Example ###
+
+#### DDL Statements ####
+
+
+        drop dataverse company if exists;
+        create dataverse company;
+        use dataverse company;
+        
+        create type Emp as open {
+          id : int32,
+          name : string
+        };
+        
+        create dataset Employee(Emp) primary key id;
+
+
+API call for the above DDL statements in the URL-encoded form.
+
+[http://localhost:19101/ddl?ddl=drop%20dataverse%20company%20if%20exists;create%20dataverse%20company;use%20dataverse%20company;create%20type%20Emp%20as%20open%20{id%20:%20int32,name%20:%20string};create%20dataset%20Employee(Emp)%20primary%20key%20id;](http://localhost:19101/ddl?ddl=drop%20dataverse%20company%20if%20exists;create%20dataverse%20company;use%20dataverse%20company;create%20type%20Emp%20as%20open%20{id%20:%20int32,name%20:%20string};create%20dataset%20Employee(Emp)%20primary%20key%20id;)
+
+#### Response ####
+*HTTP OK 200*  
+`<NO PAYLOAD>`
+
+## Update API ##
+
+*End point for update statements (INSERT, DELETE and LOAD)*
+
+Endpoint: _/update_
+
+Parameters:
+
+<table>
+<tr>
+  <td>Parameter</td>
+  <td>Description</td>
+  <td>Required?</td>
+</tr>
+<tr>
+  <td>statements</td>
+  <td>String containing update (insert/delete) statements to execute</td>
+  <td>Yes</td>
+</tr>
+</table>
+
+This call does not return any result. If the operations were successful, HTTP OK status code is returned.
+
+### Example ###
+
+#### Update Statements ####
+
+
+        use dataverse company;
+        
+        insert into dataset Employee({ "id":123,"name":"John Doe"});
+
+
+API call for the above update statement in the URL-encoded form.
+
+[http://localhost:19101/update?statements=use%20dataverse%20company;insert%20into%20dataset%20Employee({%20%22id%22:123,%22name%22:%22John%20Doe%22});](http://localhost:19101/update?statements=use%20dataverse%20company;insert%20into%20dataset%20Employee({%20%22id%22:123,%22name%22:%22John%20Doe%22});)
+
+#### Response ####
+*HTTP OK 200*  
+`<NO PAYLOAD>`
+
+## Query API ##
+
+*End point for query statements*
+
+Endpoint: _/query_
+
+Parameters:
+
+<table>
+<tr>
+  <td>Parameter</td>
+  <td>Description</td>
+  <td>Required?</td>
+</tr>
+<tr>
+  <td>query</td>
+  <td>Query string to pass to ASTERIX for execution</td>
+  <td>Yes</td>
+</tr>
+<tr>
+  <td>mode</td>
+  <td>Indicate if call should be synchronous or asynchronous. mode = synchronous blocks the call until results are available; mode = asynchronous returns immediately with a handle that can be used later to check the query’s status and to fetch results when available</td>
+  <td>No. default mode = synchronous</td>
+</tr>
+</table>
+
+Result: The result is returned as a JSON object as follows
+
+
+        {
+           results: <result as a string, if mode = synchronous>
+           error-code: [<code>, <message>] (if an error occurs)
+           handle: <opaque result handle, if mode = asynchronous>
+        }
+
+
+### Example ###
+
+#### Select query with synchronous result delivery ####
+
+
+        use dataverse company;
+        
+        for $l in dataset('Employee') return $l;
+
+
+API call for the above query statement in the URL-encoded form.
+
+[http://localhost:19101/query?query=use%20dataverse%20company;for%20$l%20in%20dataset('Employee')%20return%20$l;](http://localhost:19101/query?query=use%20dataverse%20company;for%20$l%20in%20dataset('Employee')%20return%20$l;)
+
+#### Response ####
+*HTTP OK 200*  
+Payload
+
+
+        {
+          "results": [
+              [
+                  "{ "id": 123, "name": "John Doe" }"
+              ]
+          ]
+        }
+
+
+#### Same select query with asynchronous result delivery ####
+
+API call for the above query statement in the URL-encoded form with mode=asynchronous
+
+[http://localhost:19101/query?query=use%20dataverse%20company;for%20$l%20in%20dataset('Employee')%20return%20$l;&amp;mode=asynchronous](http://localhost:19101/query?query=use%20dataverse%20company;for%20$l%20in%20dataset('Employee')%20return%20$l;&amp;mode=asynchronous)
+
+#### Response ####
+*HTTP OK 200*  
+Payload
+
+
+        {
+            "handle": [45,0]
+        }
+
+
+## Asynchronous Result API ##
+
+*End point to fetch the results of an asynchronous query*
+
+Endpoint: _/query/result_
+
+Parameters:
+
+<table>
+<tr>
+  <td>Parameter</td>
+  <td>Description</td>
+  <td>Required?</td>
+</tr>
+<tr>
+  <td>handle</td>
+  <td>Result handle that was returned by a previous call to a /query call with mode = asynchronous</td>
+  <td>Yes</td>
+</tr>
+</table>
+
+Result: The result is returned as a JSON object as follows:
+
+
+        {
+           results: <result as a string, if mode = synchronous, or mode = asynchronous and results are available>
+           error-code: [<code>, <message>] (if an error occurs)
+        }
+
+
+If mode = asynchronous and results are not available, the returned JSON object is empty: { }
+
+### Example ###
+
+#### Fetching results for asynchronous query ####
+
+We use the handle returned by the asynchronous query to get the results for the query. The handle returned was:
+
+
+        {
+            "handle": [45,0]
+        }
+
+
+API call for reading results from the previous asynchronous query in the URL-encoded form.
+
+[http://localhost:19101/query/result?handle=%7B%22handle%22%3A+%5B45%2C+0%5D%7D](http://localhost:19101/query/result?handle=%7B%22handle%22%3A+%5B45%2C+0%5D%7D)
+
+#### Response ####
+*HTTP OK 200*  
+Payload
+
+
+        {
+          "results": [
+              [
+                  "{ "id": 123, "name": "John Doe" }"
+              ]
+          ]
+        }
+
+
+## Query Status API ##
+
+*End point to check the status of the query asynchronous*
+
+Endpoint: _/query/status_
+
+Parameters:
+
+<table>
+<tr>
+  <td>Parameter</td>
+  <td>Description</td>
+  <td>Required?</td>
+</tr>
+<tr>
+  <td>handle</td>
+  <td>Result handle that was returned by a previous call to a /query call with mode = asynchronous</td>
+  <td>Yes</td>
+</tr>
+</table>
+
+Result: The result is returned as a JSON object as follows:
+
+
+        {
+           status: ("RUNNING" | "SUCCESS" | "ERROR")
+        }
+
+
+
+## Error Codes ##
+
+Table of error codes and their types:
+
+<table>
+<tr>
+  <td>Code</td>
+  <td>Type</td>
+</tr>
+<tr>
+  <td>1</td>
+  <td>Invalid statement</td>
+</tr>
+<tr>
+  <td>2</td>
+  <td>Parse failures</td>
+</tr>
+<tr>
+  <td>99</td>
+  <td>Uncategorized error</td>
+</tr>
+</table>
diff --git a/asterix-doc/src/site/markdown/AsterixQueryLanguage.md b/asterix-doc/src/site/markdown/AsterixQueryLanguage.md
new file mode 100644
index 0000000..af25cda
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AsterixQueryLanguage.md
@@ -0,0 +1,195 @@
+`<wiki:toc max_depth="2" />`
+
+# The Asterix Query Language, Version 1.0 #
+
+# Introduction #
+
+This wiki page provides an overview of the Asterix Query language and the Asterix Data model.
+
+*WARNING:* _THIS IS AN INCOMPLETE SUSPENDED WORK IN PROGRESS...
+_  It will hopefully be resumed shortly in order to produce a legit AQL spec to go out with the Beta Release of AsterixDB.  What's here is very likely inconsistent with what's in the system as of today, as this was from an older snapshot of the world.
+
+# Asterix Data Model #
+
+Data in Asterix is represented using the Asterix Data Model (ADM). The ADM derives inspiration from prior standards such as JSON, XQuery, and the Object Data Model from ODMG.
+
+## Asterix Types ##
+
+### Primitive Types ##
+
+|| *Primitive Type* || *Description* ||
+|| int8   || Signed 8-bit integer. Valid range -128 thru 127 ||
+|| int16  || Signed 16-bit integer. Valid range -32768 thru 32767 ||
+|| int32  || Signed 32-bit integer. Valid range -2147483648 thru 2147483647 ||
+|| int64  || Signed 64-bit integer. Valid range -9223372036854775807 thru 9223372036854775808 ||
+|| uint8  || Unsigned 8-bit integer. Valid range 0 thru 255 ||
+|| uint16 || Unsigned 16-bit integer. Valid range 0 thru 65535 ||
+|| uint32 || Unsigned 32-bit integer. Valid range 0 thru 4294967295 ||
+|| uint64 || Unsigned 64-bit integer. Valid range 0 thru 18446744073709551615 ||
+|| string || String of characters ||
+|| null   || null type (Type of the null value) ||
+|| date   || Date ||
+|| time   || Time of day ||
+|| boolean || Boolean ||
+|| datetime || Date and time ||
+|| point2d || A point in 2-D space ||
+|| point3d || A point in 3-D space ||
+|| binary || Binary data ||
+|| yminterval || Year-Month interval ||
+|| dtinterval || Day-Time interval ||
+|| interval || Year-Month and Day-Time interval ||
+
+### Collection Types ###
+
+|| *Collection Type* || *Description* ||
+|| Record || A record type describes the record data item. A record contains a set of fields which can have values of any ADM type. Fields of a record must be unique. ||
+|| Union || A union type is an abstract type (A value never has a union type) that describes a set of type choices. ||
+|| Ordered List || An orderedlist instance represents a sequence of values where the order of the instances is determined by creation/insertion ||
+|| UnorderedList || An unorderedlist instance represents a collection of values where the order of the instances where the order is irrelevant ||
+|| Enumeration || An enumeration type represents a choice of string values ||
+
+# AQL Expressions #
+
+## Primary Expressions ##
+
+Primary expressions are the basic expressions that form the core of AQL.
+
+### Literals ###
+
+A Literal is a syntactic representation of a constant value. The various literals allowed in AQL are described in the table below.
+
+|| *Literal type* || *Syntax* ||
+|| StringLiteral || ` STRING_LITERAL : ("\"" ("\\\"" | ~["\""])* "\"") | ("\'"("\\\'" | ~["\'"])* "\'") ` ||
+|| IntegerLiteral || ` INTEGER_LITERAL : (["0" - "9"])+ ` ||
+|| FloatLiteral || ` FLOAT_LITERAL: ((["0" - "9"])* "." (["0" - "9"])+ ("f" | "F")) ` ||
+|| DoubleLiteral || ` DOUBLE_LITERAL: ((["0" - "9"])* "." (["0" - "9"])+) ` ||
+|| NullLiteral || ` NULL_LITERAL: "null" ` ||
+|| BooleanLiteral || ` BOOLEAN_LITERAL: "true" | "false" ` ||
+
+### Function Call ###
+
+Function Calls in AQL can be used to invoke builtin functions as well as user defined functions.
+Function Calls have the following syntax.
+
+
+            IDENTIFIER "(" ( Expression ( "," Expression )* )? ")"
+
+
+### Variable Reference ###
+
+Variables in AQL are used to bind to values. Variables can be bound to values by the For, Let, Group by clauses of the FLWOR expressions. Variables can also be bound by
+the Quantified Expressions.
+
+### Ordered List Constructor ###
+
+Constructs an ordered list. An ordered list represents a collection of values. The order of values is relevant. The collection may contain duplicate values.
+
+### Unordered List Constructor ###
+
+Constructs an unordered list. An unordered list represents a collection of values. The order of values is not relevant. The collection may contain duplicate values.
+
+### Record Constructor ###
+
+Constructs an AQL Record. A record contains fields. Each field has a name and a value. The name of the field is of type string. The value of a field may be any legal ADM data type. A record may not contain duplicate fields.
+
+## Arithmetic Expressions ##
+
+AQL allows all the standard arithmetic operators on numeric data types. The specific operators allowed are:
+
+|| *Operator* || *Description* ||
+|| + || Add ||
+|| - || Subtract ||
+|| * || Multiply ||
+|| / || Divide ||
+|| mod || Modulo ||
+
+## Comparison Expressions ##
+
+AQL provides the six standard comparison expressions listed below. In addition, AQL supports fuzzy comparisons.
+
+|| *Operator* || *Description* ||
+|| = || Equal ||
+|| = || Not Equal ||
+|| `< || Less Than ||
+|| `<= || Less Than or Equal ||
+|| >` || Greater Than ||
+|| >`= || Greater Than or Equal ||
+|| >`= || Greater Than or Equal ||
+|| ~= || Fuzzy Equals ||
+
+## Logical Expressions ##
+
+AQL provides two logical connectors:
+
+|| *Operator* || *Description* ||
+|| and || Logical AND ||
+|| or || Logical OR ||
+
+## Field Access Expressions ##
+
+The "." operator is used to access fields of a record. For example,
+
+
+            $x.name
+
+
+accesses the name field of the record bound to $x.
+
+## Indexed Expressions ##
+
+Indexed expressions are used to access values in an ordered list. For example,
+
+
+            $x[5]
+
+
+accesses the 6th item in the list bound to $x. Indexes start at 0.
+
+## FLWOR Expression ##
+
+The FLWOR expression is the most elaborate expression in AQL. It is made up of two parts -- Clauses and the Return Expression.
+
+The syntax of the FLWOR expression is:
+
+
+        
+        ( ForClause | LetClause )
+        ( ForClause | LetClause | WhereClause | OrderClause | GroupClause | LimitClause | DistinctClause )*
+        "return" ReturnExpression
+        
+
+
+* For Clause
+
+        "for" Variable "in" Expression
+
+* Let Clause
+
+        "let" Variable ":=" Expression
+
+* Where Clause
+
+        "where" Expression
+
+* Order Clause
+
+        "order" "by" Expression ("asc" | "desc") ("," Expression ("asc" | "desc"))*
+
+* Group Clause
+
+        "group" "by" ((Variable ":=")? Expression) ("," ((Variable ":=")? Expression))* "with" Variable
+
+* Limit Clause
+
+        "limit" Expression ("," Expression)?
+
+
+* If Then Else Expressions *
+
+        "if" "(" Expression ")" "then" Expression "else" Expression
+
+
+* Quantified Expressions *
+
+        ("some" | "every") Variable "in" Expression "satisfies" Expression
+
diff --git a/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md b/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md
new file mode 100644
index 0000000..1ed876e
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md
@@ -0,0 +1,462 @@
+# The Asterix Query Language, Version 1.0
+## 1. Introduction
+
+This document provides an overview of the Asterix Query language.
+
+
+## 2. Expressions
+
+    Expression ::= ( OperatorExpr | IfThenElse | FLWOGR | QuantifiedExpression )
+
+
+### Primary Expressions
+
+    PrimaryExpr ::= Literal
+                  | VariableRef
+                  | ParenthesizedExpression
+                  | FunctionCallExpr
+                  | DatasetAccessExpression
+                  | ListConstructor
+                  | RecordConstructor
+                  
+
+#### Literals
+
+    Literal ::= StringLiteral
+              | <INTEGER_LITERAL>
+              | <FLOAT_LITERAL>
+              | <DOUBLE_LITERAL>
+              | "null"
+              | "true"
+              | "false"
+    StringLiteral ::= <STRING_LITERAL>
+
+##### Examples
+
+    "a string"
+    42
+
+
+#### Variable References
+
+    VariableRef ::= <VARIABLE>
+
+##### Example
+
+    $id  
+    
+
+#### Parenthesized Expressions
+    
+    ParenthesizedExpression ::= "(" Expression ")"
+
+##### Example
+
+    ( 1 + 1 )
+
+
+#### Function Calls
+
+    FunctionCallExpr ::= FunctionOrTypeName "(" ( Expression ( "," Expression )* )? ")"
+
+##### Example
+
+    string-length("a string")
+
+
+#### Dataset Access
+
+    DatasetAccessExpression ::= "dataset" ( ( Identifier ( "." Identifier )? )
+                              | ( "(" Expression ")" ) )
+    Identifier              ::= <IDENTIFIER> | StringLiteral
+
+##### Examples
+
+    dataset customers
+    dataset (string-join("customers", $country))
+    
+
+#### Constructors
+
+    ListConstructor          ::= ( OrderedListConstructor | UnorderedListConstructor )
+    OrderedListConstructor   ::= "[" ( Expression ( "," Expression )* )? "]"
+    UnorderedListConstructor ::= "{{" ( Expression ( "," Expression )* )? "}}"
+    RecordConstructor        ::= "{" ( FieldBinding ( "," FieldBinding )* )? "}"
+    FieldBinding             ::= Expression ":" Expression
+
+##### Examples
+
+    [ "a", "b", "c" ]
+    
+    {{ 42, "forty-two", "AsterixDB!" }}
+    
+    {
+      "project name"    : "AsterixDB"
+      "project members" : {{ "vinayakb", "dtabass", "chenli" }}
+    } 
+
+
+### Path Expressions
+
+    ValueExpr ::= PrimaryExpr ( Field | Index )*
+    Field     ::= "." Identifier
+    Index     ::= "[" ( Expression | "?" ) "]"
+
+##### Examples
+
+    { "list" : [ "a", "b", "c"] }.list
+    
+    [ "a", "b", "c"][2]
+    
+    { "list" : [ "a", "b", "c"] }.list[2]
+
+
+### Logical Expressions
+
+    OperatorExpr ::= AndExpr ( "or" AndExpr )*
+    AndExpr      ::= RelExpr ( "and" RelExpr )*
+    
+##### Example
+
+    $a > 3 and $a < 5
+    
+
+### Comparison Expressions
+
+    RelExpr ::= AddExpr ( ( "<" | ">" | "<=" | ">=" | "=" | "!=" | "~=" ) AddExpr )?
+    
+##### Example
+
+    5 > 3
+
+
+### Arithmetic Expressions
+
+    AddExpr  ::= MultExpr ( ( "+" | "-" ) MultExpr )*
+    MultExpr ::= UnaryExpr ( ( "*" | "/" | "%" | <CARET> | "idiv" ) UnaryExpr )*
+    UnaryExpr ::= ( ( "+" | "-" ) )? ValueExpr
+
+##### Example
+
+    3 ^ 2 + 4 ^ 2
+
+
+###  FLWOGR Expression   
+    
+    FLWOGR         ::= ( ForClause | LetClause ) ( Clause )* "return" Expression
+    Clause         ::= ForClause | LetClause | WhereClause | OrderbyClause
+                     | GroupClause | LimitClause | DistinctClause
+    ForClause      ::= "for" Variable ( "at" Variable )? "in" ( Expression )
+    LetClause      ::= "let" Variable ":=" Expression
+    WhereClause    ::= "where" Expression
+    OrderbyClause  ::= "order" "by" Expression ( ( "asc" ) | ( "desc" ) )? 
+                       ( "," Expression ( ( "asc" ) | ( "desc" ) )? )*
+    GroupClause    ::= "group" "by" ( Variable ":=" )? Expression ( "," ( Variable ":=" )? Expression )*          
+                       "with" VariableRef ( "," VariableRef )*
+    LimitClause    ::= "limit" Expression ( "offset" Expression )?
+    DistinctClause ::= "distinct" "by" Expression ( "," Expression )*
+    Variable       ::= <VARIABLE>
+
+
+##### Example
+
+    for $user in dataset FacebookUsers
+    where $user.id = 8
+    return $user
+    
+##### Example
+
+    for $user in dataset FacebookUsers
+    for $message in dataset FacebookMessages
+    where $message.author-id = $user.id
+    return
+      {
+        "uname": $user.name,
+        "message": $message.message
+      }; 
+    
+##### Example
+
+    for $user in dataset FacebookUsers
+    let $messages := 
+      for $message in dataset FacebookMessages
+      where $message.author-id = $user.id
+      return $message.message
+    return
+      {
+        "uname": $user.name,
+        "messages": $messages
+      }; 
+      
+##### Example
+      
+      for $user in dataset TwitterUsers
+      order by $user.followers_count desc, $user.lang asc
+      return $user
+      
+* null is smaller than any other value
+
+##### Example
+
+      for $x in dataset FacebookMessages
+      let $messages := $x.message
+      group by $loc := $x.sender-location with $messages
+      return
+        {
+          "location" : $loc,
+          "message" : $messages
+        }
+
+* after group by only variables that are either in the group-by-list or in the with-list are in scope
+* the variables in the with-clause contain a collection of items after the group by clause  (all the values that the variable was bound to in the tuples that make up the group)
+* null is handled as a single value for grouping
+
+##### Example
+
+      for $user in dataset TwitterUsers
+      order by $user.followers_count desc
+      limit 2
+      return $user
+
+##### Example (currently not working)
+    
+      for $x in dataset FacebookMessages
+      distinct by $x.sender-location
+      return
+        {
+          "location" : $x.sender-location,
+          "message" : $x.message
+        }
+
+* every variable that is in-scope before the distinct clause is also in scope after the distinct clause
+* works a lot like group by, but for every variable that contains more than one value after the distinct-by clause, one value is picked non-deterministically
+* if the variable is in the disctict-by list, then value is deterministic
+* null is a single value
+    
+### Conditional Expression
+    
+    IfThenElse ::= "if" "(" Expression ")" "then" Expression "else" Expression
+
+##### Example
+
+    if (2 < 3) then "yes" else "no"
+
+
+### Quantified Expressions
+    
+    QuantifiedExpression ::= ( ( "some" ) | ( "every" ) ) Variable "in" Expression 
+                             ( "," Variable "in" Expression )* "satisfies" Expression
+                             
+##### Examples
+
+    every $x in [ 1, 2, 3] satisfies $x < 3
+    some $x in [ 1, 2, 3] satisfies $x < 3
+
+## 3. Statements
+
+    Statement ::= ( SingleStatement ( ";" )? )* <EOF>
+    SingleStatement ::= DataverseDeclaration
+                      | FunctionDeclaration
+                      | CreateStatement
+                      | DropStatement
+                      | LoadStatement
+                      | SetStatement
+                      | InsertStatement
+                      | DeleteStatement
+                      | Query
+    
+### Declarations    
+    
+    DataverseDeclaration ::= "use" "dataverse" Identifier
+    SetStatement         ::= "set" Identifier StringLiteral
+    FunctionDeclaration  ::= "declare" "function" Identifier ParameterList "{" Expression "}"
+    ParameterList        ::= "(" ( <VARIABLE> ( "," <VARIABLE> )* )? ")"
+
+##### Example
+
+    use dataverse TinySocial;
+    
+##### Example
+
+    set simfunction "jaccard";
+    set simthreshold "0.6f"; 
+
+##### Example
+
+    set simfunction "jaccard";    
+    set simthreshold "0.6f"; 
+    
+##### Example
+    
+    declare function add($a, $b) {
+      $a + $b
+    };
+
+### Lifecycle Management Statements
+
+    CreateStatement ::= "create" ( DataverseSpecification
+                                 | TypeSpecification
+                                 | DatasetSpecification
+                                 | IndexSpecification
+                                 | FunctionSpecification )
+
+    QualifiedName       ::= Identifier ( "." Identifier )?
+    DoubleQualifiedName ::= Identifier "." Identifier ( "." Identifier )?
+
+#### Dataverses
+
+    DataverseSpecification ::= "dataverse" Identifier IfNotExists ( "with format" StringLiteral )?
+    
+
+##### Example
+
+    create dataverse TinySocial;
+
+#### Types
+
+    TypeSpecification    ::= "type" FunctionOrTypeName IfNotExists "as" TypeExpr
+    FunctionOrTypeName   ::= QualifiedName
+    IfNotExists          ::= ( "if not exists" )?
+    TypeExpr             ::= RecordTypeDef | TypeReference | OrderedListTypeDef | UnorderedListTypeDef
+    RecordTypeDef        ::= ( "closed" | "open" )? "{" ( RecordField ( "," RecordField )* )? "}"
+    RecordField          ::= Identifier ":" ( TypeExpr ) ( "?" )?
+    TypeReference        ::= Identifier
+    OrderedListTypeDef   ::= "[" ( TypeExpr ) "]"
+    UnorderedListTypeDef ::= "{{" ( TypeExpr ) "}}"
+
+##### Example
+
+    create type FacebookUserType as closed {
+      id: int32,
+      alias: string,
+      name: string,
+      user-since: datetime,
+      friend-ids: {{ int32 }},
+      employment: [EmploymentType]
+    }
+
+
+#### Datasets
+
+    DatasetSpecification ::= "internal"? "dataset" QualifiedName "(" Identifier ")" IfNotExists
+                             PrimaryKey ( "on" Identifier )? ( "hints" Properties )? 
+                           | "external" "dataset" QualifiedName "(" Identifier ")" IfNotExists 
+                             "using" AdapterName Configuration ( "hints" Properties )?
+    AdapterName          ::= Identifier
+    Configuration        ::= "(" ( KeyValuePair ( "," KeyValuePair )* )? ")"
+    KeyValuePair         ::= "(" StringLiteral "=" StringLiteral ")"
+    Properties           ::= ( "(" Property ( "," Property )* ")" )?
+    Property             ::= Identifier "=" ( StringLiteral | <INTEGER_LITERAL> )
+    ApplyFunction        ::= "apply" "function" FunctionSignature
+    FunctionSignature    ::= FunctionOrTypeName "@" <INTEGER_LITERAL>
+    PrimaryKey           ::= "primary" "key" Identifier ( "," Identifier )*
+
+
+##### Example
+    create internal dataset FacebookUsers(FacebookUserType) primary key id;
+
+##### Example
+
+    create external dataset Lineitem(LineitemType) using localfs (
+      ("path"="127.0.0.1://SOURCE_PATH"),
+      ("format"="delimited-text"),
+      ("delimiter"="|"));
+      
+#### Indices
+
+    IndexSpecification ::= "index" Identifier IfNotExists "on" QualifiedName 
+                           "(" ( Identifier ) ( "," Identifier )* ")" ( "type" IndexType )?
+    IndexType          ::= "btree"
+                         | "rtree"
+                         | "keyword"
+                         | "fuzzy keyword"
+                         | "ngram" "(" <INTEGER_LITERAL> ")"
+                         | "fuzzy ngram" "(" <INTEGER_LITERAL> ")"
+
+##### Example
+
+    create index fbAuthorIdx on FacebookMessages(author-id) type btree;
+
+##### Example
+
+    create index fbSenderLocIndex on FacebookMessages(sender-location) type rtree;
+
+##### Example
+
+    create index fbMessageIdx on FacebookMessages(message) type keyword;
+
+
+#### Functions
+
+    FunctionSpecification ::= "function" FunctionOrTypeName IfNotExists ParameterList "{" Expression "}"
+    
+##### Example
+    
+    create function add($a, $b) {
+      $a + $b
+    };
+    
+
+#### Removal
+
+    DropStatement       ::= "drop" ( "dataverse" Identifier IfExists
+                                   | "type" FunctionOrTypeName IfExists
+                                   | "dataset" QualifiedName IfExists
+                                   | "index" DoubleQualifiedName IfExists
+                                   | "function" FunctionSignature IfExists )
+    IfExists            ::= ( "if" "exists" )?
+    
+##### Example
+
+    drop dataset FacebookUsers if exists;
+
+##### Example
+
+    drop index fbSenderLocIndex;
+
+##### Example
+
+    drop type FacebookUserType;
+    
+##### Example
+
+    drop dataverse TinySocial;
+
+##### Example
+
+    drop function add;
+    
+
+### Import/Export Statements
+
+    LoadStatement  ::= "load" "dataset" QualifiedName "using" AdapterName Configuration ( "pre-sorted" )?
+    
+##### Example
+
+    load dataset FacebookUsers using localfs
+    (("path"="localhost:///Users/zuck/AsterixDB/load/fbu.adm"),("format"="adm"));
+
+
+### Modification Statements
+
+    InsertStatement ::= "insert" "into" "dataset" QualifiedName Query
+    DeleteStatement ::= "delete" Variable "from" "dataset" QualifiedName ( "where" Expression )?
+    
+##### Example
+
+    insert into dataset UsersCopy (for $user in dataset FacebookUsers return $user)
+
+##### Example
+    
+    delete $user from dataset FacebookUsers where $user.id = 8;
+    
+
+### Queries
+
+    Query ::= Expression
+    
+##### Example
+    
+    for $praise in {{ "great", "brilliant", "awesome" }}
+    return
+       string-concat(["AsterixDB is ", $praise])
diff --git a/asterix-doc/src/site/markdown/AsterixSimilarityQueries.md b/asterix-doc/src/site/markdown/AsterixSimilarityQueries.md
new file mode 100644
index 0000000..4f22fef
--- /dev/null
+++ b/asterix-doc/src/site/markdown/AsterixSimilarityQueries.md
@@ -0,0 +1,83 @@
+# AsterixDB Support of Similarity Queries #
+
+## Motivation ##
+
+Similarity queries are widely used in applications where users need to find records that satisfy a similarity predicate, while exact matching is not sufficient. These queries are especially important for social and Web applications, where errors, abbreviations, and inconsistencies are common.  As an example, we may want to find all the movies starring Schwarzenegger, while we don't know the exact spelling of his last name (despite his popularity in both the movie industry and politics :-)). As another example, we want to find all the Facebook users who have similar friends. To meet this type of needs, AsterixDB supports similarity queries using efficient indexes and algorithms.
+
+## Data Types and Similarity Functions ##
+
+AsterixDB supports various similarity functions, including [edit distance](http://en.wikipedia.org/wiki/Levenshtein_distance) (on strings) and [Jaccard](http://en.wikipedia.org/wiki/Jaccard_index) (on sets). For instance, in our [TinySocial](AdmAql101.html#ADM:_Modeling_Semistructed_Data_in_AsterixDB) example, the `friend-ids` of a Facebook user forms a set of friends, and we can define a similarity between two sets. We can also convert a string to a set of "q-grams" and define the Jaccard similarity between the two sets of two strings. The "q-grams" of a string are its substrings of length "q". For instance, the 3-grams of the string `schwarzenegger` are `sch`, `chw`, `hwa`, ..., `ger`.
+
+AsterixDB provides [tokenization functions](AsterixDataTypesAndFunctions.html#Tokenizing_Functions) to convert strings to sets, and the [similarity functions](AsterixDataTypesAndFunctions.html#Similarity_Functions).
+
+## Selection Queries ##
+
+The following [query](AsterixDataTypesAndFunctions.html#edit-distance) asks for all the Facebook users whose name is similar to `Suzanna Tilson`, i.e., their edit distance is at most 2.
+
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $ed := edit-distance($user.name, "Suzanna Tilson")
+        where $ed <= 2
+        return $user
+
+
+The following [query](AsterixDataTypesAndFunctions.html#similarity-jaccard) asks for all the Facebook users whose set of friend ids is similar to `[1,5,9]`, i.e., their Jaccard similarity is at least 0.6.
+
+
+        use dataverse TinySocial;
+        
+        for $user in dataset('FacebookUsers')
+        let $sim := similarity-jaccard($user.friend-ids, [1,5,9])
+        where $sim >= 0.6f
+        return $user
+
+
+AsterixDB allows a user to use a similarity operator `~=` to express a similarity condition by defining the similiarty function and threshold using "set" statements earlier. For instance, the above query can be equivalently written as:
+
+
+        use dataverse TinySocial;
+        
+        set simfunction "jaccard";
+        set simthreshold "0.6f";
+        
+        for $user in dataset('FacebookUsers')
+        where $user.friend-ids ~= [1,5,9]
+        return $user
+
+
+
+## Fuzzy Join Queries ##
+
+AsterixDB supports fuzzy joins between two data sets. The following [query](AdmAql101.html#Query_5_-_Fuzzy_Join) finds, for each Facebook user, all Twitter users with names "similar" to their name based on the edit distance.
+
+
+        use dataverse TinySocial;
+        
+        set simfunction "edit-distance";
+        set simthreshold "3";
+        
+        for $fbu in dataset FacebookUsers
+        return {
+            "id": $fbu.id,
+            "name": $fbu.name,
+            "similar-users": for $t in dataset TweetMessages
+                                let $tu := $t.user
+                                where $tu.name ~= $fbu.name
+                                return {
+                                "twitter-screenname": $tu.screen-name,
+                                "twitter-name": $tu.name
+                                }
+        };
+
+
+## Using Indexes ##
+
+AsterixDB uses inverted index to support similarity queries efficiently. For instance, the following query creates such an index on the `FacebookUser.name` attribute using an inverted index of 3-grams.  After the index is created, similarity queries with an edit distance condition on this attribute can be answered more efficiently.
+
+
+        use dataverse TinySocial;
+        
+        create index fbUserFuzzyIdx on FacebookUsers(name) type ngram(3);
+
diff --git a/asterix-doc/src/site/markdown/InstallingAsterixUsingManagix.md b/asterix-doc/src/site/markdown/InstallingAsterixUsingManagix.md
new file mode 100644
index 0000000..65aebdf
--- /dev/null
+++ b/asterix-doc/src/site/markdown/InstallingAsterixUsingManagix.md
@@ -0,0 +1,808 @@
+# Introduction #
+This is a quickstart guide for getting ASTERIX running in a distributed environment. This guide also introduces the ASTERIX installer (nicknamed _*Managix*_) and describes how it can be used to create/manage an ASTERIX instance. By following the simple steps described in this guide, you will get a running instance of ASTERIX. You shall be able to use ASTERIX from its Web interface and manage its lifecycle using Managix. This document assumes that you are running some version of _*Linux*_ or _*MacOS X*_.
+
+## Prerequisites for Installing ASTERIX ##
+Prerequisite:
+
+ * [JDK7](http://www.oracle.com/technetwork/java/javase/downloads/index.html) (Otherwise known as JDK 1.7).
+
+To know the version of Java installed on your system, execute the following:
+
+        $ java -version
+
+If you have version as 1.7.0_x, similar to the output shown below, you are good to proceed.
+
+
+        java version "1.7.0_13"
+        Java(TM) SE Runtime Environment (build 1.7.0_13-b20)
+        Java HotSpot(TM) 64-Bit Server VM (build 23.7-b01, mixed mode)
+
+If you need to upgrade or install java, please follow the instructions below.
+
+ * For Linux: [JDK 7 Linux Install](http://docs.oracle.com/javase/7/docs/webnotes/install/linux/linux-jdk.html)
+JDK would be installed at a path under /usr/lib/jvm/jdk-version .
+
+ * For Mac: [JDK 7 Mac Install](http://docs.oracle.com/javase/7/docs/webnotes/install/mac/mac-jdk.html)
+JDK would be installed at /Library/Java/JavaVirtualMachines/jdk-version/Contents/Home .
+
+The java installation directory is referred as JAVA_HOME. Since we upgraded/installed Java, we need to ensure JAVA_HOME points to the installation directory of JDK 7. Modify your ~/.bash_profile (or ~/.bashrc) and define JAVA_HOME accordingly.  After modifying, execute the following:
+
+
+        $ java -version
+
+If the version information you obtain does not show 1.7, you need to update the PATH variable. To do so, execute the following:
+
+
+        $ echo "PATH=$JAVA_HOME/bin:$PATH" >> ~/.bash_profile (or ~/.bashrc)
+        $ source ~/.bash_profile (or ~/.bashrc)
+
+We also need to ensure that $JAVA_HOME/bin is in the PATH. $JAVA_HOME/bin should be included in the PATH value. We need to change the  if $JAVA_HOME/bin is already in the PATH, we shall simply execute the following:
+
+
+        $ java
+
+If you get the following message, you need to alter the PATH variable in your ~/.bash_profile or ~/.bashrc (whichever you use).
+
+
+        -bash: java: command not found
+
+## Section 1: Single-Machine ASTERIX installation ##
+We assume a user Joe with a home directory as /home/joe. Please note that on Mac, the home directory for user Joe would be /Users/joe.
+
+### Configuring Environment ###
+Ensure that JAVA_HOME  variable is defined and points to the the java installation directory on your machine.  To verify, execute the following.
+
+
+        $ echo $JAVA_HOME
+
+If you do not see any output, JAVA_HOME is not defined. We need to add the following line to your profile located at  /home/joe/.bash_profile or /home/joe/.bashrc, whichever you are using. If you do not any of these files, create a ~/.bash_profile.
+
+
+        export JAVA_HOME=<Path to Java installation directory>
+
+After you have edited ~/.bash_profile (or ~/.bashrc), execute the following to make the changes effective in current shell.
+
+
+        $ source /home/joe/.bash_profile (or /home/joe/.bashrc)
+
+Before proceeding, verify that JAVA_HOME is defined by executing the following.
+
+
+        $ echo $JAVA_HOME
+
+### Configuring SSH ###
+If SSH is not enabled on your system, please follow the instruction below to enable/install it or else skip to the section [Configuring Password-less SSH](#Configuring_Password-less_SSH).
+
+#### Enabling SSH on Mac ####
+The Apple Mac OS X operating system has SSH installed by default but the SSH daemon is not enabled. This means you can’t login remotely or do remote copies until you enable it. To enable it, go to ‘System Preferences’. Under ‘Internet & Networking’ there is a ‘Sharing’ icon. Run that. In the list that appears, check the ‘Remote Login’ option. Also check the "All users" radio button for "Allow access for".  This starts the SSH daemon immediately and you can remotely login using your username. The ‘Sharing’ window shows at the bottom the name and IP address to use. You can also find this out using ‘whoami’ and ‘ifconfig’ from the Terminal application.
+
+#### Enabling SSH on Linux ####
+
+        sudo apt-get install openssh-server
+
+Assumming that you have enabled SSH on your system, let us proceed.
+
+#### Configuring Password-less SSH ####
+
+For our single-machine setup of ASTERIX, we need to configure password-less SSH access to localhost. We assume that you are on the machine where you want to install ASTERIX. To verify if you already have password-less SSH configured, execute the following.
+
+
+         $ ssh 127.0.0.1
+
+If you get an output similar to one shown below, type "yes" and press enter.
+
+
+        The authenticity of host '127.0.0.1 (127.0.0.1)' can't be established.
+        RSA key fingerprint is aa:7b:51:90:74:39:c4:f6:28:a2:9d:47:c2:8d:33:31.
+        Are you sure you want to continue connecting (yes/no)?
+
+If you are not prompted for a password, that is if you get an output similar to one shown below, skip to the next section [Configuring Managix](#Configuring_Managix).
+
+
+        $ ssh 127.0.0.1
+        Last login: Sat Mar 23 22:52:49 2013
+
+You are here because you were prompted for a password.  You need to configure password less SSH.   Follow the instructions below.
+
+
+         $ ssh-keygen -t rsa -P ""
+        Generating public/private rsa key pair.
+        Enter file in which to save the key (/home/joe/.ssh/id_rsa):   [We shall use the default value, so simply press enter]
+
+If a key already exists, you should get an output similar to what is shown below.  Press 'y' to overwrite the existing key.
+
+
+        /home/joe/.ssh/id_rsa already exists.
+        Overwrite (y/n)?
+
+You should see an output similar to one shown below.
+
+
+        The key fingerprint is:
+        4d:b0:30:14:45:cc:99:86:15:48:17:0b:39:a0:05:ca joe@joe-machine
+        The key's randomart image is:
+        +--[ RSA 2048]----+
+        |  ..o+B@O=       |
+        |.. o  ==*+       |
+        |.E.    oo .      |
+        |         o       |
+        |        S .      |
+        |                 |
+        |                 |
+        |                 |
+        |                 |
+        +-----------------+
+
+Note: for Linux users, you may not get an image representation of the key, but this is not an error.  Next, execute the following:
+
+
+          $ cat $HOME/.ssh/id_rsa.pub >> $HOME/.ssh/authorized_keys
+
+We shall now retry SSH without password.
+
+
+        $ ssh 127.0.0.1
+
+You may see an output similar to one shown below.
+
+
+        The authenticity of host '127.0.0.1 (127.0.0.1)' can't be established.
+        RSA key fingerprint is aa:7b:51:90:74:39:c4:f6:28:a2:9d:47:c2:8d:33:31.
+        Are you sure you want to continue connecting (yes/no)?
+
+Type 'yes' and press the enter key. You should see an output similar to one shown below.
+
+
+        Warning: Permanently added '127.0.0.1' (RSA) to the list of known hosts.
+        Last login: Thu Mar 28 12:27:10 2013
+
+You should now be able to log in without being prompted for a password or a response.
+
+
+        ssh 127.0.0.1
+        Last login: Sat Mar 23 22:54:40 2013
+
+Execute 'exit' to close the session.
+
+
+        $ exit
+        logout
+        Connection to 127.0.0.1 closed.
+
+### Configuring Managix ###
+You will need the ASTERIX installer (a.k.a Managix). Download Managix from [here](https://asterixdb.googlecode.com/files/asterix-installer-0.0.5-binary-assembly.zip); this includes the bits for Managix as well as ASTERIX.
+
+Unzip the Managix zip bundle to an appropriate location. You may create a sub-directory:  asterix-mgmt (short for asterix-management) under your home directory. We shall refer to this location as MANAGIX_HOME.
+
+
+          $ cd ~
+          /home/joe> $ mkdir asterix-mgmt
+          /home/joe> $ cd asterix-mgmt
+          /home/joe/asterix-mgmt> $ unzip  <path to the Managix zip bundle>
+          /home/joe/asterix-mgmt> $ export MANAGIX_HOME=`pwd`
+          /home/joe/asterix-mgmt> $ export PATH=$PATH:$MANAGIX_HOME/bin
+
+It is recommended that you add $MANAGIX_HOME/bin to your PATH variable in your bash profile . This can be done by executing the following.
+
+
+        currentDir=`pwd`
+        echo "export MANAGIX_HOME=$currentDir" >> ~/.bash_profile
+        echo "export PATH=$PATH:$MANAGIX_HOME/bin" >> ~/.bash_profile
+
+Above, use ~/.bashrc instead of ~/.bash_profile if you are using ~/.bashrc .
+
+To be able to create an ASTERIX instance and manage its lifecycle, the Managix requires you to configure a set of configuration files namely:
+
+ * `conf/managix-conf.xml`:  A configuration XML file that contains configuration settings for Managix.
+ * A configuration XML file that describes the nodes in the cluster, e.g., `$MANAGIX_HOME/clusters/local/local.xml`.
+
+Since we intend to run ASTERIX on a single node, Managix can auto-configure itself and populate the above mentioned configuration files.  To auto-configure Managix, execute the following in the MANAGIX_HOME directory:
+
+
+         /home/joe/asterix-mgmt> $ managix configure
+
+Let us do a sample run to validate the set of configuration files auto-generated by Managix.
+
+
+        /home/joe/asterix-mgmt> $ managix validate
+          INFO: Environment [OK]
+          INFO: Managix Configuration [OK]
+
+
+        /home/joe/asterix-mgmt> $ managix validate -c $MANAGIX_HOME/clusters/local/local.xml
+          INFO: Environment [OK]
+          INFO: Cluster configuration [OK]
+
+### Creating an ASTERIX instance ###
+Now that we have configured Managix, we shall next create an ASTERIX instance. An ASTERIX instance is identified by a unique name and is created using the `create` command. The usage description for the `create` command can be obtained by executing the following.
+
+
+         $ managix help -cmd create
+         Creates an ASTERIX instance with a specified name. Post creation, the instance is in ACTIVE state,
+         indicating its availability for executing statements/queries.
+         Usage arguments/options:
+         -n Name of the ASTERIX instance.
+         -c Path to the cluster configuration file
+
+We shall now use the create command to create an ASTERIX instance by the name "my_asterix". In doing so, we shall use the cluster configuration file that was auto-generated by managix.
+
+
+         $ managix create -n my_asterix -c $MANAGIX_HOME/clusters/local/local.xml
+
+A sample output of the above command is shown below:
+
+
+        INFO: Name:my_asterix
+        Created:Thu Mar 07 11:14:13 PST 2013
+        Web-Url:http://127.0.0.1:19001
+        State:ACTIVE
+
+The third line above shows the web-url http://127.0.0.1:19001 for ASTERIX's web-interface. The ASTERIX instance is in the 'ACTIVE' state indicating that you may access the web-interface by navigating to the web-url.
+
+Type in the following "Hello World" query in the box:
+
+
+        let $message := 'Hello World!'
+        return $message
+
+Press the "Execute" button.  If the query result shows on the output box, then Congratulations! You have successfully created an ASTERIX instance!
+
+## Section 2: Single-Machine ASTERIX installation (Advanced) ##
+We assume that you have successfully completed the single-machine ASTERIX installation by following the instructions above in section  [ASTERIX installation](#Section_1:_Single-Machine_ASTERIX_installation Single Machine).  In this section, we shall cover advanced topics related to ASTERIX configuration. Before we proceed, it is imperative to go through some preliminary concepts related to ASTERIX runtime.
+
+### ASTERIX Runtime ###
+An ASTERIX runtime comprises of a ''master node'' and a set of ''worker nodes'', each identified by a unique id. The master node runs a ''Cluster Controller'' service (a.k.a. ''CC''),  while each worker node runs a ''Node Controller'' service (a.k.a. ''NC''). Please note that a node in an ASTERIX cluster is a logical concept in the sense that multiple nodes may map to a single physical machine, which is the case for a single-machine ASTERIX installation. This association or mapping between an ASTERIX node and a physical machine is captured in a cluster configuration XML file. In addition, the XML file contains properties and parameters associated with each node.
+
+#### ASTERIX Runtime Configuration ####
+As observed earlier, Managix can auto-configure itself for a single-machine setup. As part of auto-configuration, Managix generated the cluster XML file. Let us understand the components of the generated cluster XML file. If you have configured Managix (via the "configure" command), you can find a similar cluster XML file as $MANAGIX_HOME/clusters/local/local.xml. The following is a sample XML file generated on a Ubuntu (Linux) setup:
+
+
+        <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+        <cluster xmlns="cluster">
+            <name>local</name>
+            <java_home>/usr/lib/jvm/jdk1.7.0</java_home>
+            <java_opts>-Xmx1048m</java_opts>
+            <logdir>/home/joe/asterix-mgmt/clusters/local/working_dir/logs</logdir>
+            <iodevices>/home/joe/asterix-mgmt/clusters/local/working_dir</iodevices>
+            <store>storage</store>
+            <workingDir>
+                <dir>/home/joe/asterix-mgmt/clusters/local/working_dir</dir>
+                <NFS>true</NFS>
+            </workingDir>
+            <master-node>
+                <id>master</id>
+                <client-ip>127.0.0.1</client-ip>
+                <cluster-ip>127.0.0.1</cluster-ip>
+            </master-node>
+            <node>
+                <id>node1</id>
+                <cluster-ip>127.0.0.1</cluster-ip>
+            </node>
+        </cluster>
+
+We shall next explain the components of the cluster configuration XML file.
+
+#### (1) Defining nodes in ASTERIX runtime ####
+The single-machine ASTERIX instance configuration that is auto-generated by Managix (using the "configure" command) involves a master node (CC) and a worker node (NC).  Each node is assigned a unique id and provided with an ip address (called ''cluster-ip'') that maps a node to a physical machine. The following snippet from the above XML file captures the master/worker nodes in our ASTERIX installation.
+
+
+            <master-node>
+                <id>master</id>
+                <client-ip>127.0.0.1</client-ip>
+                <cluster-ip>127.0.0.1</cluster-ip>
+            </master-node>
+            <node>
+                <id>node1</id>
+                <cluster-ip>127.0.0.1</cluster-ip>
+            </node>
+
+
+The following is a description of the different elements in the cluster configuration xml file.
+
+<table>
+<tr>
+  <td>Property</td>
+  <td>Description</td>
+</tr>
+<tr>
+  <td>id</td>
+  <td>A unique id for a node.</td>
+</tr>
+<tr>
+  <td>cluster-ip</td>
+  <td>IP address of the machine to which a node maps to. This address is used for all internal communication between the nodes.</td>
+</tr>
+<tr>
+  <td>client-ip</td>
+  <td>Provided for the master node. This IP should be reachable from clients that want to connect with ASTERIX via its web interface.</td>
+</tr>
+</table>
+
+#### (2) Properties associated with a worker node (NC) in ASTERIX ####
+The following is a list of properties associated with each worker node in an ASTERIX configuration.
+
+<table>
+<tr>
+  <td>Property</td>
+  <td>Description</td>
+</tr>
+<tr>
+  <td>java_home</td>
+  <td>Java installation directory at each node.</td>
+</tr>
+<tr>
+  <td>java_opts</td>
+  <td>JVM arguments passed on to the JVM that represents a node.</td>
+</tr>
+<tr>
+  <td>logdir</td>
+  <td>A directory where worker node may write logs.</td>
+</tr>
+<tr>
+  <td>io_devices</td>
+  <td>Comma separated list of IO Device mount points.</td>
+</tr>
+<tr>
+  <td>store</td>
+  <td>A data directory that ASTERIX uses to store data belonging to dataset(s).</td>
+</tr>
+</table>
+
+All the above properties can be defined at the global level or a local level. In the former case, these properties apply to all the nodes in an ASTERIX configuration. In the latter case, these properties apply only to the node(s) under which they are defined. A property defined at the local level overrides the definition at the global level.
+
+#### (3) Working directory of an ASTERIX instance ####
+
+Next we explain the following setting in the file $MANAGIX_HOME/clusters/local/local.xml.
+
+            <workingDir>
+                <dir>/Users/joe/asterix-mgmt/clusters/local/working_dir</dir>
+                <NFS>true</NFS>
+            </workingDir>
+
+
+Managix associates a working directory with an ASTERIX instance and uses this directory for transferring binaries to each node. If there exists a directory that is readable by each node, Managix can use it to place binaries that can be accessed and used by all the nodes in the ASTERIX set up. A network file system (NFS) provides such a functionality for a cluster of physical machines such that a path on NFS is accessible from each machine in the cluster.  In the single-machine set up described above, all nodes correspond to a single physical machine. Each path on the local file system is accessible to all the nodes in the ASTERIX setup and the boolean value for NFS above is thus set to `true`.
+
+### Managix Configuration ###
+Managix allows creation and management of multiple ASTERIX instances and uses Zookeeper as its back-end database to keep track of information related to each instance. We need to provide a set of one or more hosts that Managix can use to run a Zookeeper instance. Zookeeper runs as a daemon process on each of the specified hosts. At each host, Zookeeper stores data under the Zookeeper home directory specified as part of the configuration. The following is an example configuration `$MANAGIX_HOME/conf/managix-conf.xml` that has Zookeeper running on the localhost (127.0.0.1) :
+
+
+        <?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+        <configuration xmlns="installer">
+            <zookeeper>
+                <homeDir>/home/joe/asterix/.installer/zookeeper</homeDir>
+                <clientPort>2900</clientPort>
+                <servers>
+                    <server>127.0.0.1</server>
+                </servers>
+            </zookeeper>
+        </configuration>
+
+It is possible to have a single host for Zookeeper. A larger number of hosts would use Zookeeper's replication and fault-tolerance feature such that a failure of a host running Zookeeper would not result in loss of information about existing ASTERIX instances.
+
+## Section 3: Installing ASTERIX on a Cluster of Multiple Machines ##
+We assume that you have read the two sections above on single-machine ASTERIX setup. Next we explain how to install ASTERIX in a cluster of multiple machines.  As an example, we assume we want to setup ASTERIX on a cluster of three machines, in which we use one machine (called machine A) as the master node and two other machines (called machine B and machine C) as the worker nodes, as shown in the following diagram:
+
+![AsterixCluster](https://asterixdb.googlecode.com/files/AsterixCluster.png)
+
+Notice that each machine has a ''cluster-ip'' address, which is used by these machines for their intra-cluster communication. Meanwhile, the master machine also has a ''client-ip'' address, using which an end-user outside the cluster can communicate with this machine.  The reason we differentiate between these two types of IP addresses is that we can have a cluster of machines using a private network. In this case they have internal ip addresses that cannot be used outside the network.  In the case all the machines are on a public network, the "client-ip" and "cluster-ip" of the master machine can share the same address.
+
+Next we describe how to set up ASTERIX in this cluster, assuming no Managix has been installed on these machines.
+
+### Step (1): Define the ASTERIX cluster ###
+
+We first log into the master machine as the user "joe". On this machine, download Managix from [here](https://asterixdb.googlecode.com/files/asterix-installer-0.0.5-binary-assembly.zip) (save as above), then do the following steps similar to the single-machine case described above:
+
+
+        machineA> cd ~
+        machineA> mkdir asterix-mgmt
+        machineA> cd asterix-mgmt
+        machineA> unzip  <path to the Managix zip bundle>
+        machineA> export MANAGIX_HOME=`pwd`
+        machineA> export PATH=$PATH:$MANAGIX_HOME/bin
+
+
+We also need an ASTERIX configuration XML file for the cluster.  We give the name to the cluster, say, "rainbow".  We create a folder for the configuration of this cluster:
+
+
+        machineA> mkdir $MANAGIX_HOME/rainbow_cluster
+
+
+For this cluster we create a configuration file `$MANAGIX_HOME/rainbow_cluster/rainbow.xml`.  The following is a sample file with explanation of the properties:
+
+        <cluster xmlns="cluster">
+        
+          <!-- Name of the cluster -->
+          <name>rainbow</name>
+        
+          <!-- username, which should be valid for all the three machines -->
+          <username>joe</username>
+        
+          <!-- The working directory of Managix. It should be on a network file system (NFS) that
+            can accessed by all the machine. Need to create it before running Managix. -->
+          <workingDir>
+            <dir>/home/joe/managix-workingDir</dir>
+            <NFS>true</NFS>
+          </workingDir>
+        
+          <!-- Directory for Asterix to store log information for each machine. Needs
+           to be a local file system. Needs to create it before running Managix. -->
+          <logdir>/mnt/joe/logs</logdir>
+        
+          <!-- Directory used by each worker node to store data files. Needs
+           to be a local file system. Needs to create it before running Managix. -->
+          <iodevices>/mnt/joe</iodevices>
+          <store>storage</store>
+        
+          <!-- Java home for each machine with its JVM options -->
+          <java_home>/usr/lib/jvm/jdk1.7.0</java_home>
+          <java_opts>-Xmx1024m</java_opts>
+        
+           <!-- IP addresses of the master machine A -->
+          <master-node>
+            <id>master</id>
+            <client-ip>128.195.52.177</client-ip>
+            <cluster-ip>192.168.100.0</cluster-ip>
+          </master-node>
+        
+           <!-- IP address(es) of machine B -->
+          <node>
+            <id>nodeB</id>
+            <cluster-ip>192.168.100.1</cluster-ip>
+          </node>
+        
+           <!-- IP address(es) of machine C -->
+          <node>
+            <id>nodeC</id>
+            <cluster-ip>192.168.100.2</cluster-ip>
+          </node>
+        </cluster>
+
+
+As stated before, each of the above properties can be defined at the cluster level, in which case it applies to all the nodes in the system.  Each property can also be defined at a node level.
+
+Once we have formed the cluster XML file, we can validate the configuration by doing the following:
+
+        managix validate -c $MANAGIX_HOME/rainbow_cluster/rainbow.xml
+
+
+If the return message says "OK", it means that the XML configuration file is set properly.
+
+### Step (2): Configure SSH ###
+
+The next steps of setting up SSH are similar to those in the single-machine setup case.  We assume we have a common user account called "joe" on each machine in the cluster.
+
+On the master machine, do the following:
+
+
+        machineA> ssh localhost
+
+
+If you are prompted for a password, execute the following
+
+
+        machineA> ssh-keygen -t rsa -P ""
+        machineA> cat $HOME/.ssh/id_rsa.pub >> $HOME/.ssh/authorized_keys
+
+
+If $HOME is not on the NFS, copy the id_rsa.pub to the directory ~/.ssh (login with the same account) on each machine, and then do the following on each machine. (Notice that this step is not needed if the folder ".ssh" is on the NFS and can be accessed by all the nodes.)
+
+
+        cd ~/.ssh
+        cat id_rsa.pub >> authorized_keys
+
+
+Then run the following step again and type "Yes" if promoted:
+
+
+        machineA> ssh localhost
+
+### Step (3): Configuring Managix ###
+
+Managix is using a configuration XML file at `$MANAGIX_HOME/conf/managix-conf.xml` to configure its own properties, such as its Zookeeper service.  We can use the `configure` command to auto-generate this configuration file:
+
+
+        machineA> managix configure
+
+
+We use the validate command to validate managix configuration. To do so, execute the following.
+
+        machineA> managix validate
+        INFO: Environment [OK]
+        INFO: Managix Configuration [OK]
+
+
+Note that the `configure` command also generates a cluster configuration XML file at $MANAGIX_HOME/conf/clusters/local.xml. This file is not needed in the case of a cluster of machines.
+
+### Step (4): Creating an ASTERIX instance ###
+
+Now that we have configured Managix, we shall next create an ASTERIX instance. An ASTERIX instance is identified by a unique name and is created using the create command. The usage description for the create command can be obtained by executing the following:
+
+
+        machineA> managix help -cmd create
+        
+         Creates an ASTERIX instance with a specified name. Post creation, the instance is in ACTIVE state,
+         indicating its availability for executing statements/queries.
+         Usage arguments/options:
+         -n Name of the ASTERIX instance.
+         -c Path to the cluster configuration file
+
+
+We shall now use the `create` command to create an ASTERIX instance called "rainbow_asterix". In doing so, we shall use the cluster configuration file that was auto-generated by Managix.
+
+
+        machineA> managix create -n rainbow_asterix -c $MANAGIX_HOME/clusters/rainbow/rainbow.xml
+
+
+If the response message does not have warning, then Congratulations! You have successfully installed Asterix on this cluster of machines!
+
+Please refer to the section [Managing the Lifecycle of an ASTERIX Instance](#Section_4:_Managing_the_Lifecycle_of_an_ASTERIX_Instance) for a detailed description on the set of available commands/operations that let you manage the lifecycle of an ASTERIX instance. Note that the output of the commands varies with the cluster definition and may not apply to the cluster specification you built above.
+
+## Section 4: Managing the Lifecycle of an ASTERIX Instance ##
+
+Now that we have an ASTERIX instance running, let us use Managix to manage the instance's lifecycle. Managix provides the following set of commands/operations:
+
+#### Managix Commands ####
+
+<table>
+<tr><td>Command</td>  <td>Description</td></tr>
+<tr><td><a href="#Creating_an_ASTERIX_instance">create</a></td>   <td>Creates a new asterix instance.</td></tr>
+<tr><td><a href="#Describe_Command"            >describe</a></td> <td>Describes an existing asterix instance.</td></tr>
+<tr><td><a href="#Stop_Command"                >stop</a></td>     <td>Stops an asterix instance that is in the ACTIVE state.</td></tr>
+<tr><td><a href="#Start_Command"               >start</a></td>    <td>Starts an Asterix instance.</td></tr>
+<tr><td><a href="#Backup_Command"              >backup</a></td>   <td>Creates a backup for an existing Asterix instance.</td></tr>
+<tr><td><a href="#Restore_Command"             >restore</a></td>  <td>Restores an Asterix instance.</td></tr>
+<tr><td><a href="#Delete_Command"              >delete</a></td>   <td>Deletes an Asterix instance.</td></tr>
+<tr><td><a href="#Configuring_Managix"         >validate</a></td> <td>Validates the installer/cluster configuration.</td></tr>
+<tr><td><a href="#Configuring_Managix"         >configure</a></td><td>Auto generate configuration for an Asterix instance.</td></tr>
+<tr><td><a href="#Shutdown_Command"            >shutdown</a></td> <td>Shutdown the installer service.</td></tr>
+</table>
+
+You may obtain the above listing by simply executing 'managix' :
+
+
+        $ managix
+
+We already talked about create and validate commands. We shall next explain the rest of the commands listed above.  We also provide sample output messages of these commands assuming we are running an ASTERIX instance on a single machine.
+
+##### Describe Command #####
+The `describe` command provides information about an ASTERIX instance. The usage can be looked up by executing the following:
+
+
+         $ managix help -cmd describe
+        
+            Provides information about an ASTERIX instance.
+            The following options are available:
+            [-n]      Name of the ASTERIX instance.
+            [-admin]  Provides a detailed description
+
+The brackets indicate optional flags.
+
+The output of the `describe` command when used without the `admin` flag contains minimal information and is similar to the output of the create command.  Let us try running the describe command in "admin" mode.
+
+
+         $ managix describe -n my_asterix -admin
+         INFO: Name:my_asterix
+         Created:Thu Mar 07 19:07:00 PST 2013
+         Web-Url:http://127.0.0.1:19001
+         State:ACTIVE
+         Master node:master:127.0.0.1
+         node1:127.0.0.1
+        
+         Asterix version:0.0.5
+         Asterix Configuration
+         output_dir = /tmp/asterix_output/
+         Metadata Node:node1
+         Processes
+         NC at 127.0.0.1 [ 22195 ]
+         CC at 127.0.0.1 [ 22161 ]
+
+As seen above, the instance 'my_asterix' is configured such that  all processes running at the localhost (127.0.0.1). The process id for each process (JVM) is shown next to it.
+
+##### Stop Command #####
+The `stop` command can be used for shutting down an ASTERIX instance. After that, the instance is unavailable for executing queries. The usage can be looked up by executing the following:
+
+
+         $ managix help -cmd stop
+        
+         Shuts an ASTERIX instance that is in ACTIVE state. After executing the stop command, the ASTERIX instance transits
+         to the INACTIVE state, indicating that it is no longer available for executing queries.
+        
+         Available arguments/options
+         -n name of the ASTERIX instance.
+
+To stop the ASTERIX instance.
+
+
+         $ managix stop -n my_asterix
+           INFO: Stopped Asterix instance: my_asterix
+
+
+         $ managix describe -n my_asterix
+           INFO: Name: my_asterix
+           Created:Thu Mar 07 19:07:00 PST 2013
+           Web-Url:http://127.0.0.1:19001
+           State:INACTIVE (Fri Mar 08 09:49:00 PST 2013)
+
+
+##### Start Command #####
+The `start` command starts an ASTERIX instance that is in the INACTIVE state. The usage can be looked up by executing the following:
+
+
+         $ managix help -cmd start
+        
+         Starts an ASTERIX instance that is in INACTIVE state. After executing the start command, the ASTERIX instance transits to the ACTIVE state,  indicating that it is now available for executing statements/queries.
+        
+         Available arguments/options
+         -n name of the ASTERIX instance.
+
+Let us now start the ASTERIX instance.
+
+
+         $ managix start -n my_asterix
+           INFO: Name:my_asterix
+           Created:Thu Mar 07 19:07:00 PST 2013
+           Web-Url:http://127.0.0.1:19001
+           State:ACTIVE (Fri Mar 08 09:49:00 PST 2013)
+
+
+##### Backup Command #####
+
+In an undesirable event of data loss either due to a disk/system failure or accidental execution of a DDL statement (drop dataverse/dataset), you may need to recover the lost data. The backup command allows you to take a backup of the data stored with an ASTERIX instance. The backup can be taken on the local file system or on an HDFS instance. In either case, the snapshots are stored under a backup directory. You need to make sure the backup directory has appropriate read/write permissions. Configuring settings for backup can be found inside the Managix's configuration file located at `$MANAGIX_HOME/conf/managix-conf.xml`.
+
+*Configuring backup on the local file system*
+
+We need to provide path to a backup directory on the local file system. The backup directory can be configured be editing the Managix configuration XML, found at `$MANAGIX_HOME/conf/managix-conf.xml`.
+
+
+        <backup>
+          <backupDir>Provide path to the backup directory here</backupDir>
+        </backup>
+
+Prior to taking a backup of an ASTERIX instance, it is required for the instance to be in the INACTIVE state. We do so by using the `stop` command, as shown below:
+
+
+        $ managix stop -n my_asterix
+          INFO: Stopped Asterix instance: my_asterix
+
+We can now take the backup by executing the following:
+
+
+        $ managix backup -n my_asterix
+          INFO: my_asterix backed up 0_Fri Mar 08 16:16:34 PST 2013 (LOCAL)
+
+
+*Configuring backup on an HDFS instance*
+
+To configure a backups to be taken on an HDFS instance, we need to provide required information about the running HDFS instance.  This information includes the HDFS version and the HDFS url.  Simply edit the Managix configuration file and provide the required information.
+
+
+        <backup>
+           <backupDir>Provide path to the backup directory here</backupDir>
+           <hdfs>
+               <version>0.20.2</version>
+               <url></url>
+           </hdfs>
+        </backup>
+
+A sample output when a backup is taken on an HDFS is shown below:
+
+
+        $ managix backup -n my_asterix
+          INFO: my_asterix backed up 1_Fri Mar 08 17:10:38 PST 2013 (HDFS)
+
+
+Each time we take a backup,  we are provided with a unique id  (a monotonically increasing value starting with 0).  This id is required when we need to restore from a previously taken backup. Information about all available backup snapshots can be obtained by using the `describe` command in the admin mode, as shown below:
+
+
+         $ managix describe -n my_asterix -admin
+         INFO: Name:my_asterix
+         Created:Fri Mar 08 15:11:12 PST 2013
+         Web-Url:http://127.0.0.1:19001
+         State:INACTIVE (Fri Mar 08 16:14:20 PST 2013)
+         Master node:master:127.0.0.1
+         node1:127.0.0.1
+        
+         Backup:0 created at Fri Mar 08 16:16:34 PST 2013 (LOCAL)
+         Backup:1 created at Fri Mar 08 17:10:38 PST 2013 (HDFS)
+        
+         Asterix version:0.0.5
+         Asterix Configuration
+         Metadata Node:node1
+         Processes
+
+
+The above output shows the available backup identified by it's id (0). We shall next describe the method for restoring an ASTERIX instance from a backup snapshot.
+
+##### Restore Command #####
+
+The `restore` command allows you to restore an ASTERIX instance's data from a previously taken backup. The usage description can be obtained as follows:
+
+
+          $ managix help -cmd restore
+        
+          Restores an ASTERIX instance's data from a previously taken backup.
+          Available arguments/options
+        
+          -n name of the ASTERIX instance
+          -b id of the backup snapshot
+
+
+The following command restores our ASTERIX instance from the backup snapshot identified by the id (0). Prior to restoring an instance from a backup, it is required that the instance is in the INACTIVE state.
+
+
+         $ managix restore -n my_asterix -b 0
+         INFO: Asterix instance: my_asterix has been restored from backup
+
+You can start the ASTERIX instance by using the start command.
+
+##### Delete Command #####
+As the name suggests, the `delete` command permanently removes an ASTERIX instance by cleaning up all associated data/artifacts. The usage can be looked up by executing the following:
+
+
+          $ managix help -cmd delete
+          Permanently deletes an ASTERIX instance. The instance must be in the INACTIVE state.
+        
+          Available arguments/options
+          -n name of the ASTERIX instance.
+
+
+          $ managix delete -n my_asterix
+           INFO: Asterix instance my_asterix deleted.
+
+
+##### Shutdown Command #####
+Managix uses Zookeeper service for storing all information about created ASTERIX instances. The Zookeeper service runs in the background and can be shut down using the `shutdown` command.
+
+
+          $ managix shutdown
+
+
+##### Help Command #####
+The `help` command provides a usage description of a Managix command.
+
+
+        $ managix help -cmd <command name>
+
+As an example, for looking up the help for the `configure` command, execute the following
+
+
+        $ managix help -cmd configure
+        
+        Auto-generates the ASTERIX installer configruation settings and ASTERIX cluster
+        configuration settings for a single node setup.
+
+
+## Section 5: Frequently Asked Questions ##
+
+
+*Question*
+What is meant by the "UNUSABLE" state in the lifecycle of  an ASTERIX instance ?
+
+
+*Answer*
+When Managix fails to start a required process (CC/NC), the instance transits to an UNUSABLE state.
+The reason for the failure needs to be looked up in the logs.
+Before we attempt to start the instance again, any processes that got launched
+as part of failed attempt must be stopped. No other operation except "stop" is supported in the UNUSABLE state.
+
+Get rid of the started processes:-
+
+        $MANAGIX_HOME/bin/managix stop -n my_asterix
+
+
+Any processes associated with the instance are killed and the instance moves to the INACTIVE state.
+You may now delete the instance by executing the following
+
+
+        $MANAGIX_HOME/bin/managix delete -n <name of your ASTERIX instance>
+
+
+Note that above would remove all traces of the instance including the logs and thus the reason for the failed attempt.
+
+OR
+
+make a subsequent attempt to start the instance if you realized a mistake in the cluster configuration XML and have corrected it. To start the instance, we execute the following.
+
+
+        $MANAGIX_HOME/bin/managix start -n <name of your ASTERIX instance>
+
diff --git a/asterix-events/pom.xml b/asterix-events/pom.xml
index 7b187bb..94042e1 100644
--- a/asterix-events/pom.xml
+++ b/asterix-events/pom.xml
@@ -166,5 +166,12 @@
       <artifactId>commons-io</artifactId>
       <version>1.4</version>
     </dependency>
+   <dependency>
+     <groupId>edu.uci.ics.asterix</groupId>
+     <artifactId>asterix-common</artifactId>
+     <version>0.0.6-SNAPSHOT</version>
+     <type>jar</type>
+     <scope>compile</scope>
+    </dependency>
   </dependencies>
 </project>
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
index 352d787..e702ef3 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
@@ -53,7 +53,7 @@
                 if (p.getKey().equals("JAVA_HOME")) {
                     String val = node.getJavaHome() == null ? p.getValue() : node.getJavaHome();
                     envBuffer.append(p.getKey() + "=" + val + " ");
-                } else if (p.getKey().equals("NC_JAVA_OPTS")) {
+                } else if (p.getKey().equals(EventUtil.NC_JAVA_OPTS)) {
                     if (!isMasterNode) {
                         StringBuilder builder = new StringBuilder();
                         builder.append("\"");
@@ -64,7 +64,7 @@
                         builder.append("\"");
                         envBuffer.append("JAVA_OPTS" + "=" + builder + " ");
                     }
-                } else if (p.getKey().equals("CC_JAVA_OPTS")) {
+                } else if (p.getKey().equals(EventUtil.CC_JAVA_OPTS)) {
                     if (isMasterNode) {
                         StringBuilder builder = new StringBuilder();
                         builder.append("\"");
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
index 7cbb515..533b2a4 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
@@ -36,230 +36,252 @@
 
 public class EventUtil {
 
-    public static final String EVENTS_DIR = "events";
-    public static final String CLUSTER_CONF = "config/cluster.xml";
-    public static final String PATTERN_CONF = "config/pattern.xml";
-    public static final DateFormat dateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss");
+	public static final String EVENTS_DIR = "events";
+	public static final String CLUSTER_CONF = "config/cluster.xml";
+	public static final String PATTERN_CONF = "config/pattern.xml";
+	public static final DateFormat dateFormat = new SimpleDateFormat(
+			"yyyy/MM/dd HH:mm:ss");
+	public static final String NC_JAVA_OPTS = "nc.java.opts";
+	public static final String CC_JAVA_OPTS = "cc.java.opts";
 
-    private static final String IP_LOCATION = "IP_LOCATION";
-    private static final String CLUSTER_ENV = "ENV";
-    private static final String SCRIPT = "SCRIPT";
-    private static final String ARGS = "ARGS";
-    private static final String EXECUTE_SCRIPT = "events/execute.sh";
-    private static final String LOCALHOST = "localhost";
-    private static final String LOCALHOST_IP = "127.0.0.1";
+	private static final String IP_LOCATION = "IP_LOCATION";
+	private static final String CLUSTER_ENV = "ENV";
+	private static final String SCRIPT = "SCRIPT";
+	private static final String ARGS = "ARGS";
+	private static final String EXECUTE_SCRIPT = "events/execute.sh";
+	private static final String LOCALHOST = "localhost";
+	private static final String LOCALHOST_IP = "127.0.0.1";
 
-    public static Cluster getCluster(String clusterConfigurationPath) throws JAXBException {
-        File file = new File(clusterConfigurationPath);
-        JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
-        Unmarshaller unmarshaller = ctx.createUnmarshaller();
-        Cluster cluster = (Cluster) unmarshaller.unmarshal(file);
-        if (cluster.getMasterNode().getClusterIp().equals(LOCALHOST)) {
-            cluster.getMasterNode().setClusterIp(LOCALHOST_IP);
-        }
-        for (Node node : cluster.getNode()) {
-            if (node.getClusterIp().equals(LOCALHOST)) {
-                node.setClusterIp(LOCALHOST_IP);
-            }
-        }
-        return cluster;
-    }
+	public static Cluster getCluster(String clusterConfigurationPath)
+			throws JAXBException {
+		File file = new File(clusterConfigurationPath);
+		JAXBContext ctx = JAXBContext.newInstance(Cluster.class);
+		Unmarshaller unmarshaller = ctx.createUnmarshaller();
+		Cluster cluster = (Cluster) unmarshaller.unmarshal(file);
+		if (cluster.getMasterNode().getClusterIp().equals(LOCALHOST)) {
+			cluster.getMasterNode().setClusterIp(LOCALHOST_IP);
+		}
+		for (Node node : cluster.getNode()) {
+			if (node.getClusterIp().equals(LOCALHOST)) {
+				node.setClusterIp(LOCALHOST_IP);
+			}
+		}
+		return cluster;
+	}
 
-    public static long parseTimeInterval(ValueType v, String unit) throws IllegalArgumentException {
-        int val = 0;
-        switch (v.getType()) {
-            case ABS:
-                val = Integer.parseInt(v.getAbsoluteValue());
-                break;
-            case RANDOM_MIN_MAX:
-                val = Randomizer.getInstance().getRandomInt(v.getMin(), v.getMax());
-                break;
-            case RANDOM_RANGE:
-                String[] values = v.getRangeSet();
-                val = Integer.parseInt(values[Randomizer.getInstance().getRandomInt(0, values.length - 1)]);
-                break;
-        }
-        return computeInterval(val, unit);
-    }
+	public static long parseTimeInterval(ValueType v, String unit)
+			throws IllegalArgumentException {
+		int val = 0;
+		switch (v.getType()) {
+		case ABS:
+			val = Integer.parseInt(v.getAbsoluteValue());
+			break;
+		case RANDOM_MIN_MAX:
+			val = Randomizer.getInstance().getRandomInt(v.getMin(), v.getMax());
+			break;
+		case RANDOM_RANGE:
+			String[] values = v.getRangeSet();
+			val = Integer.parseInt(values[Randomizer.getInstance()
+					.getRandomInt(0, values.length - 1)]);
+			break;
+		}
+		return computeInterval(val, unit);
+	}
 
-    public static long parseTimeInterval(String v, String unit) throws IllegalArgumentException {
-        int value = Integer.parseInt(v);
-        return computeInterval(value, unit);
-    }
+	public static long parseTimeInterval(String v, String unit)
+			throws IllegalArgumentException {
+		int value = Integer.parseInt(v);
+		return computeInterval(value, unit);
+	}
 
-    private static long computeInterval(int val, String unit) {
-        int vmult = 1;
-        if ("hr".equalsIgnoreCase(unit)) {
-            vmult = 3600 * 1000;
-        } else if ("min".equalsIgnoreCase(unit)) {
-            vmult = 60 * 1000;
-        } else if ("sec".equalsIgnoreCase(unit)) {
-            vmult = 1000;
-        } else
-            throw new IllegalArgumentException(" invalid unit value specified for frequency (hr,min,sec)");
-        return val * vmult;
+	private static long computeInterval(int val, String unit) {
+		int vmult = 1;
+		if ("hr".equalsIgnoreCase(unit)) {
+			vmult = 3600 * 1000;
+		} else if ("min".equalsIgnoreCase(unit)) {
+			vmult = 60 * 1000;
+		} else if ("sec".equalsIgnoreCase(unit)) {
+			vmult = 1000;
+		} else
+			throw new IllegalArgumentException(
+					" invalid unit value specified for frequency (hr,min,sec)");
+		return val * vmult;
 
-    }
+	}
 
-    public static Event getEvent(Pattern pattern, Events events) {
-        for (Event event : events.getEvent()) {
-            if (event.getType().equals(pattern.getEvent().getType())) {
-                return event;
-            }
-        }
-        throw new IllegalArgumentException(" Unknown event type" + pattern.getEvent().getType());
-    }
+	public static Event getEvent(Pattern pattern, Events events) {
+		for (Event event : events.getEvent()) {
+			if (event.getType().equals(pattern.getEvent().getType())) {
+				return event;
+			}
+		}
+		throw new IllegalArgumentException(" Unknown event type"
+				+ pattern.getEvent().getType());
+	}
 
-    public static Node getEventLocation(Pattern pattern, List<Node> candidateLocations, Cluster cluster) {
-        ValueType value = new ValueType(pattern.getEvent().getNodeid().getValue());
-        Node location = null;
-        Type vtype = value.getType();
+	public static Node getEventLocation(Pattern pattern,
+			List<Node> candidateLocations, Cluster cluster) {
+		ValueType value = new ValueType(pattern.getEvent().getNodeid()
+				.getValue());
+		Node location = null;
+		Type vtype = value.getType();
 
-        switch (vtype) {
-            case ABS:
-                location = getNodeFromId(value.getAbsoluteValue(), cluster);
-                break;
-            case RANDOM_RANGE:
-                int nodeIndex = Randomizer.getInstance().getRandomInt(0, candidateLocations.size() - 1);
-                location = candidateLocations.get(nodeIndex);
-                break;
-            case RANDOM_MIN_MAX:
-                throw new IllegalStateException(" Canont configure a min max value range for location");
-        }
-        return location;
+		switch (vtype) {
+		case ABS:
+			location = getNodeFromId(value.getAbsoluteValue(), cluster);
+			break;
+		case RANDOM_RANGE:
+			int nodeIndex = Randomizer.getInstance().getRandomInt(0,
+					candidateLocations.size() - 1);
+			location = candidateLocations.get(nodeIndex);
+			break;
+		case RANDOM_MIN_MAX:
+			throw new IllegalStateException(
+					" Canont configure a min max value range for location");
+		}
+		return location;
 
-    }
+	}
 
-    public static List<Node> getCandidateLocations(Pattern pattern, Cluster cluster) {
-        ValueType value = new ValueType(pattern.getEvent().getNodeid().getValue());
-        List<Node> candidateList = new ArrayList<Node>();
-        switch (value.getType()) {
-            case ABS:
-                candidateList.add(getNodeFromId(value.getAbsoluteValue(), cluster));
-                break;
-            case RANDOM_RANGE:
-                boolean anyOption = false;
-                String[] values = value.getRangeSet();
-                for (String v : values) {
-                    if (v.equalsIgnoreCase("ANY")) {
-                        anyOption = true;
-                    }
-                }
-                if (anyOption) {
-                    for (Node node : cluster.getNode()) {
-                        candidateList.add(node);
-                    }
-                } else {
-                    boolean found = false;
-                    for (String v : values) {
-                        for (Node node : cluster.getNode()) {
-                            if (node.getId().equals(v)) {
-                                candidateList.add(node);
-                                found = true;
-                                break;
-                            }
-                        }
-                        if (!found) {
-                            throw new IllegalStateException("Unknonw nodeId : " + v);
-                        }
-                        found = false;
-                    }
+	public static List<Node> getCandidateLocations(Pattern pattern,
+			Cluster cluster) {
+		ValueType value = new ValueType(pattern.getEvent().getNodeid()
+				.getValue());
+		List<Node> candidateList = new ArrayList<Node>();
+		switch (value.getType()) {
+		case ABS:
+			candidateList.add(getNodeFromId(value.getAbsoluteValue(), cluster));
+			break;
+		case RANDOM_RANGE:
+			boolean anyOption = false;
+			String[] values = value.getRangeSet();
+			for (String v : values) {
+				if (v.equalsIgnoreCase("ANY")) {
+					anyOption = true;
+				}
+			}
+			if (anyOption) {
+				for (Node node : cluster.getNode()) {
+					candidateList.add(node);
+				}
+			} else {
+				boolean found = false;
+				for (String v : values) {
+					for (Node node : cluster.getNode()) {
+						if (node.getId().equals(v)) {
+							candidateList.add(node);
+							found = true;
+							break;
+						}
+					}
+					if (!found) {
+						throw new IllegalStateException("Unknonw nodeId : " + v);
+					}
+					found = false;
+				}
 
-                }
-                String[] excluded = value.getRangeExcluded();
-                if (excluded != null && excluded.length > 0) {
-                    List<Node> markedForRemoval = new ArrayList<Node>();
-                    for (String exclusion : excluded) {
-                        for (Node node : candidateList) {
-                            if (node.getId().equals(exclusion)) {
-                                markedForRemoval.add(node);
-                            }
-                        }
-                    }
-                    candidateList.removeAll(markedForRemoval);
-                }
-                break;
-            case RANDOM_MIN_MAX:
-                throw new IllegalStateException(" Invalid value configured for location");
-        }
-        return candidateList;
-    }
+			}
+			String[] excluded = value.getRangeExcluded();
+			if (excluded != null && excluded.length > 0) {
+				List<Node> markedForRemoval = new ArrayList<Node>();
+				for (String exclusion : excluded) {
+					for (Node node : candidateList) {
+						if (node.getId().equals(exclusion)) {
+							markedForRemoval.add(node);
+						}
+					}
+				}
+				candidateList.removeAll(markedForRemoval);
+			}
+			break;
+		case RANDOM_MIN_MAX:
+			throw new IllegalStateException(
+					" Invalid value configured for location");
+		}
+		return candidateList;
+	}
 
-    private static Node getNodeFromId(String nodeid, Cluster cluster) {
-        if (nodeid.equals(EventDriver.CLIENT_NODE.getId())) {
-            return EventDriver.CLIENT_NODE;
-        }
+	private static Node getNodeFromId(String nodeid, Cluster cluster) {
+		if (nodeid.equals(EventDriver.CLIENT_NODE.getId())) {
+			return EventDriver.CLIENT_NODE;
+		}
 
-        if (nodeid.equals(cluster.getMasterNode().getId())) {
-            String logDir = cluster.getMasterNode().getLogDir() == null ? cluster.getLogDir() : cluster.getMasterNode()
-                    .getLogDir();
-            String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster.getJavaHome() : cluster
-                    .getMasterNode().getJavaHome();
-            return new Node(cluster.getMasterNode().getId(), cluster.getMasterNode().getClusterIp(), javaHome, logDir,
-                    null, null, null);
-        }
+		if (nodeid.equals(cluster.getMasterNode().getId())) {
+			String logDir = cluster.getMasterNode().getLogDir() == null ? cluster
+					.getLogDir()
+					: cluster.getMasterNode().getLogDir();
+			String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster
+					.getJavaHome()
+					: cluster.getMasterNode().getJavaHome();
+			return new Node(cluster.getMasterNode().getId(), cluster
+					.getMasterNode().getClusterIp(), javaHome, logDir, null,
+					null, null);
+		}
 
-        List<Node> nodeList = cluster.getNode();
-        for (Node node : nodeList) {
-            if (node.getId().equals(nodeid)) {
-                return node;
-            }
-        }
-        StringBuffer buffer = new StringBuffer();
-        buffer.append(EventDriver.CLIENT_NODE.getId() + ",");
-        buffer.append(cluster.getMasterNode().getId() + ",");
-        for (Node v : cluster.getNode()) {
-            buffer.append(v.getId() + ",");
-        }
-        buffer.deleteCharAt(buffer.length() - 1);
-        throw new IllegalArgumentException("Unknown node id :" + nodeid + " valid ids:" + buffer);
-    }
+		List<Node> nodeList = cluster.getNode();
+		for (Node node : nodeList) {
+			if (node.getId().equals(nodeid)) {
+				return node;
+			}
+		}
+		StringBuffer buffer = new StringBuffer();
+		buffer.append(EventDriver.CLIENT_NODE.getId() + ",");
+		buffer.append(cluster.getMasterNode().getId() + ",");
+		for (Node v : cluster.getNode()) {
+			buffer.append(v.getId() + ",");
+		}
+		buffer.deleteCharAt(buffer.length() - 1);
+		throw new IllegalArgumentException("Unknown node id :" + nodeid
+				+ " valid ids:" + buffer);
+	}
 
-    public static void executeEventScript(Node node, String script, List<String> args, Cluster cluster)
-            throws IOException, InterruptedException {
-        List<String> pargs = new ArrayList<String>();
-        pargs.add("/bin/bash");
-        pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
-        StringBuffer argBuffer = new StringBuffer();
-        String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION + "=" + node.getClusterIp();
-        if (args != null) {
-            for (String arg : args) {
-                argBuffer.append(arg + " ");
-            }
-        }
-        ProcessBuilder pb = new ProcessBuilder(pargs);
-        pb.environment().putAll(EventDriver.getEnvironment());
-        pb.environment().put(IP_LOCATION, node.getClusterIp());
-        pb.environment().put(CLUSTER_ENV, env);
-        pb.environment().put(SCRIPT, script);
-        pb.environment().put(ARGS, argBuffer.toString());
-        pb.start();
-    }
+	public static void executeEventScript(Node node, String script,
+			List<String> args, Cluster cluster) throws IOException,
+			InterruptedException {
+		List<String> pargs = new ArrayList<String>();
+		pargs.add("/bin/bash");
+		pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
+		StringBuffer argBuffer = new StringBuffer();
+		String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION
+				+ "=" + node.getClusterIp();
+		if (args != null) {
+			for (String arg : args) {
+				argBuffer.append(arg + " ");
+			}
+		}
+		ProcessBuilder pb = new ProcessBuilder(pargs);
+		pb.environment().putAll(EventDriver.getEnvironment());
+		pb.environment().put(IP_LOCATION, node.getClusterIp());
+		pb.environment().put(CLUSTER_ENV, env);
+		pb.environment().put(SCRIPT, script);
+		pb.environment().put(ARGS, argBuffer.toString());
+		pb.start();
+	}
 
-    public static void executeLocalScript(Node node, String script, List<String> args) throws IOException,
-            InterruptedException {
-        List<String> pargs = new ArrayList<String>();
-        pargs.add("/bin/bash");
-        pargs.add(script);
-        if (args != null) {
-            pargs.addAll(args);
-        }
-        ProcessBuilder pb = new ProcessBuilder(pargs);
-        pb.environment().putAll(EventDriver.getEnvironment());
-        pb.environment().put(IP_LOCATION, node.getClusterIp());
-        pb.start();
-    }
+	public static void executeLocalScript(Node node, String script,
+			List<String> args) throws IOException, InterruptedException {
+		List<String> pargs = new ArrayList<String>();
+		pargs.add("/bin/bash");
+		pargs.add(script);
+		if (args != null) {
+			pargs.addAll(args);
+		}
+		ProcessBuilder pb = new ProcessBuilder(pargs);
+		pb.environment().putAll(EventDriver.getEnvironment());
+		pb.environment().put(IP_LOCATION, node.getClusterIp());
+		pb.start();
+	}
 
-    public static List<String> getEventArgs(Pattern pattern) {
-        List<String> pargs = new ArrayList<String>();
-        if (pattern.getEvent().getPargs() == null) {
-            return pargs;
-        }
-        String[] args = pattern.getEvent().getPargs().split(" ");
-        for (String arg : args) {
-            pargs.add(arg.trim());
-        }
-        return pargs;
-    }
+	public static List<String> getEventArgs(Pattern pattern) {
+		List<String> pargs = new ArrayList<String>();
+		if (pattern.getEvent().getPargs() == null) {
+			return pargs;
+		}
+		String[] args = pattern.getEvent().getPargs().split(" ");
+		for (String arg : args) {
+			pargs.add(arg.trim());
+		}
+		return pargs;
+	}
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
index c9395ca..2b884ef 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
@@ -51,6 +51,7 @@
 import edu.uci.ics.asterix.common.configuration.Store;
 import edu.uci.ics.asterix.event.driver.EventDriver;
 import edu.uci.ics.asterix.event.management.EventrixClient;
+import edu.uci.ics.asterix.event.management.EventUtil;
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.cluster.Env;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
@@ -96,10 +97,10 @@
             clusterProperties = new ArrayList<Property>();
         }
         for (edu.uci.ics.asterix.common.configuration.Property property : asterixConfiguration.getProperty()) {
-            if (property.getName().equalsIgnoreCase(AsterixInstance.CC_JAVA_OPTS)) {
-                clusterProperties.add(new Property("CC_JAVA_OPTS", property.getValue()));
-            } else if (property.getName().equalsIgnoreCase(AsterixInstance.NC_JAVA_OPTS)) {
-                clusterProperties.add(new Property("NC_JAVA_OPTS", property.getValue()));
+            if (property.getName().equalsIgnoreCase(EventUtil.CC_JAVA_OPTS)) {
+                clusterProperties.add(new Property(EventUtil.CC_JAVA_OPTS, property.getValue()));
+            } else if (property.getName().equalsIgnoreCase(EventUtil.NC_JAVA_OPTS)) {
+                clusterProperties.add(new Property(EventUtil.NC_JAVA_OPTS, property.getValue()));
             }
         }
         clusterProperties.add(new Property("ASTERIX_HOME", cluster.getWorkingDir().getDir() + File.separator
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
index 4216978..0f382f9 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
@@ -315,14 +315,11 @@
         List<Pattern> patternList = new ArrayList<Pattern>();
         Cluster cluster = instance.getCluster();
         Nodeid nodeid = null;
-        String pargs = null;
         Event event = null;
         for (Node node : cluster.getNode()) {
-            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
-            String primaryIODevice = iodevices.split(",")[0].trim();
-            pargs = primaryIODevice + File.separator + InstallerUtil.TXN_LOG_DIR;
+            String txnLogDir = node.getTxnLogDir() == null ? cluster.getTxnLogDir() : node.getTxnLogDir();
             nodeid = new Nodeid(new Value(null, node.getId()));
-            event = new Event("file_delete", nodeid, pargs);
+            event = new Event("file_delete", nodeid, txnLogDir);
             patternList.add(new Pattern(null, 1, null, event));
         }
 
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
index 1eba89f..9839b90 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
@@ -28,9 +28,7 @@
 
     private static final long serialVersionUID = 2874439550187520449L;
 
-    public static final String CC_JAVA_OPTS = "cc_java_opts";
-    public static final String NC_JAVA_OPTS = "nc_java_opts";
-
+  
     public enum State {
         ACTIVE,
         INACTIVE,
diff --git a/asterix-installer/src/main/resources/conf/asterix-configuration.xml b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
index 37ddf88..b6958c5 100644
--- a/asterix-installer/src/main/resources/conf/asterix-configuration.xml
+++ b/asterix-installer/src/main/resources/conf/asterix-configuration.xml
@@ -1,116 +1,190 @@
 <asterixConfiguration xmlns="asterixconf">
 
 	<property>
-		<name>nc_java_opts</name>
+		<name>nc.java.opts</name>
 		<value>-Xmx1024m</value>
-                <description>JVM parameters for each Node Contoller (NC)</description>
+		<description>JVM parameters for each Node Contoller (NC)</description>
 	</property>
 
 	<property>
-		<name>cc_java_opts</name>
+		<name>cc.java.opts</name>
 		<value>-Xmx1024m</value>
-                <description>JVM parameters for each Cluster Contoller (CC)</description>
+		<description>JVM parameters for each Cluster Contoller (CC)
+		</description>
 	</property>
 
 	<property>
-		<name>size_memory_component</name>
-		<value>512m</value>
-                <description></description>
+		<name>storage.buffercache.pagesize</name>
+		<value>32768</value>
+		<description>The page size in bytes for pages in the buffer cache.
+			(Default = "32768" // 32KB)
+		</description>
 	</property>
 
 	<property>
-		<name>total_size_memory_component</name>
-		<value>512m</value>
-                <description></description>
+		<name>storage.buffercache.numpages</name>
+		<value>1024</value>
+		<description>The number of pages allocated to the disk buffer cache.
+			(Default = "1024")
+		</description>
 	</property>
 
 	<property>
-		<name>log_buffer_num_pages</name>
+		<name>storage.buffercache.maxopenfiles</name>
+		<value>214748364</value>
+		<description>The maximum number of open files in the buffer cache.
+			(Default = "214748364")
+		</description>
+	</property>
+
+	<property>
+		<name>storage.memorycomponent.pagesize</name>
+		<value>32768</value>
+		<description>The page size in bytes for pages allocated to memory
+			components. (Default = "32768" // 32KB)
+		</description>
+	</property>
+
+	<property>
+		<name>storage.memorycomponent.numpages</name>
+		<value>4096</value>
+		<description>The number of pages to allocate for a memory component.
+			(Default = 4096)
+		</description>
+	</property>
+
+	<property>
+		<name>storage.memorycomponent.globalbudget</name>
+		<value>1073741824</value>
+		<description>The total size of memory in bytes that the sum of all
+			open memory
+			components cannot exceed. (Default = "1073741824" // 1GB)
+		</description>
+	</property>
+
+	<property>
+		<name>storage.lsm.mergethreshold</name>
+		<value>3</value>
+		<description>The number of on-disk components an LSM index can have
+			before a merge is triggered. (Default = 3)
+		</description>
+	</property>
+
+	<property>
+		<name>storage.lsm.bloomfilter.falsepositiverate</name>
+		<value>0.01</value>
+		<description>The maximum acceptable false positive rate for bloom
+			filters associated with LSM indexes. (Default = "0.01" // 1%)
+		</description>
+	</property>
+
+	<property>
+		<name>txn.log.buffer.numpages</name>
 		<value>8</value>
-                <description></description>
+		<description>The number of in-memory log buffer pages. (Default = "8")
+		</description>
 	</property>
 
 	<property>
-		<name>group_commit_interval</name>
-		<value>200ms</value>
-                <description></description>
+		<name>txn.log.buffer.pagesize</name>
+		<value>131072</value>
+		<description>The size of pages in the in-memory log buffer. (Default =
+			"131072" // 128KB)
+		</description>
 	</property>
 
 	<property>
-		<name>sort_op_memory</name>
-		<value>200m</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>join_op_memory</name>
-		<value>200m</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>web_interface_port</name>
-		<value>19001</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>num_pages_buffer_cache</name>
-		<value>8</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>log_level</name>
-		<value>INFO</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>lsn_threshold</name>
-		<value>64m</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>checkpointTermsInSecs</name>
-		<value>120</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>escalate_threshold_entity_to_dataset</name>
-		<value>8</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>num_pages_buffer_cache</name>
-		<value>1000</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>log_buffer_page_size</name>
-		<value>128k</value>
-                <description></description>
-	</property>
-
-	<property>
-		<name>log_partition_size</name>
+		<name>txn.log.partitionsize</name>
 		<value>2147483648</value>
-                <description></description>
+		<description>The maximum size of a log file partition allowed before
+			rotating the log to the next partition. (Default = "2147483648" //
+			2GB)
+		</description>
 	</property>
 
 	<property>
-		<name>shrink_timer_threshold</name>
+		<name>txn.log.groupcommitinterval</name>
+		<value>200</value>
+		<description>The group commit wait time in milliseconds. (Default =
+			"200" // 2ms)
+		</description>
+	</property>
+
+	<property>
+		<name>txn.log.checkpoint.lsnthreshold</name>
+		<value>67108864</value>
+		<description>The size of the window that the maximum LSN is allowed to
+			be ahead of the checkpoint LSN by. (Default = ""67108864" // 64M)
+		</description>
+	</property>
+
+	<property>
+		<name>txn.log.checkpoint.pollfrequency</name>
+		<value>120</value>
+		<description>The time in seconds between that the checkpoint thread
+			waits between polls. (Default = "120" // 120s)
+		</description>
+	</property>
+
+	<property>
+		<name>txn.lock.escalationthreshold</name>
+		<value>1000</value>
+		<description>The number of entity level locks that need to be acquired
+			before the locks are coalesced and escalated into a dataset level
+			lock. (Default = "1000")
+		</description>
+	</property>
+
+	<property>
+		<name>txn.lock.shrinktimer</name>
 		<value>120000</value>
-                <description></description>
+		<description>The time in milliseconds to wait before deallocating
+			unused lock manager memory. (Default = "120000" // 120s)
+		</description>
 	</property>
 
 	<property>
-		<name>log_buffer_num_pages</name>
-		<value>8</value>
-                <description></description>
+		<name>compiler.sortmemory</name>
+		<value>536870912</value>
+		<description>The amount of memory in bytes given to sort operations.
+			(Default = "536870912" // 512mb)
+		</description>
+	</property>
+
+	<property>
+		<name>compiler.joinmemory</name>
+		<value>536870912</value>
+		<description>The amount of memory in bytes given to join operations.
+			(Default = "536870912" // 512mb)
+		</description>
+	</property>
+
+	<property>
+		<name>compiler.framesize</name>
+		<value>32768</value>
+		<description>The Hyracks frame size that the compiler configures per
+			job. (Default = "32768" // 32KB)
+		</description>
+	</property>
+
+	<property>
+		<name>web.port</name>
+		<value>19001</value>
+		<description>The port for the ASTERIX web interface. (Default = 19001)
+		</description>
+	</property>
+
+	<property>
+		<name>api.port</name>
+		<value>19101</value>
+		<description>The port for the ASTERIX API server. (Default = 19101)
+		</description>
+	</property>
+
+	<property>
+		<name>log.level</name>
+		<value>INFO</value>
+		<description>The minimum log level to be displayed. (Default = INFO)
+		</description>
 	</property>
 </asterixConfiguration>
diff --git a/asterix-metadata/pom.xml b/asterix-metadata/pom.xml
index 425e8ff..6f70568 100644
--- a/asterix-metadata/pom.xml
+++ b/asterix-metadata/pom.xml
@@ -42,6 +42,12 @@
 			<scope>compile</scope>
 		</dependency>
 		<dependency>
+			<groupId>edu.uci.ics.asterix</groupId>
+			<artifactId>asterix-runtime</artifactId>
+			<version>0.0.6-SNAPSHOT</version>
+			<scope>compile</scope>
+		</dependency>
+		<dependency>
 			<groupId>org.apache.hadoop</groupId>
 			<artifactId>hadoop-core</artifactId>
 			<version>0.20.2</version>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
index 5442cd9..f9f5260 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataManager.java
@@ -20,6 +20,7 @@
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.common.transactions.JobId;
@@ -84,12 +85,14 @@
     private IAsterixStateProxy proxy;
     private IMetadataNode metadataNode;
     private final ReadWriteLock metadataLatch;
+    private final AsterixMetadataProperties metadataProperties;
 
-    public MetadataManager(IAsterixStateProxy proxy) {
+    public MetadataManager(IAsterixStateProxy proxy, AsterixMetadataProperties metadataProperties) {
         if (proxy == null) {
             throw new Error("Null proxy given to MetadataManager.");
         }
         this.proxy = proxy;
+        this.metadataProperties = metadataProperties;
         this.metadataNode = null;
         this.metadataLatch = new ReentrantReadWriteLock(true);
     }
@@ -104,7 +107,7 @@
             metadataNode = proxy.getMetadataNode();
             if (metadataNode == null) {
                 throw new Error("Failed to get the MetadataNode.\n" + "The MetadataNode was configured to run on NC: "
-                        + proxy.getAsterixProperties().getMetadataNodeName());
+                        + metadataProperties.getMetadataNodeName());
             }
         }
     }
@@ -543,10 +546,10 @@
             throw new MetadataException(e);
         }
     }
-    
+
     @Override
     public int getMostRecentDatasetId() throws MetadataException {
-        try { 
+        try {
             return metadataNode.getMostRecentDatasetId();
         } catch (RemoteException e) {
             throw new MetadataException(e);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java
index d67cd27..7f06c30 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/api/IAsterixStateProxy.java
@@ -19,17 +19,11 @@
 import java.rmi.Remote;
 import java.rmi.RemoteException;
 
-import edu.uci.ics.asterix.common.config.AsterixProperties;
-
 /**
  * Interface for setting/getting distributed state of Asterix.
  */
 public interface IAsterixStateProxy extends Remote, Serializable {
     public void setMetadataNode(IMetadataNode metadataNode) throws RemoteException;
 
-    public void setAsterixProperties(AsterixProperties asterixProperties) throws RemoteException;
-
     public IMetadataNode getMetadataNode() throws RemoteException;
-
-    public AsterixProperties getAsterixProperties() throws RemoteException;
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java
index 55aaf33..33f6994 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/AsterixStateProxy.java
@@ -19,7 +19,6 @@
 import java.rmi.server.UnicastRemoteObject;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.common.config.AsterixProperties;
 import edu.uci.ics.asterix.metadata.api.IAsterixStateProxy;
 import edu.uci.ics.asterix.metadata.api.IMetadataNode;
 
@@ -31,7 +30,6 @@
     private static final Logger LOGGER = Logger.getLogger(AsterixStateProxy.class.getName());
 
     private IMetadataNode metadataNode;
-    private AsterixProperties asterixProperties;
     private static final IAsterixStateProxy cc = new AsterixStateProxy();
 
     public static IAsterixStateProxy registerRemoteObject() throws RemoteException {
@@ -54,14 +52,4 @@
     public IMetadataNode getMetadataNode() throws RemoteException {
         return this.metadataNode;
     }
-
-    @Override
-    public void setAsterixProperties(AsterixProperties asterixProperity) throws RemoteException {
-        this.asterixProperties = asterixProperity;
-    }
-
-    @Override
-    public AsterixProperties getAsterixProperties() throws RemoteException {
-        return this.asterixProperties;
-    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 6d8945c..791666d 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -20,17 +20,18 @@
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Set;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.api.IAsterixAppRuntimeContext;
-import edu.uci.ics.asterix.common.config.AsterixProperties;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
-import edu.uci.ics.asterix.common.config.GlobalConfig;
+import edu.uci.ics.asterix.common.config.IAsterixPropertiesProvider;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
@@ -110,12 +111,14 @@
 
     private static String metadataNodeName;
     private static String metadataStore;
-    private static HashSet<String> nodeNames;
+    private static Set<String> nodeNames;
     private static String outputDir;
 
     private static IMetadataIndex[] primaryIndexes;
     private static IMetadataIndex[] secondaryIndexes;
 
+    private static IAsterixPropertiesProvider propertiesProvider;
+
     private static void initLocalIndexArrays() {
         primaryIndexes = new IMetadataIndex[] { MetadataPrimaryIndexes.DATAVERSE_DATASET,
                 MetadataPrimaryIndexes.DATASET_DATASET, MetadataPrimaryIndexes.DATATYPE_DATASET,
@@ -127,9 +130,10 @@
                 MetadataSecondaryIndexes.DATATYPENAME_ON_DATATYPE_INDEX };
     }
 
-    public static void startUniverse(AsterixProperties asterixProperties, INCApplicationContext ncApplicationContext,
-            boolean isNewUniverse) throws Exception {
+    public static void startUniverse(IAsterixPropertiesProvider asterixPropertiesProvider,
+            INCApplicationContext ncApplicationContext, boolean isNewUniverse) throws Exception {
         runtimeContext = (IAsterixAppRuntimeContext) ncApplicationContext.getApplicationObject();
+        propertiesProvider = asterixPropertiesProvider;
 
         // Initialize static metadata objects, such as record types and metadata
         // index descriptors.
@@ -149,9 +153,10 @@
         resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_INVERTED_INDEX,
                 new IndexResourceManager(ResourceType.LSM_INVERTED_INDEX, runtimeContext.getTransactionSubsystem()));
 
-        metadataNodeName = asterixProperties.getMetadataNodeName();
-        metadataStore = asterixProperties.getMetadataStore();
-        nodeNames = asterixProperties.getNodeNames();
+        AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
+        metadataNodeName = metadataProperties.getMetadataNodeName();
+        metadataStore = metadataProperties.getMetadataStore();
+        nodeNames = metadataProperties.getNodeNames();
         // nodeStores = asterixProperity.getStores();
 
         indexLifecycleManager = runtimeContext.getIndexLifecycleManager();
@@ -165,7 +170,7 @@
             // Begin a transaction against the metadata.
             // Lock the metadata in X mode.
             MetadataManager.INSTANCE.lock(mdTxnCtx, LockMode.X);
-            
+
             if (isNewUniverse) {
                 for (int i = 0; i < primaryIndexes.length; i++) {
                     enlistMetadataDataset(primaryIndexes[i], true);
@@ -196,7 +201,7 @@
                     LOGGER.info("Finished enlistment of metadata B-trees.");
                 }
             }
-            
+
             //#. initialize datasetIdFactory
             MetadataManager.INSTANCE.initializeDatasetIdFactory(mdTxnCtx);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
@@ -327,7 +332,7 @@
                 adapterFactoryClassName, DatasourceAdapter.AdapterType.INTERNAL);
     }
 
-    public static void enlistMetadataDataset(IMetadataIndex index, boolean create) throws Exception {
+    private static void enlistMetadataDataset(IMetadataIndex index, boolean create) throws Exception {
         String filePath = metadataStore + File.separator + index.getFileNameRelativePath();
         FileReference file = new FileReference(new File(filePath));
         IInMemoryBufferCache memBufferCache = new InMemoryBufferCache(new HeapBufferAllocator(), DEFAULT_MEM_PAGE_SIZE,
@@ -343,15 +348,17 @@
         if (create) {
             lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file, bufferCache,
                     fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
-                    runtimeContext.getLSMMergePolicy(), runtimeContext.getLSMBTreeOperationTrackerFactory(),
-                    runtimeContext.getLSMIOScheduler(), AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
+                    runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
+                    runtimeContext.getLSMBTreeOperationTrackerFactory(), runtimeContext.getLSMIOScheduler(),
+                    AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
             lsmBtree.create();
             resourceID = runtimeContext.getResourceIdFactory().createId();
             indexLifecycleManager.register(resourceID, lsmBtree);
 
+            AsterixStorageProperties storageProperties = propertiesProvider.getStorageProperties();
             ILocalResourceMetadata localResourceMetadata = new LSMBTreeLocalResourceMetadata(typeTraits,
                     comparatorFactories, bloomFilterKeyFields, index.isPrimaryIndex(),
-                    GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES);
+                    storageProperties.getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages());
             ILocalResourceFactoryProvider localResourceFactoryProvider = new PersistentLocalResourceFactoryProvider(
                     localResourceMetadata, LocalResource.LSMBTreeResource);
             ILocalResourceFactory localResourceFactory = localResourceFactoryProvider.getLocalResourceFactory();
@@ -363,8 +370,9 @@
             if (lsmBtree == null) {
                 lsmBtree = LSMBTreeUtils.createLSMTree(memBufferCache, memFreePageManager, ioManager, file,
                         bufferCache, fileMapProvider, typeTraits, comparatorFactories, bloomFilterKeyFields,
-                        runtimeContext.getLSMMergePolicy(), runtimeContext.getLSMBTreeOperationTrackerFactory(),
-                        runtimeContext.getLSMIOScheduler(), AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
+                        runtimeContext.getBloomFilterFalsePositiveRate(), runtimeContext.getLSMMergePolicy(),
+                        runtimeContext.getLSMBTreeOperationTrackerFactory(), runtimeContext.getLSMIOScheduler(),
+                        AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER);
                 indexLifecycleManager.register(resourceID, lsmBtree);
             }
         }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
index 70383f9..f79b9b7 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlCompiledMetadataDeclarations.java
@@ -22,7 +22,8 @@
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.annotations.TypeDataGen;
-import edu.uci.ics.asterix.common.config.AsterixProperties;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixMetadataProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
 import edu.uci.ics.asterix.formats.base.IDataFormat;
@@ -73,8 +74,9 @@
         this.dataverseName = dataverseName;
         this.outputFile = outputFile;
         this.config = config;
+        AsterixMetadataProperties metadataProperties = AsterixAppContextInfo.getInstance().getMetadataProperties();
         if (stores == null && online) {
-            this.stores = AsterixProperties.INSTANCE.getStores();
+            this.stores = metadataProperties.getStores();
         } else {
             this.stores = stores;
         }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index 6b96420..14020aa 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -23,7 +23,8 @@
 import java.util.Map;
 import java.util.logging.Logger;
 
-import edu.uci.ics.asterix.common.config.AsterixProperties;
+import edu.uci.ics.asterix.common.api.AsterixAppContextInfo;
+import edu.uci.ics.asterix.common.config.AsterixStorageProperties;
 import edu.uci.ics.asterix.common.config.DatasetConfig.DatasetType;
 import edu.uci.ics.asterix.common.config.DatasetConfig.IndexType;
 import edu.uci.ics.asterix.common.config.GlobalConfig;
@@ -66,6 +67,10 @@
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.om.util.NonTaggedFormatUtil;
+import edu.uci.ics.asterix.runtime.base.AsterixTupleFilterFactory;
+import edu.uci.ics.asterix.runtime.formats.FormatUtils;
+import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
+import edu.uci.ics.asterix.runtime.job.listener.JobEventListenerFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexInstantSearchOperationCallbackFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallbackFactory;
 import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexSearchOperationCallbackFactory;
@@ -148,6 +153,8 @@
     private final Dataverse defaultDataverse;
     private JobId jobId;
 
+    private final AsterixStorageProperties storageProperties;
+
     private static final Map<String, String> adapterFactoryMapping = initializeAdapterFactoryMapping();
 
     public String getPropertyValue(String propertyName) {
@@ -168,7 +175,8 @@
 
     public AqlMetadataProvider(Dataverse defaultDataverse) {
         this.defaultDataverse = defaultDataverse;
-        this.stores = AsterixProperties.INSTANCE.getStores();
+        this.stores = AsterixAppContextInfo.getInstance().getMetadataProperties().getStores();
+        this.storageProperties = AsterixAppContextInfo.getInstance().getStorageProperties();
     }
 
     public void setJobId(JobId jobId) {
@@ -543,7 +551,7 @@
                             storageProperties.getMemoryComponentPageSize(),
                             storageProperties.getMemoryComponentNumPages(),
                             storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
-                           
+
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeSearchOp, spPc.second);
 
         } catch (MetadataException me) {
@@ -611,8 +619,9 @@
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
-                    retainInput, searchCallbackFactory);
+                            storageProperties.getMemoryComponentPageSize(),
+                            storageProperties.getMemoryComponentNumPages(),
+                            storageProperties.getBloomFilterFalsePositiveRate()), retainInput, searchCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeSearchOp, spPc.second);
 
         } catch (MetadataException me) {
@@ -642,7 +651,6 @@
         ResultSetDataSink rsds = (ResultSetDataSink) sink;
         ResultSetSinkId rssId = (ResultSetSinkId) rsds.getId();
         ResultSetId rsId = rssId.getResultSetId();
-        String nodeName = rssId.getResultNodeName();
 
         ResultWriterOperatorDescriptor resultWriter = null;
         try {
@@ -653,8 +661,7 @@
             throw new AlgebricksException(e);
         }
 
-        AlgebricksPartitionConstraint apc = new AlgebricksAbsolutePartitionConstraint(new String[] { nodeName });
-        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(resultWriter, apc);
+        return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(resultWriter, null);
     }
 
     @Override
@@ -773,8 +780,9 @@
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
-                    NoOpOperationCallbackFactory.INSTANCE);
+                            storageProperties.getMemoryComponentPageSize(),
+                            storageProperties.getMemoryComponentNumPages(),
+                            storageProperties.getBloomFilterFalsePositiveRate()), NoOpOperationCallbackFactory.INSTANCE);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
         } catch (MetadataException me) {
@@ -840,9 +848,9 @@
                     new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES), null,
-                    modificationCallbackFactory);
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
+                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                            storageProperties.getBloomFilterFalsePositiveRate()), null, modificationCallbackFactory);
 
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
@@ -1036,9 +1044,10 @@
                     new LSMBTreeDataflowHelperFactory(AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER,
-                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
-                    filterFactory, modificationCallbackFactory);
+                            AsterixRuntimeComponentsProvider.LSMBTREE_PROVIDER, storageProperties
+                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
+                    modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(btreeBulkLoad,
                     splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1162,9 +1171,10 @@
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER,
-                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
-                    filterFactory, modificationCallbackFactory);
+                            AsterixRuntimeComponentsProvider.LSMINVERTEDINDEX_PROVIDER, storageProperties
+                                    .getMemoryComponentPageSize(), storageProperties.getMemoryComponentNumPages(),
+                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
+                    modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(insertDeleteOp,
                     splitsAndConstraint.second);
         } catch (MetadataException e) {
@@ -1258,8 +1268,10 @@
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER,
                             AsterixRuntimeComponentsProvider.LSMRTREE_PROVIDER, proposeLinearizer(
                                     nestedKeyType.getTypeTag(), comparatorFactories.length),
-                            GlobalConfig.DEFAULT_INDEX_MEM_PAGE_SIZE, GlobalConfig.DEFAULT_INDEX_MEM_NUM_PAGES),
-                    filterFactory, modificationCallbackFactory);
+                            storageProperties.getMemoryComponentPageSize(),
+                            storageProperties.getMemoryComponentNumPages(),
+                            storageProperties.getBloomFilterFalsePositiveRate()), filterFactory,
+                    modificationCallbackFactory);
             return new Pair<IOperatorDescriptor, AlgebricksPartitionConstraint>(rtreeUpdate, splitsAndConstraint.second);
         } catch (MetadataException | IOException e) {
             throw new AlgebricksException(e);
@@ -1296,6 +1308,17 @@
     public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForInternalOrFeedDataset(
             String dataverseName, String datasetName, String targetIdxName) throws AlgebricksException {
         FileSplit[] splits = splitsForInternalOrFeedDataset(mdTxnCtx, dataverseName, datasetName, targetIdxName);
+        return splitProviderAndPartitionConstraints(splits);
+    }
+
+    public Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraintsForDataverse(
+            String dataverse) {
+        FileSplit[] splits = splitsForDataverse(mdTxnCtx, dataverse);
+        return splitProviderAndPartitionConstraints(splits);
+    }
+
+    private Pair<IFileSplitProvider, AlgebricksPartitionConstraint> splitProviderAndPartitionConstraints(
+            FileSplit[] splits) {
         IFileSplitProvider splitProvider = new ConstantFileSplitProvider(splits);
         String[] loc = new String[splits.length];
         for (int p = 0; p < splits.length; p++) {
@@ -1305,6 +1328,23 @@
         return new Pair<IFileSplitProvider, AlgebricksPartitionConstraint>(splitProvider, pc);
     }
 
+    private FileSplit[] splitsForDataverse(MetadataTransactionContext mdTxnCtx, String dataverseName) {
+        File relPathFile = new File(dataverseName);
+        List<FileSplit> splits = new ArrayList<FileSplit>();
+        for (Map.Entry<String, String[]> entry : stores.entrySet()) {
+            String node = entry.getKey();
+            String[] nodeStores = entry.getValue();
+            if (nodeStores == null) {
+                continue;
+            }
+            for (int i = 0; i < nodeStores.length; i++) {
+                File f = new File(nodeStores[i] + File.separator + relPathFile);
+                splits.add(new FileSplit(node, new FileReference(f)));
+            }
+        }
+        return splits.toArray(new FileSplit[] {});
+    }
+
     private FileSplit[] splitsForInternalOrFeedDataset(MetadataTransactionContext mdTxnCtx, String dataverseName,
             String datasetName, String targetIdxName) throws AlgebricksException {
 
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
index 53513d2..ad91111 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetDataSink.java
@@ -17,6 +17,8 @@
 
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSink;
 import edu.uci.ics.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
+import edu.uci.ics.hyracks.algebricks.core.algebra.properties.ResultSetDomain;
 
 public class ResultSetDataSink implements IDataSink {
 
@@ -40,7 +42,6 @@
 
     @Override
     public IPartitioningProperty getPartitioningProperty() {
-        return IPartitioningProperty.UNPARTITIONED;
+        return new RandomPartitioningProperty(new ResultSetDomain());
     }
-
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
index fc5152f..1eb4336 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/ResultSetSinkId.java
@@ -20,23 +20,16 @@
 
     private final ResultSetId resultSetId;
 
-    private final String resultNodeName;
-
-    public ResultSetSinkId(ResultSetId resultSetId, String resultNodeName) {
+    public ResultSetSinkId(ResultSetId resultSetId) {
         this.resultSetId = resultSetId;
-        this.resultNodeName = resultNodeName;
     }
 
     @Override
     public String toString() {
-        return "ResultSetId: " + resultSetId + "@" + resultNodeName;
+        return "ResultSetId: " + resultSetId;
     }
 
     public ResultSetId getResultSetId() {
         return resultSetId;
     }
-
-    public String getResultNodeName() {
-        return resultNodeName;
-    }
 }
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java
index 27cf542..aa8f452 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/entities/AsterixBuiltinTypeMap.java
@@ -40,6 +40,8 @@
         _builtinTypeMap.put("time", BuiltinType.ATIME);
         _builtinTypeMap.put("datetime", BuiltinType.ADATETIME);
         _builtinTypeMap.put("duration", BuiltinType.ADURATION);
+        _builtinTypeMap.put("year-month-duration", BuiltinType.AYEARMONTHDURATION);
+        _builtinTypeMap.put("day-time-duration", BuiltinType.ADAYTIMEDURATION);
         _builtinTypeMap.put("interval", BuiltinType.AINTERVAL);
         _builtinTypeMap.put("point", BuiltinType.APOINT);
         _builtinTypeMap.put("point3d", BuiltinType.APOINT3D);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/builders/RecordBuilder.java b/asterix-om/src/main/java/edu/uci/ics/asterix/builders/RecordBuilder.java
index f5d07ae..10b6071 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/builders/RecordBuilder.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/builders/RecordBuilder.java
@@ -150,7 +150,7 @@
         // +1 because we do not store the value tag.
         closedPartOutputStream.write(value.getByteArray(), value.getStartOffset() + 1, len);
         numberOfClosedFields++;
-        if (isNullable && value.getByteArray()[0] != SER_NULL_TYPE_TAG) {
+        if (isNullable && value.getByteArray()[value.getStartOffset()] != SER_NULL_TYPE_TAG) {
             nullBitMap[id / 8] |= (byte) (1 << (7 - (id % 8)));
         }
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ACirclePartialBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ACirclePartialBinaryComparatorFactory.java
new file mode 100644
index 0000000..0c54ea1
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ACirclePartialBinaryComparatorFactory.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.Coordinate;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ACirclePartialBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final ACirclePartialBinaryComparatorFactory INSTANCE = new ACirclePartialBinaryComparatorFactory();
+
+    private ACirclePartialBinaryComparatorFactory() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory#createBinaryComparator()
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                try {
+                    // center.x
+                    int c = Double
+                            .compare(
+                                    ADoubleSerializerDeserializer.getDouble(
+                                            b1,
+                                            s1
+                                                    + ACircleSerializerDeserializer
+                                                            .getCenterPointCoordinateOffset(Coordinate.X) - 1),
+                                    ADoubleSerializerDeserializer.getDouble(
+                                            b2,
+                                            s2
+                                                    + ACircleSerializerDeserializer
+                                                            .getCenterPointCoordinateOffset(Coordinate.X) - 1));
+                    if (c == 0) {
+                        // center.y
+                        c = Double.compare(
+                                ADoubleSerializerDeserializer.getDouble(b1,
+                                        s1 + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y)
+                                                - 1),
+                                ADoubleSerializerDeserializer.getDouble(b2,
+                                        s2 + ACircleSerializerDeserializer.getCenterPointCoordinateOffset(Coordinate.Y)
+                                                - 1));
+                        if (c == 0) {
+                            // radius
+                            return Double.compare(
+                                    ADoubleSerializerDeserializer.getDouble(b1,
+                                            s1 + ACircleSerializerDeserializer.getRadiusOffset() - 1),
+                                    ADoubleSerializerDeserializer.getDouble(b2,
+                                            s2 + ACircleSerializerDeserializer.getRadiusOffset() - 1));
+                        }
+                    }
+                    return c;
+                } catch (HyracksDataException hex) {
+                    throw new IllegalStateException(hex);
+                }
+            }
+        };
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADateOrTimeAscBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADateOrTimeAscBinaryComparatorFactory.java
deleted file mode 100644
index 464a03c..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADateOrTimeAscBinaryComparatorFactory.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Copyright 2009-2011 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-
-public class ADateOrTimeAscBinaryComparatorFactory implements IBinaryComparatorFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final ADateOrTimeAscBinaryComparatorFactory INSTANCE = new ADateOrTimeAscBinaryComparatorFactory();
-
-    private ADateOrTimeAscBinaryComparatorFactory() {
-    }
-
-    /* (non-Javadoc)
-     * @see edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory#createBinaryComparator()
-     */
-    @Override
-    public IBinaryComparator createBinaryComparator() {
-        return new IBinaryComparator() {
-
-            @Override
-            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-                int chrononTime1 = getInt(b1, s1);
-                int chrononTime2 = getInt(b2, s2);
-
-                if (chrononTime1 > chrononTime2) {
-                    return 1;
-                } else if (chrononTime1 < chrononTime2) {
-                    return -1;
-                } else {
-                    return 0;
-                }
-            }
-
-            private int getInt(byte[] bytes, int start) {
-                return ((bytes[start] & 0xff) << 24) + ((bytes[start + 1] & 0xff) << 16)
-                        + ((bytes[start + 2] & 0xff) << 8) + ((bytes[start + 3] & 0xff) << 0);
-            }
-        };
-    }
-}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADateTimeAscBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADateTimeAscBinaryComparatorFactory.java
deleted file mode 100644
index 2dd8785..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADateTimeAscBinaryComparatorFactory.java
+++ /dev/null
@@ -1,43 +0,0 @@
-package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-
-public class ADateTimeAscBinaryComparatorFactory implements IBinaryComparatorFactory {
-    private static final long serialVersionUID = 1L;
-
-    public static final ADateTimeAscBinaryComparatorFactory INSTANCE = new ADateTimeAscBinaryComparatorFactory();
-
-    private ADateTimeAscBinaryComparatorFactory() {
-    }
-
-    @Override
-    public IBinaryComparator createBinaryComparator() {
-        return new IBinaryComparator() {
-
-            @Override
-            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
-
-                long chrononTime1 = getLong(b1, s1);
-                long chrononTime2 = getLong(b2, s2);
-
-                if (chrononTime1 > chrononTime2) {
-                    return 1;
-                } else if (chrononTime1 < chrononTime2) {
-                    return -1;
-                } else {
-                    return 0;
-                }
-            }
-
-            private long getLong(byte[] bytes, int start) {
-                return (((long) (bytes[start] & 0xff)) << 56) + (((long) (bytes[start + 1] & 0xff)) << 48)
-                        + (((long) (bytes[start + 2] & 0xff)) << 40) + (((long) (bytes[start + 3] & 0xff)) << 32)
-                        + (((long) (bytes[start + 4] & 0xff)) << 24) + (((long) (bytes[start + 5] & 0xff)) << 16)
-                        + (((long) (bytes[start + 6] & 0xff)) << 8) + (((long) (bytes[start + 7] & 0xff)) << 0);
-            }
-
-        };
-    }
-
-}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADurationPartialBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADurationPartialBinaryComparatorFactory.java
new file mode 100644
index 0000000..877426d
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ADurationPartialBinaryComparatorFactory.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class ADurationPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final ADurationPartialBinaryComparatorFactory INSTANCE = new ADurationPartialBinaryComparatorFactory();
+
+    private ADurationPartialBinaryComparatorFactory() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory#createBinaryComparator()
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int c = Integer.compare(
+                        AInt32SerializerDeserializer.getInt(b1,
+                                s1 + ADurationSerializerDeserializer.getYearMonthOffset()),
+                        AInt32SerializerDeserializer.getInt(b2,
+                                s2 + ADurationSerializerDeserializer.getYearMonthOffset()));
+                if (c == 0) {
+                    return Double.compare(
+                            ADoubleSerializerDeserializer.getDouble(b1,
+                                    s1 + ADurationSerializerDeserializer.getDayTimeOffset()),
+                            ADoubleSerializerDeserializer.getDouble(b2,
+                                    s2 + ADurationSerializerDeserializer.getDayTimeOffset()));
+                }
+                return c;
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java
new file mode 100644
index 0000000..69f5051
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AIntervalPartialBinaryComparatorFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class AIntervalPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final AIntervalPartialBinaryComparatorFactory INSTANCE = new AIntervalPartialBinaryComparatorFactory();
+
+    private AIntervalPartialBinaryComparatorFactory() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory#createBinaryComparator()
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int c = Double.compare(
+                        AInt64SerializerDeserializer.getLong(b1,
+                                s1 + AIntervalSerializerDeserializer.getIntervalStartOffset()),
+                        AInt64SerializerDeserializer.getLong(b2,
+                                s2 + AIntervalSerializerDeserializer.getIntervalStartOffset()));
+                if (c == 0) {
+                    c = Double.compare(
+                            AInt64SerializerDeserializer.getLong(b1,
+                                    s1 + AIntervalSerializerDeserializer.getIntervalEndOffset()),
+                            AInt64SerializerDeserializer.getLong(b2,
+                                    s2 + AIntervalSerializerDeserializer.getIntervalEndOffset()));
+                    if (c == 0) {
+                        c = Byte.compare(b1[s1 + 16], b2[s2 + 16]);
+                    }
+                }
+                return c;
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ALinePartialBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ALinePartialBinaryComparatorFactory.java
new file mode 100644
index 0000000..9f0fb57
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ALinePartialBinaryComparatorFactory.java
@@ -0,0 +1,88 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.Coordinate;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ALineSerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ALinePartialBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final ALinePartialBinaryComparatorFactory INSTANCE = new ALinePartialBinaryComparatorFactory();
+
+    private ALinePartialBinaryComparatorFactory() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory#createBinaryComparator()
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                try {
+                    int c = Double.compare(
+                            ADoubleSerializerDeserializer.getDouble(b1,
+                                    s1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X) - 1),
+                            ADoubleSerializerDeserializer.getDouble(b2,
+                                    s2 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.X) - 1));
+                    if (c == 0) {
+                        c = Double.compare(
+                                ADoubleSerializerDeserializer.getDouble(b1,
+                                        s1 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y)
+                                                - 1),
+                                ADoubleSerializerDeserializer.getDouble(b2,
+                                        s2 + ALineSerializerDeserializer.getStartPointCoordinateOffset(Coordinate.Y)
+                                                - 1));
+                        if (c == 0) {
+                            c = Double.compare(
+                                    ADoubleSerializerDeserializer.getDouble(b1,
+                                            s1 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X)
+                                                    - 1),
+                                    ADoubleSerializerDeserializer.getDouble(b2,
+                                            s2 + ALineSerializerDeserializer.getEndPointCoordinateOffset(Coordinate.X)
+                                                    - 1));
+                            if (c == 0) {
+                                return Double.compare(
+                                        ADoubleSerializerDeserializer.getDouble(
+                                                b1,
+                                                s1
+                                                        + ALineSerializerDeserializer
+                                                                .getEndPointCoordinateOffset(Coordinate.Y) - 1),
+                                        ADoubleSerializerDeserializer.getDouble(
+                                                b2,
+                                                s2
+                                                        + ALineSerializerDeserializer
+                                                                .getEndPointCoordinateOffset(Coordinate.Y) - 1));
+                            }
+                        }
+                    }
+                    return c;
+                } catch (HyracksDataException hex) {
+                    throw new IllegalStateException(hex);
+                }
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
index 0e486e7..f795b17 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
@@ -2,7 +2,6 @@
 
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryComparatorFactory;
@@ -33,12 +32,22 @@
                     .createBinaryComparator();
             final IBinaryComparator ascDoubleComp = new PointableBinaryComparatorFactory(DoublePointable.FACTORY)
                     .createBinaryComparator();
-            final IBinaryComparator ascRectangleComp = RectangleBinaryComparatorFactory.INSTANCE
+            final IBinaryComparator ascRectangleComp = ARectanglePartialBinaryComparatorFactory.INSTANCE
                     .createBinaryComparator();
-            final IBinaryComparator ascDateTimeComp = ADateTimeAscBinaryComparatorFactory.INSTANCE
+            final IBinaryComparator ascCircleComp = ACirclePartialBinaryComparatorFactory.INSTANCE
                     .createBinaryComparator();
-            final IBinaryComparator ascDateOrTimeComp = ADateOrTimeAscBinaryComparatorFactory.INSTANCE
+            final IBinaryComparator ascDurationComp = ADurationPartialBinaryComparatorFactory.INSTANCE
                     .createBinaryComparator();
+            final IBinaryComparator ascIntervalComp = AIntervalPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascLineComp = ALinePartialBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+            final IBinaryComparator ascPointComp = APointPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascPoint3DComp = APoint3DPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator ascPolygonComp = APolygonPartialBinaryComparatorFactory.INSTANCE
+                    .createBinaryComparator();
+            final IBinaryComparator rawComp = RawBinaryComparatorFactory.INSTANCE.createBinaryComparator();
 
             @Override
             public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
@@ -53,14 +62,24 @@
                         return 1;
                 }
 
-                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b1[s1]);
-                switch (tag) {
+                ATypeTag tag1 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b1[s1]);
+                ATypeTag tag2 = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b2[s2]);
+                if (tag1 != tag2) {
+                    throw new IllegalStateException("The values of two inconsistent types (" + tag1 + " and " + tag2
+                            + ") cannot be compared!");
+                }
+                switch (tag1) {
                     case BOOLEAN: {
                         return ascBoolComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
+                    case TIME:
+                    case DATE:
+                    case YEARMONTHDURATION:
                     case INT32: {
                         return ascIntComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
+                    case DATETIME:
+                    case DAYTIMEDURATION:
                     case INT64: {
                         return ascLongComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
@@ -76,15 +95,29 @@
                     case RECTANGLE: {
                         return ascRectangleComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
-                    case DATETIME: {
-                        return ascDateTimeComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    case CIRCLE: {
+                        return ascCircleComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
-                    case TIME:
-                    case DATE: {
-                        return ascDateOrTimeComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    case POINT: {
+                        return ascPointComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    }
+                    case POINT3D: {
+                        return ascPoint3DComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    }
+                    case LINE: {
+                        return ascLineComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    }
+                    case POLYGON: {
+                        return ascPolygonComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    }
+                    case DURATION: {
+                        return ascDurationComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
+                    }
+                    case INTERVAL: {
+                        return ascIntervalComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
                     default: {
-                        throw new NotImplementedException("Comparison for type " + tag + " is not implemented");
+                        return rawComp.compare(b1, s1 + 1, l1 - 1, b2, s2 + 1, l2 - 1);
                     }
                 }
             }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APoint3DPartialBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APoint3DPartialBinaryComparatorFactory.java
new file mode 100644
index 0000000..caf855a
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APoint3DPartialBinaryComparatorFactory.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.Coordinate;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APoint3DSerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksException;
+
+public class APoint3DPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final APoint3DPartialBinaryComparatorFactory INSTANCE = new APoint3DPartialBinaryComparatorFactory();
+
+    private APoint3DPartialBinaryComparatorFactory() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory#createBinaryComparator()
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                try {
+                    int c = Double.compare(
+                            ADoubleSerializerDeserializer.getDouble(b1,
+                                    s1 + APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.X) - 1),
+                            ADoubleSerializerDeserializer.getDouble(b2,
+                                    s2 + APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.X) - 1));
+                    if (c == 0) {
+                        c = Double.compare(
+                                ADoubleSerializerDeserializer.getDouble(b1,
+                                        s1 + APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.Y) - 1),
+                                ADoubleSerializerDeserializer.getDouble(b2,
+                                        s2 + APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.Y) - 1));
+                        if (c == 0) {
+                            return Double.compare(
+                                    ADoubleSerializerDeserializer.getDouble(b1,
+                                            s1 + APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.Z) - 1),
+                                    ADoubleSerializerDeserializer.getDouble(b2,
+                                            s2 + APoint3DSerializerDeserializer.getCoordinateOffset(Coordinate.Z) - 1));
+                        }
+                    }
+                    return c;
+                } catch (HyracksException hex) {
+                    throw new IllegalStateException(hex);
+                }
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APointPartialBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APointPartialBinaryComparatorFactory.java
new file mode 100644
index 0000000..24d47eb
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APointPartialBinaryComparatorFactory.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.Coordinate;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APointSerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class APointPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public final static APointPartialBinaryComparatorFactory INSTANCE = new APointPartialBinaryComparatorFactory();
+
+    private APointPartialBinaryComparatorFactory() {
+
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                try {
+                    int c = Double.compare(
+                            ADoubleSerializerDeserializer.getDouble(b1,
+                                    s1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X) - 1),
+                            ADoubleSerializerDeserializer.getDouble(b2,
+                                    s2 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.X) - 1));
+                    if (c == 0) {
+                        return Double.compare(
+                                ADoubleSerializerDeserializer.getDouble(b1,
+                                        s1 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y) - 1),
+                                ADoubleSerializerDeserializer.getDouble(b2,
+                                        s2 + APointSerializerDeserializer.getCoordinateOffset(Coordinate.Y) - 1));
+                    }
+                    return c;
+                } catch (HyracksDataException hex) {
+                    throw new IllegalStateException(hex);
+                }
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APolygonPartialBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APolygonPartialBinaryComparatorFactory.java
new file mode 100644
index 0000000..fec245f
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/APolygonPartialBinaryComparatorFactory.java
@@ -0,0 +1,90 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.Coordinate;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+import edu.uci.ics.hyracks.dataflow.common.data.marshalling.DoubleSerializerDeserializer;
+
+public class APolygonPartialBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+
+    public final static APolygonPartialBinaryComparatorFactory INSTANCE = new APolygonPartialBinaryComparatorFactory();
+
+    private APolygonPartialBinaryComparatorFactory() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory#createBinaryComparator()
+     */
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                try {
+                    short pointCount1 = AInt16SerializerDeserializer.getShort(b1,
+                            s1 + APolygonSerializerDeserializer.getNumberOfPointsOffset() - 1);
+                    int c = Short.compare(
+                            pointCount1,
+                            AInt16SerializerDeserializer.getShort(b2,
+                                    s2 + APolygonSerializerDeserializer.getNumberOfPointsOffset() - 1));
+
+                    if (c == 0) {
+                        int ci = 0;
+                        for (int i = 0; i < pointCount1; i++) {
+                            ci = Double.compare(
+                                    DoubleSerializerDeserializer.getDouble(b1,
+                                            s1 + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.X)
+                                                    - 1),
+                                    DoubleSerializerDeserializer.getDouble(b2,
+                                            s1 + APolygonSerializerDeserializer.getCoordinateOffset(i, Coordinate.X)
+                                                    - 1));
+                            if (ci == 0) {
+                                ci = Double.compare(
+                                        DoubleSerializerDeserializer.getDouble(
+                                                b1,
+                                                s1
+                                                        + APolygonSerializerDeserializer.getCoordinateOffset(i,
+                                                                Coordinate.Y) - 1),
+                                        DoubleSerializerDeserializer.getDouble(
+                                                b2,
+                                                s1
+                                                        + APolygonSerializerDeserializer.getCoordinateOffset(i,
+                                                                Coordinate.Y) - 1));
+                                if (ci == 0) {
+                                    continue;
+                                }
+                            }
+                            return ci;
+                        }
+                    }
+
+                    return c;
+                } catch (HyracksDataException hex) {
+                    throw new IllegalStateException(hex);
+                }
+            }
+        };
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/RectangleBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ARectanglePartialBinaryComparatorFactory.java
similarity index 86%
rename from asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/RectangleBinaryComparatorFactory.java
rename to asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ARectanglePartialBinaryComparatorFactory.java
index bdc8409..63d0d24 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/RectangleBinaryComparatorFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/ARectanglePartialBinaryComparatorFactory.java
@@ -4,12 +4,12 @@
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 
-public class RectangleBinaryComparatorFactory implements IBinaryComparatorFactory {
+public class ARectanglePartialBinaryComparatorFactory implements IBinaryComparatorFactory {
     private static final long serialVersionUID = 1L;
 
-    public final static RectangleBinaryComparatorFactory INSTANCE = new RectangleBinaryComparatorFactory();
+    public final static ARectanglePartialBinaryComparatorFactory INSTANCE = new ARectanglePartialBinaryComparatorFactory();
 
-    private RectangleBinaryComparatorFactory() {
+    private ARectanglePartialBinaryComparatorFactory() {
     }
 
     @Override
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/RawBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/RawBinaryComparatorFactory.java
new file mode 100644
index 0000000..2c4f951
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/RawBinaryComparatorFactory.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.dataflow.data.nontagged.comparators;
+
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
+import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class RawBinaryComparatorFactory implements IBinaryComparatorFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static IBinaryComparatorFactory INSTANCE = new RawBinaryComparatorFactory();
+
+    private RawBinaryComparatorFactory() {
+    }
+
+    @Override
+    public IBinaryComparator createBinaryComparator() {
+        return new IBinaryComparator() {
+
+            @Override
+            public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) {
+                int commonLength = Math.min(l1, l2);
+                for (int i = 0; i < commonLength; i++) {
+                    if (b1[s1 + i] != b2[s2 + i]) {
+                        return b1[s1 + i] - b2[s2 + i];
+                    }
+                }
+                int difference = l1 - l2;
+                return difference == 0 ? 0 : (difference > 0 ? 1 : -1);
+            }
+
+        };
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java
index 15fd8c8..495b41f 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/AObjectBinaryHashFunctionFactory.java
@@ -2,9 +2,9 @@
 
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.EnumDeserializer;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
@@ -34,8 +34,9 @@
 
             private IBinaryHashFunction doubleHash = DoubleBinaryHashFunctionFactory.INSTANCE
                     .createBinaryHashFunction();
-            private IBinaryHashFunction rectangleHash = RectangleBinaryHashFunctionFactory.INSTANCE
-                    .createBinaryHashFunction();
+
+            private IBinaryHashFunction genericBinaryHash = MurmurHash3BinaryHashFunctionFamily.INSTANCE
+                    .createBinaryHashFunction(0);
 
             @Override
             public int hash(byte[] bytes, int offset, int length) {
@@ -44,9 +45,14 @@
                     case BOOLEAN: {
                         return boolHash.hash(bytes, offset + 1, length - 1);
                     }
+                    case TIME:
+                    case DATE:
+                    case YEARMONTHDURATION:
                     case INT32: {
                         return intHash.hash(bytes, offset + 1, length - 1);
                     }
+                    case DATETIME:
+                    case DAYTIMEDURATION:
                     case INT64: {
                         return longHash.hash(bytes, offset + 1, length - 1);
                     }
@@ -59,14 +65,11 @@
                     case STRING: {
                         return stringHash.hash(bytes, offset + 1, length - 1);
                     }
-                    case RECTANGLE: {
-                        return rectangleHash.hash(bytes, offset + 1, length - 1);
-                    }
                     case NULL: {
                         return 0;
                     }
                     default: {
-                        throw new NotImplementedException("Binary hashing for the " + tag + " type is not implemented.");
+                        return genericBinaryHash.hash(bytes, offset + 1, length - 1);
                     }
                 }
             }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/MurmurHash3BinaryHashFunctionFamily.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/MurmurHash3BinaryHashFunctionFamily.java
deleted file mode 100644
index 83b165b..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/MurmurHash3BinaryHashFunctionFamily.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Copyright 2009-2010 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package edu.uci.ics.asterix.dataflow.data.nontagged.hash;
-
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
-
-/**
- * An implementation of the Murmur3 hash family. The code is implemented based
- * on the original <a
- * href=http://code.google.com/p/guava-libraries/source/browse
- * /guava/src/com/google/common/hash/Murmur3_32HashFunction.java>guava
- * implementation</a> from Google Guava library.
- */
-public class MurmurHash3BinaryHashFunctionFamily implements
-		IBinaryHashFunctionFamily {
-
-	public static final IBinaryHashFunctionFamily INSTANCE = new MurmurHash3BinaryHashFunctionFamily();
-
-	private static final long serialVersionUID = 1L;
-
-	private MurmurHash3BinaryHashFunctionFamily() {
-	}
-
-	private static final int C1 = 0xcc9e2d51;
-	private static final int C2 = 0x1b873593;
-	private static final int C3 = 5;
-	private static final int C4 = 0xe6546b64;
-	private static final int C5 = 0x85ebca6b;
-	private static final int C6 = 0xc2b2ae35;
-
-	@Override
-	public IBinaryHashFunction createBinaryHashFunction(final int seed) {
-		return new IBinaryHashFunction() {
-			@Override
-			public int hash(byte[] bytes, int offset, int length) {
-				int h = seed;
-				int p = offset;
-				int remain = length;
-				while (remain >= 4) {
-					int k = (bytes[p] & 0xff) | ((bytes[p + 1] & 0xff) << 8)
-							| ((bytes[p + 2] & 0xff) << 16)
-							| ((bytes[p + 3] & 0xff) << 24);
-					k *= C1;
-					k = Integer.rotateLeft(k, 15);
-					k *= C2;
-					h ^= k;
-					h = Integer.rotateLeft(h, 13);
-					h = h * C3 + C4;
-					p += 4;
-					remain -= 4;
-				}
-				if (remain > 0) {
-					int k = 0;
-					for (int i = 0; remain > 0; i += 8) {
-						k ^= (bytes[p++] & 0xff) << i;
-						remain--;
-					}
-					k *= C1;
-					k = Integer.rotateLeft(k, 15);
-					k *= C2;
-					h ^= k;
-				}
-				h ^= length;
-				h ^= (h >>> 16);
-				h *= C5;
-				h ^= (h >>> 13);
-				h *= C6;
-				h ^= (h >>> 16);
-				return h;
-			}
-		};
-	}
-}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/RectangleBinaryHashFunctionFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/RectangleBinaryHashFunctionFactory.java
deleted file mode 100644
index 83bc50e..0000000
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/hash/RectangleBinaryHashFunctionFactory.java
+++ /dev/null
@@ -1,28 +0,0 @@
-package edu.uci.ics.asterix.dataflow.data.nontagged.hash;
-
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
-import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
-
-public class RectangleBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
-
-    private static final long serialVersionUID = 1L;
-
-    public static final RectangleBinaryHashFunctionFactory INSTANCE = new RectangleBinaryHashFunctionFactory();
-
-    private RectangleBinaryHashFunctionFactory() {
-    }
-
-    @Override
-    public IBinaryHashFunction createBinaryHashFunction() {
-        return new IBinaryHashFunction() {
-            @Override
-            public int hash(byte[] bytes, int offset, int length) {
-                long xBits = ADoubleSerializerDeserializer.getLongBits(bytes, offset);
-                long yBits = ADoubleSerializerDeserializer.getLongBits(bytes, offset + 8);
-                return (int) ((xBits ^ (xBits >>> 32)) ^ (yBits ^ (yBits >>> 32)));
-            }
-        };
-    }
-
-}
\ No newline at end of file
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADayTimeDurationPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADayTimeDurationPrinter.java
new file mode 100644
index 0000000..c596a8c
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADayTimeDurationPrinter.java
@@ -0,0 +1,77 @@
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.utils.WriteValueTools;
+
+public class ADayTimeDurationPrinter implements IPrinter {
+
+    public static final ADayTimeDurationPrinter INSTANCE = new ADayTimeDurationPrinter();
+    private static final GregorianCalendarSystem gCalInstance = GregorianCalendarSystem.getInstance();
+    
+    @Override
+    public void init() throws AlgebricksException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+        boolean positive = true;
+        long milliseconds = AInt64SerializerDeserializer.getLong(b, s + 1);
+
+        // set the negative flag. "||" is necessary in case that months field is not there (so it is 0)
+        if (milliseconds < 0) {
+            milliseconds *= -1;
+            positive = false;
+        }
+
+        int millisecond = gCalInstance.getDurationMillisecond(milliseconds);
+        int second = gCalInstance.getDurationSecond(milliseconds);
+        int minute = gCalInstance.getDurationMinute(milliseconds);
+        int hour = gCalInstance.getDurationHour(milliseconds);
+        int day = gCalInstance.getDurationDay(milliseconds);
+
+        ps.print("day-time-duration(\"");
+        if (!positive) {
+            ps.print("-");
+        }
+        try {
+            ps.print("P");
+            if (day != 0) {
+                WriteValueTools.writeInt(day, ps);
+                ps.print("D");
+            }
+            if (hour != 0 || minute != 0 || second != 0 || millisecond != 0) {
+                ps.print("T");
+            }
+            if (hour != 0) {
+                WriteValueTools.writeInt(hour, ps);
+                ps.print("H");
+            }
+            if (minute != 0) {
+                WriteValueTools.writeInt(minute, ps);
+                ps.print("M");
+            }
+            if (second != 0 || millisecond != 0) {
+                WriteValueTools.writeInt(second, ps);
+            }
+            if (millisecond > 0) {
+                ps.print(".");
+                WriteValueTools.writeInt(millisecond, ps);
+            }
+            if (second != 0 || millisecond != 0) {
+                ps.print("S");
+            }
+            ps.print("\")");
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADayTimeDurationPrinterFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADayTimeDurationPrinterFactory.java
new file mode 100644
index 0000000..4817916
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/ADayTimeDurationPrinterFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers;
+
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+
+public class ADayTimeDurationPrinterFactory implements IPrinterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final ADayTimeDurationPrinterFactory INSTANCE = new ADayTimeDurationPrinterFactory();
+
+    @Override
+    public IPrinter createPrinter() {
+        return ADayTimeDurationPrinter.INSTANCE;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java
index 478ad2c..41fd904 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AObjectPrinter.java
@@ -73,6 +73,14 @@
                 ADurationPrinter.INSTANCE.print(b, s, l, ps);
                 break;
             }
+            case YEARMONTHDURATION: {
+                AYearMonthDurationPrinter.INSTANCE.print(b, s, l, ps);
+                break;
+            }
+            case DAYTIMEDURATION: {
+                ADayTimeDurationPrinter.INSTANCE.print(b, s, l, ps);
+                break;
+            }
             case INTERVAL: {
                 AIntervalPrinter.INSTANCE.print(b, s, l, ps);
                 break;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AYearMonthDurationPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AYearMonthDurationPrinter.java
new file mode 100644
index 0000000..29e2529
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AYearMonthDurationPrinter.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.utils.WriteValueTools;
+
+public class AYearMonthDurationPrinter implements IPrinter {
+
+    public static final AYearMonthDurationPrinter INSTANCE = new AYearMonthDurationPrinter();
+    private static final GregorianCalendarSystem gCalInstance = GregorianCalendarSystem.getInstance();
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinter#init()
+     */
+    @Override
+    public void init() throws AlgebricksException {
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinter#print(byte[], int, int, java.io.PrintStream)
+     */
+    @Override
+    public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+        boolean positive = true;
+        int months = AInt32SerializerDeserializer.getInt(b, s + 1);
+
+        // set the negative flag. "||" is necessary in case that months field is not there (so it is 0)
+        if (months < 0) {
+            months *= -1;
+            positive = false;
+        }
+
+        int month = gCalInstance.getDurationMonth(months);
+        int year = gCalInstance.getDurationYear(months);
+
+        ps.print("year-month-duration(\"");
+        if (!positive) {
+            ps.print("-");
+        }
+        try {
+            ps.print("P");
+            if (year != 0) {
+                WriteValueTools.writeInt(year, ps);
+                ps.print("Y");
+            }
+            if (month != 0) {
+                WriteValueTools.writeInt(month, ps);
+                ps.print("M");
+            }
+            ps.print("\")");
+        } catch (IOException e) {
+            throw new AlgebricksException(e);
+        }
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AYearMonthDurationPrinterFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AYearMonthDurationPrinterFactory.java
new file mode 100644
index 0000000..7e6111d
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/AYearMonthDurationPrinterFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers;
+
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+
+public class AYearMonthDurationPrinterFactory implements IPrinterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final AYearMonthDurationPrinterFactory INSTANCE = new AYearMonthDurationPrinterFactory();
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinterFactory#createPrinter()
+     */
+    @Override
+    public IPrinter createPrinter() {
+        return AYearMonthDurationPrinter.INSTANCE;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/ADayTimeDurationPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/ADayTimeDurationPrinter.java
new file mode 100644
index 0000000..552e8d7
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/ADayTimeDurationPrinter.java
@@ -0,0 +1,28 @@
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers.json;
+
+import java.io.PrintStream;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt64SerializerDeserializer;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+
+public class ADayTimeDurationPrinter implements IPrinter {
+
+    public static final ADayTimeDurationPrinter INSTANCE = new ADayTimeDurationPrinter();
+
+    @Override
+    public void init() throws AlgebricksException {
+        // TODO Auto-generated method stub
+
+    }
+
+    @Override
+    public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+        long milliseconds = AInt64SerializerDeserializer.getLong(b, s + 1);
+
+        ps.print("{ day-time-duration: ");
+        ps.print(milliseconds);
+        ps.print("}");
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/ADayTimeDurationPrinterFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/ADayTimeDurationPrinterFactory.java
new file mode 100644
index 0000000..bbf0a70
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/ADayTimeDurationPrinterFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers.json;
+
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+
+public class ADayTimeDurationPrinterFactory implements IPrinterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final ADayTimeDurationPrinterFactory INSTANCE = new ADayTimeDurationPrinterFactory();
+
+    @Override
+    public IPrinter createPrinter() {
+        return ADayTimeDurationPrinter.INSTANCE;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AIntervalPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AIntervalPrinter.java
new file mode 100644
index 0000000..c6afb74
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AIntervalPrinter.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers.json;
+
+import java.io.PrintStream;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt8SerializerDeserializer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+
+public class AIntervalPrinter implements IPrinter {
+
+    public static final AIntervalPrinter INSTANCE = new AIntervalPrinter();
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinter#init()
+     */
+    @Override
+    public void init() throws AlgebricksException {
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinter#print(byte[], int, int, java.io.PrintStream)
+     */
+    @Override
+    public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+        ps.print("{ interval: { start: ");
+
+        short typetag = AInt8SerializerDeserializer.getByte(b, s + 1 + 8 * 2);
+
+        IPrinter timeInstancePrinter;
+
+        if (typetag == ATypeTag.DATE.serialize()) {
+            timeInstancePrinter = ADatePrinter.INSTANCE;
+            ((ADatePrinter) timeInstancePrinter).print(b, s + 4, 4, ps);
+            ps.print(", end: ");
+            ((ADatePrinter) timeInstancePrinter).print(b, s + 12, 4, ps);
+        } else if (typetag == ATypeTag.TIME.serialize()) {
+            timeInstancePrinter = ATimePrinter.INSTANCE;
+            ((ATimePrinter) timeInstancePrinter).print(b, s + 4, 4, ps);
+            ps.print(", end: ");
+            ((ATimePrinter) timeInstancePrinter).print(b, s + 12, 4, ps);
+        } else if (typetag == ATypeTag.DATETIME.serialize()) {
+            timeInstancePrinter = ADateTimePrinter.INSTANCE;
+            ((ADateTimePrinter) timeInstancePrinter).print(b, s, 8, ps);
+            ps.print(", end: ");
+            ((ADateTimePrinter) timeInstancePrinter).print(b, s + 8, 8, ps);
+        } else {
+            throw new AlgebricksException("Unsupport internal time types in interval: " + typetag);
+        }
+
+        ps.print("}}");
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AIntervalPrinterFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AIntervalPrinterFactory.java
new file mode 100644
index 0000000..7b3c3fe
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AIntervalPrinterFactory.java
@@ -0,0 +1,30 @@
+/*
+ * Copyright 2009-2011 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers.json;
+
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+
+public class AIntervalPrinterFactory implements IPrinterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final AIntervalPrinterFactory INSTANCE = new AIntervalPrinterFactory();
+
+    @Override
+    public IPrinter createPrinter() {
+        return AIntervalPrinter.INSTANCE;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AObjectPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AObjectPrinter.java
index 29c4ee1..3cf4ff2 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AObjectPrinter.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AObjectPrinter.java
@@ -73,6 +73,14 @@
                 ADurationPrinter.INSTANCE.print(b, s, l, ps);
                 break;
             }
+            case YEARMONTHDURATION: {
+                AYearMonthDurationPrinter.INSTANCE.print(b, s, l, ps);
+                break;
+            }
+            case DAYTIMEDURATION: {
+                ADayTimeDurationPrinter.INSTANCE.print(b, s, l, ps);
+                break;
+            }
             case POINT: {
                 APointPrinter.INSTANCE.print(b, s, l, ps);
                 break;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AYearMonthDurationPrinter.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AYearMonthDurationPrinter.java
new file mode 100644
index 0000000..163dc52
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AYearMonthDurationPrinter.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers.json;
+
+import java.io.PrintStream;
+
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+
+public class AYearMonthDurationPrinter implements IPrinter {
+
+    public static final AYearMonthDurationPrinter INSTANCE = new AYearMonthDurationPrinter();
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinter#init()
+     */
+    @Override
+    public void init() throws AlgebricksException {
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinter#print(byte[], int, int, java.io.PrintStream)
+     */
+    @Override
+    public void print(byte[] b, int s, int l, PrintStream ps) throws AlgebricksException {
+
+        int months = AInt32SerializerDeserializer.getInt(b, s + 1);
+
+        ps.print("{ year-month-duration: ");
+        ps.print(months);
+        ps.print("}");
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AYearMonthDurationPrinterFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AYearMonthDurationPrinterFactory.java
new file mode 100644
index 0000000..b32d010
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/printers/json/AYearMonthDurationPrinterFactory.java
@@ -0,0 +1,33 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.printers.json;
+
+import edu.uci.ics.hyracks.algebricks.data.IPrinter;
+import edu.uci.ics.hyracks.algebricks.data.IPrinterFactory;
+
+public class AYearMonthDurationPrinterFactory implements IPrinterFactory {
+
+    private static final long serialVersionUID = 1L;
+    public static final AYearMonthDurationPrinterFactory INSTANCE = new AYearMonthDurationPrinterFactory();
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.algebricks.data.IPrinterFactory#createPrinter()
+     */
+    @Override
+    public IPrinter createPrinter() {
+        return AYearMonthDurationPrinter.INSTANCE;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
new file mode 100644
index 0000000..b016af0
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADayTimeDurationSerializerDeserializer.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ADayTimeDuration;
+import edu.uci.ics.asterix.om.base.AMutableDayTimeDuration;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class ADayTimeDurationSerializerDeserializer implements ISerializerDeserializer<ADayTimeDuration> {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final ADayTimeDurationSerializerDeserializer INSTNACE = new ADayTimeDurationSerializerDeserializer();
+
+    @SuppressWarnings("unchecked")
+    private static final ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.ADAYTIMEDURATION);
+    private static final AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
+
+    @Override
+    public ADayTimeDuration deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return new ADayTimeDuration(in.readLong());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(ADayTimeDuration instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeLong(instance.getMilliseconds());
+        } catch (IOException e) {
+            throw new HyracksDataException();
+        }
+    }
+
+    public static void parse(String durationString, DataOutput out) throws HyracksDataException {
+        try {
+            ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aDayTimeDuration,
+                    ADurationParseOption.All);
+            dayTimeDurationSerde.serialize(aDayTimeDuration, out);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public static long getDayTime(byte[] data, int offset) {
+        return AInt64SerializerDeserializer.getLong(data, offset);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
index 6930f60..498a5b8 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/ADurationSerializerDeserializer.java
@@ -8,6 +8,7 @@
 import edu.uci.ics.asterix.om.base.ADuration;
 import edu.uci.ics.asterix.om.base.AMutableDuration;
 import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -46,7 +47,7 @@
 
     public static void parse(String duration, DataOutput out) throws HyracksDataException {
         try {
-            ADurationParserFactory.parseDuration(duration, 0, duration.length(), aDuration);
+            ADurationParserFactory.parseDuration(duration, 0, duration.length(), aDuration, ADurationParseOption.All);
             durationSerde.serialize(aDuration, out);
         } catch (Exception e) {
             throw new HyracksDataException(e);
@@ -61,7 +62,7 @@
      * @return
      */
     public static int getYearMonth(byte[] data, int offset) {
-        return AInt32SerializerDeserializer.getInt(data, offset);
+        return AInt32SerializerDeserializer.getInt(data, offset + getYearMonthOffset());
     }
 
     /**
@@ -72,6 +73,14 @@
      * @return
      */
     public static long getDayTime(byte[] data, int offset) {
-        return AInt64SerializerDeserializer.getLong(data, offset + 4);
+        return AInt64SerializerDeserializer.getLong(data, offset + getDayTimeOffset());
+    }
+
+    public static int getYearMonthOffset() {
+        return 0;
+    }
+
+    public static int getDayTimeOffset() {
+        return 4;
     }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
index 8c4def0..318d10e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AIntervalSerializerDeserializer.java
@@ -66,11 +66,19 @@
     }
 
     public static long getIntervalStart(byte[] data, int offset) {
-        return AInt64SerializerDeserializer.getLong(data, offset);
+        return AInt64SerializerDeserializer.getLong(data, offset + getIntervalStartOffset());
     }
 
     public static long getIntervalEnd(byte[] data, int offset) {
-        return AInt64SerializerDeserializer.getLong(data, offset + 8);
+        return AInt64SerializerDeserializer.getLong(data, offset + getIntervalEndOffset());
+    }
+
+    public static int getIntervalStartOffset() {
+        return 0;
+    }
+
+    public static int getIntervalEndOffset() {
+        return 8;
     }
 
     public static byte getIntervalTimeType(byte[] data, int offset) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
index 2e4f72c..88aa7f4 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AObjectSerializerDeserializer.java
@@ -86,6 +86,12 @@
             case DURATION: {
                 return ADurationSerializerDeserializer.INSTANCE.deserialize(in);
             }
+            case YEARMONTHDURATION: {
+                return AYearMonthDurationerializerDeserializer.INSTANCE.deserialize(in);
+            }
+            case DAYTIMEDURATION: {
+                return ADayTimeDurationSerializerDeserializer.INSTNACE.deserialize(in);
+            }
             case INTERVAL: {
                 return AIntervalSerializerDeserializer.INSTANCE.deserialize(in);
             }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationerializerDeserializer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationerializerDeserializer.java
new file mode 100644
index 0000000..79bf3c0
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/serde/AYearMonthDurationerializerDeserializer.java
@@ -0,0 +1,73 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.dataflow.data.nontagged.serde;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableYearMonthDuration;
+import edu.uci.ics.asterix.om.base.AYearMonthDuration;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
+
+public class AYearMonthDurationerializerDeserializer implements ISerializerDeserializer<AYearMonthDuration> {
+
+    private static final long serialVersionUID = 1L;
+
+    public static final AYearMonthDurationerializerDeserializer INSTANCE = new AYearMonthDurationerializerDeserializer();
+
+    @SuppressWarnings("unchecked")
+    private static final ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
+            .getSerializerDeserializer(BuiltinType.AYEARMONTHDURATION);
+    private static final AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
+
+    @Override
+    public AYearMonthDuration deserialize(DataInput in) throws HyracksDataException {
+        try {
+            return new AYearMonthDuration(in.readInt());
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    @Override
+    public void serialize(AYearMonthDuration instance, DataOutput out) throws HyracksDataException {
+        try {
+            out.writeInt(instance.getMonths());
+        } catch (IOException e) {
+            throw new HyracksDataException();
+        }
+    }
+
+    public static void parse(String durationString, DataOutput out) throws HyracksDataException {
+        try {
+            ADurationParserFactory.parseDuration(durationString, 0, durationString.length(), aYearMonthDuration,
+                    ADurationParseOption.All);
+            yearMonthDurationSerde.serialize(aYearMonthDuration, out);
+        } catch (Exception e) {
+            throw new HyracksDataException(e);
+        }
+    }
+
+    public static int getYearMonth(byte[] data, int offset) {
+        return AInt32SerializerDeserializer.getInt(data, offset);
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java
index 97a0c13..4d85537 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryBooleanInspectorImpl.java
@@ -1,6 +1,7 @@
 package edu.uci.ics.asterix.formats.nontagged;
 
 import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspector;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryBooleanInspectorFactory;
 import edu.uci.ics.hyracks.api.context.IHyracksTaskContext;
@@ -24,6 +25,11 @@
     public boolean getBooleanValue(byte[] bytes, int offset, int length) {
         if (bytes[offset] == SER_NULL_TYPE_TAG)
             return false;
+        /** check if the runtime type is boolean */
+        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
+        if (typeTag != ATypeTag.BOOLEAN) {
+            throw new IllegalStateException("Runtime error: the select condition should be of the boolean type!");
+        }
         return bytes[offset + 1] == 1;
     }
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
index 02f0e47..fb9d195 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryComparatorFactoryProvider.java
@@ -2,15 +2,20 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ADateOrTimeAscBinaryComparatorFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ADateTimeAscBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ACirclePartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ADurationPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AIntervalPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ALinePartialBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AObjectDescBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.APoint3DPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.APointPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.APolygonPartialBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.BooleanBinaryComparatorFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.RectangleBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ARectanglePartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.RawBinaryComparatorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparator;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryComparatorFactory;
@@ -102,9 +107,14 @@
             case INT16: {
                 return addOffset(SHORT_POINTABLE_INSTANCE, ascending);
             }
+            case DATE:
+            case TIME:
+            case YEARMONTHDURATION:
             case INT32: {
                 return addOffset(INTEGER_POINTABLE_INSTANCE, ascending);
             }
+            case DATETIME:
+            case DAYTIMEDURATION:
             case INT64: {
                 return addOffset(LONG_POINTABLE_INSTANCE, ascending);
             }
@@ -118,17 +128,31 @@
                 return addOffset(UTF8STRING_POINTABLE_INSTANCE, ascending);
             }
             case RECTANGLE: {
-                return addOffset(RectangleBinaryComparatorFactory.INSTANCE, ascending);
+                return addOffset(ARectanglePartialBinaryComparatorFactory.INSTANCE, ascending);
             }
-            case DATE:
-            case TIME: {
-                return addOffset(ADateOrTimeAscBinaryComparatorFactory.INSTANCE, ascending);
+            case CIRCLE: {
+                return addOffset(ACirclePartialBinaryComparatorFactory.INSTANCE, ascending);
             }
-            case DATETIME: {
-                return addOffset(ADateTimeAscBinaryComparatorFactory.INSTANCE, ascending);
+            case POINT: {
+                return addOffset(APointPartialBinaryComparatorFactory.INSTANCE, ascending);
+            }
+            case POINT3D: {
+                return addOffset(APoint3DPartialBinaryComparatorFactory.INSTANCE, ascending);
+            }
+            case LINE: {
+                return addOffset(ALinePartialBinaryComparatorFactory.INSTANCE, ascending);
+            }
+            case POLYGON: {
+                return addOffset(APolygonPartialBinaryComparatorFactory.INSTANCE, ascending);
+            }
+            case DURATION: {
+                return addOffset(ADurationPartialBinaryComparatorFactory.INSTANCE, ascending);
+            }
+            case INTERVAL: {
+                return addOffset(AIntervalPartialBinaryComparatorFactory.INSTANCE, ascending);
             }
             default: {
-                throw new NotImplementedException("No binary comparator factory implemented for type " + type + " .");
+                return addOffset(RawBinaryComparatorFactory.INSTANCE, ascending);
             }
         }
     }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
index 1afdbf8..7bfdb050 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
@@ -6,12 +6,11 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.hash.BooleanBinaryHashFunctionFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.hash.DoubleBinaryHashFunctionFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.hash.LongBinaryHashFunctionFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.hash.RectangleBinaryHashFunctionFactory;
 import edu.uci.ics.asterix.om.types.IAType;
-import edu.uci.ics.hyracks.algebricks.common.exceptions.NotImplementedException;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFactoryProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunction;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
+import edu.uci.ics.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory;
 import edu.uci.ics.hyracks.data.std.primitive.DoublePointable;
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
@@ -69,9 +68,14 @@
             case BOOLEAN: {
                 return addOffset(BooleanBinaryHashFunctionFactory.INSTANCE);
             }
+            case DATE:
+            case TIME:
+            case YEARMONTHDURATION:
             case INT32: {
                 return addOffset(new PointableBinaryHashFunctionFactory(IntegerPointable.FACTORY));
             }
+            case DAYTIMEDURATION:
+            case DATETIME:
             case INT64: {
                 return addOffset(LongBinaryHashFunctionFactory.INSTANCE);
             }
@@ -84,12 +88,8 @@
             case STRING: {
                 return addOffset(new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY));
             }
-            case RECTANGLE: {
-                return addOffset(RectangleBinaryHashFunctionFactory.INSTANCE);
-            }
             default: {
-                throw new NotImplementedException("No binary hash function factory implemented for type "
-                        + aqlType.getTypeTag() + " .");
+                return addOffsetForGenericBinaryHash();
             }
         }
     }
@@ -106,11 +106,23 @@
 
                     @Override
                     public int hash(byte[] bytes, int offset, int length) {
-                        return bhf.hash(bytes, offset + 1, length);
+                        return bhf.hash(bytes, offset + 1, length - 1);
                     }
                 };
             }
         };
     }
 
+    private IBinaryHashFunctionFactory addOffsetForGenericBinaryHash() {
+        return new IBinaryHashFunctionFactory() {
+
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IBinaryHashFunction createBinaryHashFunction() {
+                return MurmurHash3BinaryHashFunctionFamily.INSTANCE.createBinaryHashFunction(0);
+            }
+        };
+    }
+
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFamilyProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFamilyProvider.java
index bc7ba26..61ac6fa 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFamilyProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFamilyProvider.java
@@ -17,8 +17,8 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.asterix.dataflow.data.nontagged.hash.MurmurHash3BinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.data.std.accessors.MurmurHash3BinaryHashFunctionFamily;
 import edu.uci.ics.hyracks.algebricks.data.IBinaryHashFunctionFamilyProvider;
 import edu.uci.ics.hyracks.api.dataflow.value.IBinaryHashFunctionFamily;
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java
index b8c6ab0..842a6e8 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlJSONPrinterFactoryProvider.java
@@ -14,11 +14,11 @@
  */
 package edu.uci.ics.asterix.formats.nontagged;
 
-
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ABooleanPrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ACirclePrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADatePrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADateTimePrinterFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADayTimeDurationPrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADoublePrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADurationPrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AFloatPrinterFactory;
@@ -40,6 +40,8 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ATimePrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AUnionPrinterFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AUnorderedlistPrinterFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AYearMonthDurationPrinter;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AYearMonthDurationPrinterFactory;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.AUnionType;
@@ -63,8 +65,8 @@
 
         if (aqlType != null) {
             switch (aqlType.getTypeTag()) {
-                // case ANYTYPE:
-                // return AAnyTypePrinterFactory.INSTANCE;
+            // case ANYTYPE:
+            // return AAnyTypePrinterFactory.INSTANCE;
                 case INT8:
                     return AInt8PrinterFactory.INSTANCE;
                 case INT16:
@@ -89,6 +91,10 @@
                     return ADateTimePrinterFactory.INSTANCE;
                 case DURATION:
                     return ADurationPrinterFactory.INSTANCE;
+                case YEARMONTHDURATION:
+                    return AYearMonthDurationPrinterFactory.INSTANCE;
+                case DAYTIMEDURATION:
+                    return ADayTimeDurationPrinterFactory.INSTANCE;
                 case POINT:
                     return APointPrinterFactory.INSTANCE;
                 case POINT3D:
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlNormalizedKeyComputerFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlNormalizedKeyComputerFactoryProvider.java
index 23306ed..b4a1fd8 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlNormalizedKeyComputerFactoryProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlNormalizedKeyComputerFactoryProvider.java
@@ -23,9 +23,14 @@
         IAType aqlType = (IAType) type;
         if (ascending) {
             switch (aqlType.getTypeTag()) {
+                case DATE:
+                case TIME:
+                case YEARMONTHDURATION:
                 case INT32: {
                     return new AWrappedAscNormalizedKeyComputerFactory(new IntegerNormalizedKeyComputerFactory());
                 }
+                case DATETIME:
+                case DAYTIMEDURATION:
                 case INT64: {
                     return new AWrappedAscNormalizedKeyComputerFactory(new Integer64NormalizedKeyComputerFactory());
                 }
@@ -44,9 +49,14 @@
             }
         } else {
             switch (aqlType.getTypeTag()) {
+                case DATE:
+                case TIME:
+                case YEARMONTHDURATION:
                 case INT32: {
                     return new AWrappedDescNormalizedKeyComputerFactory(new IntegerNormalizedKeyComputerFactory());
                 }
+                case DATETIME:
+                case DAYTIMEDURATION:
                 case INT64: {
                     return new AWrappedDescNormalizedKeyComputerFactory(new Integer64NormalizedKeyComputerFactory());
                 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
index 29e33fd..92cfc5b 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlSerializerDeserializerProvider.java
@@ -9,6 +9,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
@@ -29,6 +30,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AStringSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.SerializerDeserializerUtil;
 import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
@@ -112,6 +114,12 @@
             case DURATION: {
                 return ADurationSerializerDeserializer.INSTANCE;
             }
+            case YEARMONTHDURATION: {
+                return AYearMonthDurationerializerDeserializer.INSTANCE;
+            }
+            case DAYTIMEDURATION: {
+                return ADayTimeDurationSerializerDeserializer.INSTNACE;
+            }
             case INTERVAL: {
                 return AIntervalSerializerDeserializer.INSTANCE;
             }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADayTimeDuration.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADayTimeDuration.java
new file mode 100644
index 0000000..03a9627
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/ADayTimeDuration.java
@@ -0,0 +1,94 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.visitors.IOMVisitor;
+
+public class ADayTimeDuration implements IAObject {
+
+    protected long chrononInMillisecond;
+
+    public ADayTimeDuration(long millisecond) {
+        this.chrononInMillisecond = millisecond;
+    }
+
+    public long getMilliseconds() {
+        return chrononInMillisecond;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.JSONSerializable#toJSON()
+     */
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject json = new JSONObject();
+
+        JSONObject duration = new JSONObject();
+        duration.put("milliseconds", chrononInMillisecond);
+        json.put("ADuration", duration);
+
+        return json;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#getType()
+     */
+    @Override
+    public IAType getType() {
+        return BuiltinType.ADAYTIMEDURATION;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#accept(edu.uci.ics.asterix.om.visitors.IOMVisitor)
+     */
+    @Override
+    public void accept(IOMVisitor visitor) throws AsterixException {
+        visitor.visitADayTimeDuration(this);
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#deepEqual(edu.uci.ics.asterix.om.base.IAObject)
+     */
+    @Override
+    public boolean deepEqual(IAObject obj) {
+        return equals(obj);
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#hash()
+     */
+    @Override
+    public int hash() {
+        return hashCode();
+    }
+
+    public boolean equals(Object o) {
+        if (o instanceof ADayTimeDuration) {
+            return ((ADayTimeDuration) o).chrononInMillisecond == chrononInMillisecond;
+        }
+        return false;
+    }
+
+    public int hashCode() {
+        return (int) (chrononInMillisecond ^ (chrononInMillisecond >>> 32));
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableDayTimeDuration.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableDayTimeDuration.java
new file mode 100644
index 0000000..ebd0a18
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableDayTimeDuration.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base;
+
+public class AMutableDayTimeDuration extends ADayTimeDuration {
+
+    public AMutableDayTimeDuration(long milliseconds) {
+        super(milliseconds);
+    }
+
+    public void setMilliseconds(long milliseconds) {
+        this.chrononInMillisecond = milliseconds;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableYearMonthDuration.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableYearMonthDuration.java
new file mode 100644
index 0000000..3a21144
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AMutableYearMonthDuration.java
@@ -0,0 +1,27 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base;
+
+public class AMutableYearMonthDuration extends AYearMonthDuration {
+
+    public AMutableYearMonthDuration(int months) {
+        super(months);
+    }
+
+    public void setMonths(int months) {
+        this.chrononInMonth = months;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AYearMonthDuration.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AYearMonthDuration.java
new file mode 100644
index 0000000..1714bb0
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/AYearMonthDuration.java
@@ -0,0 +1,102 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.base;
+
+import org.json.JSONException;
+import org.json.JSONObject;
+
+import edu.uci.ics.asterix.common.exceptions.AsterixException;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.om.visitors.IOMVisitor;
+
+/**
+ * AYearMonthDuration represents the duration in the unit of months.
+ * <p/>
+ * An AYearMonthDuration may contain the following two fields:<br/>
+ * - year;<br/>
+ * - month.
+ * <p/>
+ */
+public class AYearMonthDuration implements IAObject {
+
+    protected int chrononInMonth;
+
+    public AYearMonthDuration(int months) {
+        this.chrononInMonth = months;
+    }
+
+    public int getMonths() {
+        return chrononInMonth;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.hyracks.api.dataflow.value.JSONSerializable#toJSON()
+     */
+    @Override
+    public JSONObject toJSON() throws JSONException {
+        JSONObject json = new JSONObject();
+
+        JSONObject duration = new JSONObject();
+        duration.put("months", chrononInMonth);
+        json.put("ADuration", duration);
+
+        return json;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#getType()
+     */
+    @Override
+    public IAType getType() {
+        return BuiltinType.AYEARMONTHDURATION;
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#accept(edu.uci.ics.asterix.om.visitors.IOMVisitor)
+     */
+    @Override
+    public void accept(IOMVisitor visitor) throws AsterixException {
+        visitor.visitAYearMonthDuration(this);
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#deepEqual(edu.uci.ics.asterix.om.base.IAObject)
+     */
+    @Override
+    public boolean deepEqual(IAObject obj) {
+        return equals(obj);
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.base.IAObject#hash()
+     */
+    @Override
+    public int hash() {
+        return hashCode();
+    }
+
+    public boolean equals(Object o) {
+        if (o instanceof AYearMonthDuration) {
+            return ((AYearMonthDuration) o).chrononInMonth == chrononInMonth;
+        }
+        return false;
+    }
+
+    public int hashCode() {
+        return chrononInMonth;
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java
index de5a6a1..84e2386 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/base/temporal/ADurationParserFactory.java
@@ -17,7 +17,10 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
+import edu.uci.ics.asterix.om.base.AMutableDayTimeDuration;
 import edu.uci.ics.asterix.om.base.AMutableDuration;
+import edu.uci.ics.asterix.om.base.AMutableYearMonthDuration;
+import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParser;
 import edu.uci.ics.hyracks.dataflow.common.data.parsers.IValueParserFactory;
@@ -28,7 +31,11 @@
 
     private static final long serialVersionUID = 1L;
 
-    private static final String durationErrorMessage = "Wrong Input Format for a Duration Value";
+    private static final String durationErrorMessage = "Wrong Input Format for a duration/year-month-duration/day-time-duration Value";
+    private static final String onlyYearMonthErrorMessage = "Only year-month fields are allowed";
+    private static final String onlyDayTimeErrorMessage = "Only day-time fields are allowed";
+
+    private static final int DECIMAL_UNIT = 10;
 
     private ADurationParserFactory() {
 
@@ -41,7 +48,7 @@
 
             @Override
             public void parse(char[] buffer, int start, int length, DataOutput out) throws HyracksDataException {
-                parseDuration(buffer, start, length, aMutableDuration);
+                parseDuration(buffer, start, length, aMutableDuration, ADurationParseOption.All);
                 try {
                     out.writeInt(aMutableDuration.getMonths());
                     out.writeLong(aMutableDuration.getMilliseconds());
@@ -64,299 +71,75 @@
         SEC;
     };
 
-    public static void parseDuration(String durationString, int start, int length, AMutableDuration aDuration)
-            throws HyracksDataException {
-
-        boolean positive = true;
-        int offset = 0;
-        int value = 0, hour = 0, minute = 0, second = 0, millisecond = 0, year = 0, month = 0, day = 0;
-        State state = State.NOTHING_READ;
-
-        if (durationString.charAt(start + offset) == '-') {
-            offset++;
-            positive = false;
-        }
-
-        if (durationString.charAt(start + offset) != 'P') {
-            throw new HyracksDataException(durationErrorMessage + ": Missing leading 'P'.");
-        }
-
-        offset++;
-
-        for (; offset < length; offset++) {
-            if (durationString.charAt(start + offset) >= '0' && durationString.charAt(start + offset) <= '9') {
-                // accumulate the digit fields
-                value = value * 10 + durationString.charAt(start + offset) - '0';
-            } else {
-                switch (durationString.charAt(start + offset)) {
-                    case 'Y':
-                        if (state.compareTo(State.YEAR) < 0) {
-                            year = value;
-                            state = State.YEAR;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong YEAR feild.");
-                        }
-                        break;
-                    case 'M':
-                        if (state.compareTo(State.TIME) < 0) {
-                            if (state.compareTo(State.MONTH) < 0) {
-                                month = value;
-                                state = State.MONTH;
-                            } else {
-                                throw new HyracksDataException(durationErrorMessage + ": wrong MONTH field.");
-                            }
-                        } else if (state.compareTo(State.MIN) < 0) {
-                            minute = value;
-                            state = State.MIN;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong MIN field.");
-                        }
-                        break;
-                    case 'D':
-                        if (state.compareTo(State.DAY) < 0) {
-                            day = value;
-                            state = State.DAY;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong DAY field");
-                        }
-                        break;
-                    case 'T':
-                        if (state.compareTo(State.TIME) < 0) {
-                            state = State.TIME;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong TIME field.");
-                        }
-                        break;
-
-                    case 'H':
-                        if (state.compareTo(State.HOUR) < 0) {
-                            hour = value;
-                            state = State.HOUR;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong HOUR field.");
-                        }
-                        break;
-                    case '.':
-                        if (state.compareTo(State.MILLISEC) < 0) {
-                            int i = 1;
-                            for (; offset + i < length; i++) {
-                                if (durationString.charAt(start + offset + i) >= '0'
-                                        && durationString.charAt(start + offset + i) <= '9') {
-                                    if (i < 4) {
-                                        millisecond = millisecond * 10
-                                                + (durationString.charAt(start + offset + i) - '0');
-                                    } else {
-                                        throw new HyracksDataException(durationErrorMessage
-                                                + ": wrong MILLISECOND field.");
-                                    }
-                                } else {
-                                    break;
-                                }
-                            }
-                            offset += i;
-                            state = State.MILLISEC;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong MILLISECOND field.");
-                        }
-                    case 'S':
-                        if (state.compareTo(State.SEC) < 0) {
-                            second = value;
-                            state = State.SEC;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong SECOND field.");
-                        }
-                        break;
-                    default:
-                        throw new HyracksDataException(durationErrorMessage + ": wrong format for duration.");
-
-                }
-                value = 0;
-            }
-        }
-
-        if (state.compareTo(State.TIME) == 0) {
-            throw new HyracksDataException(durationErrorMessage + ": no time fields after time separator.");
-        }
-
-        short temp = 1;
-        if (!positive) {
-            temp = -1;
-        }
-
-        aDuration.setValue(temp * (year * 12 + month), temp
-                * (day * 24 * 3600 * 1000L + 3600 * 1000L * hour + 60 * minute * 1000L + second * 1000L + millisecond));
-
+    public enum ADurationParseOption {
+        YEAR_MONTH,
+        DAY_TIME,
+        All
     }
 
-    /**
-     * Copy-and-paste the code in {@link #parseDuration(String, int, int, AMutableDuration)} in order to eliminate
-     * object creation.
-     * 
-     * @param charAccessor
-     * @param start
-     * @param length
-     * @param aDuration
-     * @throws HyracksDataException
-     */
-    public static void parseDuration(char[] charAccessor, int start, int length, AMutableDuration aDuration)
-            throws HyracksDataException {
-
-        boolean positive = true;
-        int offset = 0;
-        int value = 0, hour = 0, minute = 0, second = 0, millisecond = 0, year = 0, month = 0, day = 0;
-        State state = State.NOTHING_READ;
-
-        if (charAccessor[start + offset] == '-') {
-            offset++;
-            positive = false;
-        }
-
-        if (charAccessor[start + offset] != 'P') {
-            throw new HyracksDataException(durationErrorMessage + ": Missing leading 'P'.");
-        }
-
-        offset++;
-
-        for (; offset < length; offset++) {
-            if (charAccessor[start + offset] >= '0' && charAccessor[start + offset] <= '9') {
-                // accumulate the digit fields
-                value = value * 10 + charAccessor[start + offset] - '0';
-            } else {
-                switch (charAccessor[start + offset]) {
-                    case 'Y':
-                        if (state.compareTo(State.YEAR) < 0) {
-                            year = value;
-                            state = State.YEAR;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong YEAR feild.");
-                        }
-                        break;
-                    case 'M':
-                        if (state.compareTo(State.TIME) < 0) {
-                            if (state.compareTo(State.MONTH) < 0) {
-                                month = value;
-                                state = State.MONTH;
-                            } else {
-                                throw new HyracksDataException(durationErrorMessage + ": wrong MONTH field.");
-                            }
-                        } else if (state.compareTo(State.MIN) < 0) {
-                            minute = value;
-                            state = State.MIN;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong MIN field.");
-                        }
-                        break;
-                    case 'D':
-                        if (state.compareTo(State.DAY) < 0) {
-                            day = value;
-                            state = State.DAY;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong DAY field");
-                        }
-                        break;
-                    case 'T':
-                        if (state.compareTo(State.TIME) < 0) {
-                            state = State.TIME;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong TIME field.");
-                        }
-                        break;
-
-                    case 'H':
-                        if (state.compareTo(State.HOUR) < 0) {
-                            hour = value;
-                            state = State.HOUR;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong HOUR field.");
-                        }
-                        break;
-                    case '.':
-                        if (state.compareTo(State.MILLISEC) < 0) {
-                            int i = 1;
-                            for (; offset + i < length; i++) {
-                                if (charAccessor[start + offset + i] >= '0' && charAccessor[start + offset + i] <= '9') {
-                                    if (i < 4) {
-                                        millisecond = millisecond * 10 + (charAccessor[start + offset + i] - '0');
-                                    } else {
-                                        throw new HyracksDataException(durationErrorMessage
-                                                + ": wrong MILLISECOND field.");
-                                    }
-                                } else {
-                                    break;
-                                }
-                            }
-                            offset += i;
-                            state = State.MILLISEC;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong MILLISECOND field.");
-                        }
-                    case 'S':
-                        if (state.compareTo(State.SEC) < 0) {
-                            second = value;
-                            state = State.SEC;
-                        } else {
-                            throw new HyracksDataException(durationErrorMessage + ": wrong SECOND field.");
-                        }
-                        break;
-                    default:
-                        throw new HyracksDataException(durationErrorMessage + ": wrong format for duration.");
-
-                }
-                value = 0;
-            }
-        }
-
-        if (state.compareTo(State.TIME) == 0) {
-            throw new HyracksDataException(durationErrorMessage + ": no time fields after time separator.");
-        }
-
-        short temp = 1;
-        if (!positive) {
-            temp = -1;
-        }
-
-        aDuration.setValue(temp * (year * 12 + month), temp
-                * (day * 24 * 3600 * 1000L + 3600 * 1000L * hour + 60 * minute * 1000L + second * 1000L + millisecond));
-
+    interface IStringAccessor {
+        char getCharAt(int index);
     }
 
-    /**
-     * Copy-and-paste the code in {@link #parseDuration(String, int, int, AMutableDuration)} in order to eliminate
-     * object creation.
-     * 
-     * @param charAccessor
-     * @param start
-     * @param length
-     * @param aDuration
-     * @throws HyracksDataException
-     */
-    public static void parseDuration(byte[] charAccessor, int start, int length, AMutableDuration aDuration)
-            throws HyracksDataException {
+    public static void parseDuration(final Object durationString, final int start, int length, IAObject mutableObject,
+            ADurationParseOption parseOption) throws HyracksDataException {
 
-        boolean positive = true;
         int offset = 0;
         int value = 0, hour = 0, minute = 0, second = 0, millisecond = 0, year = 0, month = 0, day = 0;
         State state = State.NOTHING_READ;
 
-        if (charAccessor[start + offset] == '-') {
-            offset++;
-            positive = false;
+        IStringAccessor charAccessor;
+
+        if (durationString instanceof char[]) {
+            charAccessor = new IStringAccessor() {
+                @Override
+                public char getCharAt(int index) {
+                    return ((char[]) durationString)[start + index];
+                }
+            };
+        } else if (durationString instanceof byte[]) {
+            charAccessor = new IStringAccessor() {
+
+                @Override
+                public char getCharAt(int index) {
+                    return (char) (((byte[]) durationString)[start + index]);
+                }
+            };
+        } else if (durationString instanceof String) {
+            charAccessor = new IStringAccessor() {
+
+                @Override
+                public char getCharAt(int index) {
+                    return ((String) durationString).charAt(start + index);
+                }
+            };
+        } else {
+            throw new HyracksDataException(durationErrorMessage);
         }
 
-        if (charAccessor[start + offset] != 'P') {
+        short sign = 1;
+        if (charAccessor.getCharAt(offset) == '-') {
+            offset++;
+            sign = -1;
+        }
+
+        if (charAccessor.getCharAt(offset) != 'P') {
             throw new HyracksDataException(durationErrorMessage + ": Missing leading 'P'.");
         }
 
         offset++;
 
         for (; offset < length; offset++) {
-            if (charAccessor[start + offset] >= '0' && charAccessor[start + offset] <= '9') {
+            if (charAccessor.getCharAt(offset) >= '0' && charAccessor.getCharAt(offset) <= '9') {
                 // accumulate the digit fields
-                value = value * 10 + charAccessor[start + offset] - '0';
+                value = value * DECIMAL_UNIT + charAccessor.getCharAt(offset) - '0';
             } else {
-                switch (charAccessor[start + offset]) {
+                switch (charAccessor.getCharAt(offset)) {
                     case 'Y':
                         if (state.compareTo(State.YEAR) < 0) {
+                            if (parseOption == ADurationParseOption.DAY_TIME) {
+                                throw new HyracksDataException(onlyDayTimeErrorMessage);
+                            }
                             year = value;
                             state = State.YEAR;
                         } else {
@@ -366,12 +149,18 @@
                     case 'M':
                         if (state.compareTo(State.TIME) < 0) {
                             if (state.compareTo(State.MONTH) < 0) {
+                                if (parseOption == ADurationParseOption.DAY_TIME) {
+                                    throw new HyracksDataException(onlyDayTimeErrorMessage);
+                                }
                                 month = value;
                                 state = State.MONTH;
                             } else {
                                 throw new HyracksDataException(durationErrorMessage + ": wrong MONTH field.");
                             }
                         } else if (state.compareTo(State.MIN) < 0) {
+                            if (parseOption == ADurationParseOption.YEAR_MONTH) {
+                                throw new HyracksDataException(onlyYearMonthErrorMessage);
+                            }
                             minute = value;
                             state = State.MIN;
                         } else {
@@ -380,6 +169,9 @@
                         break;
                     case 'D':
                         if (state.compareTo(State.DAY) < 0) {
+                            if (parseOption == ADurationParseOption.YEAR_MONTH) {
+                                throw new HyracksDataException(onlyYearMonthErrorMessage);
+                            }
                             day = value;
                             state = State.DAY;
                         } else {
@@ -388,6 +180,9 @@
                         break;
                     case 'T':
                         if (state.compareTo(State.TIME) < 0) {
+                            if (parseOption == ADurationParseOption.YEAR_MONTH) {
+                                throw new HyracksDataException(onlyYearMonthErrorMessage);
+                            }
                             state = State.TIME;
                         } else {
                             throw new HyracksDataException(durationErrorMessage + ": wrong TIME field.");
@@ -396,6 +191,9 @@
 
                     case 'H':
                         if (state.compareTo(State.HOUR) < 0) {
+                            if (parseOption == ADurationParseOption.YEAR_MONTH) {
+                                throw new HyracksDataException(onlyYearMonthErrorMessage);
+                            }
                             hour = value;
                             state = State.HOUR;
                         } else {
@@ -404,11 +202,16 @@
                         break;
                     case '.':
                         if (state.compareTo(State.MILLISEC) < 0) {
+                            if (parseOption == ADurationParseOption.YEAR_MONTH) {
+                                throw new HyracksDataException(onlyYearMonthErrorMessage);
+                            }
                             int i = 1;
                             for (; offset + i < length; i++) {
-                                if (charAccessor[start + offset + i] >= '0' && charAccessor[start + offset + i] <= '9') {
+                                if (charAccessor.getCharAt(offset + i) >= '0'
+                                        && charAccessor.getCharAt(offset + i) <= '9') {
                                     if (i < 4) {
-                                        millisecond = millisecond * 10 + (charAccessor[start + offset + i] - '0');
+                                        millisecond = millisecond * DECIMAL_UNIT
+                                                + (charAccessor.getCharAt(offset + i) - '0');
                                     } else {
                                         throw new HyracksDataException(durationErrorMessage
                                                 + ": wrong MILLISECOND field.");
@@ -424,6 +227,9 @@
                         }
                     case 'S':
                         if (state.compareTo(State.SEC) < 0) {
+                            if (parseOption == ADurationParseOption.YEAR_MONTH) {
+                                throw new HyracksDataException(onlyYearMonthErrorMessage);
+                            }
                             second = value;
                             state = State.SEC;
                         } else {
@@ -442,13 +248,18 @@
             throw new HyracksDataException(durationErrorMessage + ": no time fields after time separator.");
         }
 
-        short temp = 1;
-        if (!positive) {
-            temp = -1;
+        if (mutableObject instanceof AMutableDuration) {
+            ((AMutableDuration) mutableObject)
+                    .setValue(sign * (year * 12 + month),
+                            sign
+                                    * (day * 24 * 3600 * 1000L + 3600 * 1000L * hour + 60 * minute * 1000L + second
+                                            * 1000L + millisecond));
+        } else if (mutableObject instanceof AMutableYearMonthDuration) {
+            ((AMutableYearMonthDuration) mutableObject).setMonths(sign * (year * 12 + month));
+        } else if (mutableObject instanceof AMutableDayTimeDuration) {
+            ((AMutableDayTimeDuration) mutableObject)
+                    .setMilliseconds(sign
+                            * (day * 24 * 3600 * 1000L + 3600 * 1000L * hour + 60 * minute * 1000L + second * 1000L + millisecond));
         }
-
-        aDuration.setValue(temp * (year * 12 + month), temp
-                * (day * 24 * 3600 * 1000L + 3600 * 1000L * hour + 60 * minute * 1000L + second * 1000L + millisecond));
-
     }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
index 3cad3be..eb882b8 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/functions/AsterixBuiltinFunctions.java
@@ -8,7 +8,6 @@
 
 import org.apache.commons.lang3.mutable.Mutable;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ABooleanTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ACircleTypeComputer;
@@ -27,8 +26,10 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.BinaryBooleanOrNullFunctionTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringBoolOrNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringStringOrNullTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.CastListResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.CastRecordResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ClosedRecordConstructorResultType;
+import edu.uci.ics.asterix.om.typecomputer.impl.ConcatNonNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
 import edu.uci.ics.asterix.om.typecomputer.impl.InjectFailureTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.NonTaggedCollectionMemberResultType;
@@ -47,6 +48,7 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalACircleTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalADateTimeTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalADateTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.OptionalADayTimeDurationTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalADoubleTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalADurationTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalAFloatTypeComputer;
@@ -63,6 +65,7 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalAStringTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalATemporalInstanceTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OptionalATimeTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.OptionalAYearMonthDurationTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OrderedListConstructorResultType;
 import edu.uci.ics.asterix.om.typecomputer.impl.OrderedListOfAInt32TypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.OrderedListOfAPointTypeComputer;
@@ -84,7 +87,6 @@
 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.TypeHelper;
 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.expressions.AbstractFunctionCallExpression;
@@ -101,7 +103,15 @@
         SI
     }
 
-    private static final FunctionInfoRepository finfoRepo = new FunctionInfoRepository();
+    /*
+     * A function is categorized as public or private depending upon whether it can be used by the end-user in AQL queries.
+     */
+    public enum FunctionNamespace {
+        ASTERIX_PUBLIC,
+        ASTERIX_PRIVATE
+    }
+
+    private static final FunctionInfoRepository registeredFunctions = new FunctionInfoRepository();
 
     // it is supposed to be an identity mapping
     private final static Map<IFunctionInfo, IFunctionInfo> builtinFunctionsSet = new HashMap<IFunctionInfo, IFunctionInfo>();
@@ -116,417 +126,449 @@
     private final static Map<IFunctionInfo, IFunctionInfo> scalarToAggregateFunctionMap = new HashMap<IFunctionInfo, IFunctionInfo>();
     private static final Map<IFunctionInfo, SpatialFilterKind> spatialFilterFunctions = new HashMap<IFunctionInfo, SpatialFilterKind>();
 
-    public final static FunctionIdentifier TYPE_OF = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "type-of", 1);
-    public final static FunctionIdentifier GET_HANDLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "get-handle", 2);
-    public final static FunctionIdentifier GET_DATA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-data",
-            2);
-    public final static FunctionIdentifier EMBED_TYPE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "embed-type", 1);
+    public final static FunctionIdentifier TYPE_OF = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "type-of", 1);
+    public final static FunctionIdentifier GET_HANDLE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "get-handle", 2);
+    public final static FunctionIdentifier GET_DATA = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "get-data", 2);
+    public final static FunctionIdentifier EMBED_TYPE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "embed-type", 1);
 
-    public final static FunctionIdentifier GET_ITEM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-item",
-            2);
-    public final static FunctionIdentifier ANY_COLLECTION_MEMBER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "any-collection-member", 1);
-    public final static FunctionIdentifier LISTIFY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "listify", 1);
+    public final static FunctionIdentifier GET_ITEM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "get-item", 2);
+    public final static FunctionIdentifier ANY_COLLECTION_MEMBER = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "any-collection-member", 1);
+    public final static FunctionIdentifier LISTIFY = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "listify", 1);
     // public final static FunctionIdentifier BAGIFY = new
     // FunctionIdentifier(ASTERIX_NS, "bagify", 1, true);
-    public final static FunctionIdentifier LEN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "len", 1);
+    public final static FunctionIdentifier LEN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(), "len",
+            1);
 
-    public final static FunctionIdentifier CONCAT_NON_NULL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "concat-non-null", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier EMPTY_STREAM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "empty-stream", 0);
-    public final static FunctionIdentifier NON_EMPTY_STREAM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "non-empty-stream", 0);
+    public final static FunctionIdentifier CONCAT_NON_NULL = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "concat-non-null", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier EMPTY_STREAM = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "empty-stream", 0);
+    public final static FunctionIdentifier NON_EMPTY_STREAM = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "non-empty-stream", 0);
     public final static FunctionIdentifier ORDERED_LIST_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "ordered-list-constructor", FunctionIdentifier.VARARGS);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "ordered-list-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier UNORDERED_LIST_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "unordered-list-constructor", FunctionIdentifier.VARARGS);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "unordered-list-constructor", FunctionIdentifier.VARARGS);
 
     // records
     public final static FunctionIdentifier CLOSED_RECORD_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "closed-record-constructor", FunctionIdentifier.VARARGS);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "closed-record-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier OPEN_RECORD_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "open-record-constructor", FunctionIdentifier.VARARGS);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "open-record-constructor", FunctionIdentifier.VARARGS);
     public final static FunctionIdentifier RECORD_TYPE_CONSTRUCTOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "record-type-constructor", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier FIELD_ACCESS_BY_INDEX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "field-access-by-index", 2);
-    public final static FunctionIdentifier FIELD_ACCESS_BY_NAME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "field-access-by-name", 2);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "record-type-constructor", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier FIELD_ACCESS_BY_INDEX = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "field-access-by-index", 2);
+    public final static FunctionIdentifier FIELD_ACCESS_BY_NAME = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "field-access-by-name", 2);
 
-    public final static FunctionIdentifier NUMERIC_UNARY_MINUS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-unary-minus", 1);
+    public final static FunctionIdentifier NUMERIC_UNARY_MINUS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-unary-minus", 1);
 
-    public final static FunctionIdentifier NUMERIC_SUBTRACT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-subtract", 2);
-    public final static FunctionIdentifier NUMERIC_MULTIPLY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-multiply", 2);
-    public final static FunctionIdentifier NUMERIC_DIVIDE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-divide", 2);
-    public final static FunctionIdentifier NUMERIC_MOD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-mod", 2);
-    public final static FunctionIdentifier NUMERIC_IDIV = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-idiv", 2);
-    public final static FunctionIdentifier CARET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "caret", 2);
+    public final static FunctionIdentifier NUMERIC_SUBTRACT = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-subtract", 2);
+    public final static FunctionIdentifier NUMERIC_MULTIPLY = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-multiply", 2);
+    public final static FunctionIdentifier NUMERIC_DIVIDE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-divide", 2);
+    public final static FunctionIdentifier NUMERIC_MOD = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-mod", 2);
+    public final static FunctionIdentifier NUMERIC_IDIV = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-idiv", 2);
+    public final static FunctionIdentifier CARET = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "caret", 2);
 
-    public final static FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-abs", 1);
-    public final static FunctionIdentifier NUMERIC_CEILING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-ceiling", 1);
-    public final static FunctionIdentifier NUMERIC_FLOOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-floor", 1);
-    public final static FunctionIdentifier NUMERIC_ROUND = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "numeric-round", 1);
+    public final static FunctionIdentifier NUMERIC_ABS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-abs", 1);
+    public final static FunctionIdentifier NUMERIC_CEILING = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-ceiling", 1);
+    public final static FunctionIdentifier NUMERIC_FLOOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-floor", 1);
+    public final static FunctionIdentifier NUMERIC_ROUND = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-round", 1);
     public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-round-half-to-even", 1);
     public final static FunctionIdentifier NUMERIC_ROUND_HALF_TO_EVEN2 = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "numeric-round-half-to-even", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "numeric-round-half-to-even", 2);
     // String funcitons
-    public final static FunctionIdentifier STRING_EQUAL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "string-equal", 2);
-    public final static FunctionIdentifier STRING_START_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "start-with", 2);
-    public final static FunctionIdentifier STRING_END_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "end-with", 2);
-    public final static FunctionIdentifier STRING_MATCHES = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "matches", 2);
+    public final static FunctionIdentifier STRING_EQUAL = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "string-equal", 2);
+    public final static FunctionIdentifier STRING_START_WITH = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "start-with", 2);
+    public final static FunctionIdentifier STRING_END_WITH = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "end-with", 2);
+    public final static FunctionIdentifier STRING_MATCHES = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "matches", 2);
     public final static FunctionIdentifier STRING_MATCHES_WITH_FLAG = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "matches", 3);
-    public final static FunctionIdentifier STRING_LOWERCASE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "lowercase", 1);
-    public final static FunctionIdentifier STRING_REPLACE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "replace", 3);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "matches", 3);
+    public final static FunctionIdentifier STRING_LOWERCASE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "lowercase", 1);
+    public final static FunctionIdentifier STRING_REPLACE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "replace", 3);
     public final static FunctionIdentifier STRING_REPLACE_WITH_FLAG = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "replace", 4);
-    public final static FunctionIdentifier STRING_LENGTH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "string-length", 1);
-    public final static FunctionIdentifier SUBSTRING2 = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "replace", 4);
+    public final static FunctionIdentifier STRING_LENGTH = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-length", 1);
+    public final static FunctionIdentifier SUBSTRING2 = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
             "substring", 2);
-    public final static FunctionIdentifier SUBSTRING_BEFORE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "substring-before", 2);
-    public final static FunctionIdentifier SUBSTRING_AFTER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "substring-after", 2);
-    public final static FunctionIdentifier STRING_TO_CODEPOINT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "string-to-codepoint", 1);
-    public final static FunctionIdentifier CODEPOINT_TO_STRING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "codepoint-to-string", 1);
-    public final static FunctionIdentifier STRING_CONCAT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "string-concat", 1);
-    public final static FunctionIdentifier STRING_JOIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "string-join", 2);
+    public final static FunctionIdentifier SUBSTRING_BEFORE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "substring-before", 2);
+    public final static FunctionIdentifier SUBSTRING_AFTER = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "substring-after", 2);
+    public final static FunctionIdentifier STRING_TO_CODEPOINT = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-to-codepoint", 1);
+    public final static FunctionIdentifier CODEPOINT_TO_STRING = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "codepoint-to-string", 1);
+    public final static FunctionIdentifier STRING_CONCAT = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-concat", 1);
+    public final static FunctionIdentifier STRING_JOIN = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "string-join", 2);
 
-    public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "dataset", 1);
-    public final static FunctionIdentifier FEED_INGEST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "feed-ingest", 1);
+    public final static FunctionIdentifier DATASET = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "dataset", 1);
+    public final static FunctionIdentifier FEED_INGEST = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "feed-ingest", 1);
 
-    public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "index-search", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier INDEX_SEARCH = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "index-search", FunctionIdentifier.VARARGS);
 
     public final static FunctionIdentifier MAKE_FIELD_INDEX_HANDLE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "make-field-index-handle", 2);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "make-field-index-handle", 2);
     public final static FunctionIdentifier MAKE_FIELD_NAME_HANDLE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "make-field-name-handle", 1);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "make-field-name-handle", 1);
 
-    public final static FunctionIdentifier SUBSTRING = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+    public final static FunctionIdentifier SUBSTRING = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
             "substring", 3);
-    public final static FunctionIdentifier LIKE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "like", 2);
-    public final static FunctionIdentifier CONTAINS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "contains",
-            2);
-    public final static FunctionIdentifier STARTS_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "starts-with", 2);
-    public final static FunctionIdentifier ENDS_WITH = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+    public final static FunctionIdentifier LIKE = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "like", 2);
+    public final static FunctionIdentifier CONTAINS = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "contains", 2);
+    public final static FunctionIdentifier STARTS_WITH = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "starts-with", 2);
+    public final static FunctionIdentifier ENDS_WITH = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
             "ends-with", 2);
 
-    public final static FunctionIdentifier AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-avg", 1);
-    public final static FunctionIdentifier COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-count", 1);
-    public final static FunctionIdentifier SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-sum", 1);
-    public final static FunctionIdentifier LOCAL_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+    public final static FunctionIdentifier AVG = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "agg-avg", 1);
+    public final static FunctionIdentifier COUNT = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "agg-count", 1);
+    public final static FunctionIdentifier SUM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "agg-sum", 1);
+    public final static FunctionIdentifier LOCAL_SUM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
             "agg-local-sum", 1);
-    public final static FunctionIdentifier MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-max", 1);
-    public final static FunctionIdentifier LOCAL_MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+    public final static FunctionIdentifier MAX = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "agg-max", 1);
+    public final static FunctionIdentifier LOCAL_MAX = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
             "agg-local-max", 1);
-    public final static FunctionIdentifier MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-min", 1);
-    public final static FunctionIdentifier LOCAL_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+    public final static FunctionIdentifier MIN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "agg-min", 1);
+    public final static FunctionIdentifier LOCAL_MIN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
             "agg-local-min", 1);
-    public final static FunctionIdentifier GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "agg-global-avg", 1);
-    public final static FunctionIdentifier LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+    public final static FunctionIdentifier GLOBAL_AVG = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "agg-global-avg", 1);
+    public final static FunctionIdentifier LOCAL_AVG = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
             "agg-local-avg", 1);
 
-    public final static FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg", 1);
-    public final static FunctionIdentifier SCALAR_COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "count",
-            1);
-    public final static FunctionIdentifier SCALAR_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum", 1);
-    public final static FunctionIdentifier SCALAR_MAX = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "max", 1);
-    public final static FunctionIdentifier SCALAR_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "min", 1);
-    public final static FunctionIdentifier SCALAR_GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "global-avg", 1);
-    public final static FunctionIdentifier SCALAR_LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "local-avg", 1);
+    public final static FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "avg", 1);
+    public final static FunctionIdentifier SCALAR_COUNT = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "count", 1);
+    public final static FunctionIdentifier SCALAR_SUM = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "sum", 1);
+    public final static FunctionIdentifier SCALAR_MAX = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "max", 1);
+    public final static FunctionIdentifier SCALAR_MIN = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "min", 1);
+    public final static FunctionIdentifier SCALAR_GLOBAL_AVG = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "global-avg", 1);
+    public final static FunctionIdentifier SCALAR_LOCAL_AVG = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "local-avg", 1);
 
     // serializable aggregate functions
-    public final static FunctionIdentifier SERIAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "avg-serial", 1);
-    public final static FunctionIdentifier SERIAL_COUNT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "count-serial", 1);
-    public final static FunctionIdentifier SERIAL_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "sum-serial", 1);
-    public final static FunctionIdentifier SERIAL_LOCAL_SUM = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "local-sum-serial", 1);
-    public final static FunctionIdentifier SERIAL_GLOBAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "global-avg-serial", 1);
-    public final static FunctionIdentifier SERIAL_LOCAL_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "local-avg-serial", 1);
+    public final static FunctionIdentifier SERIAL_AVG = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "avg-serial", 1);
+    public final static FunctionIdentifier SERIAL_COUNT = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "count-serial", 1);
+    public final static FunctionIdentifier SERIAL_SUM = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "sum-serial", 1);
+    public final static FunctionIdentifier SERIAL_LOCAL_SUM = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "local-sum-serial", 1);
+    public final static FunctionIdentifier SERIAL_GLOBAL_AVG = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "global-avg-serial", 1);
+    public final static FunctionIdentifier SERIAL_LOCAL_AVG = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "local-avg-serial", 1);
 
-    public final static FunctionIdentifier SCAN_COLLECTION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "scan-collection", 1);
-    public final static FunctionIdentifier SUBSET_COLLECTION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "subset-collection", 3);
+    public final static FunctionIdentifier SCAN_COLLECTION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "scan-collection", 1);
+    public final static FunctionIdentifier SUBSET_COLLECTION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "subset-collection", 3);
 
-    public final static FunctionIdentifier RANGE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "range", 2);
+    public final static FunctionIdentifier RANGE = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "range", 2);
 
     // fuzzy functions:
-    public final static FunctionIdentifier FUZZY_EQ = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "fuzzy-eq",
-            2);
+    public final static FunctionIdentifier FUZZY_EQ = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "fuzzy-eq", 2);
 
-    public final static FunctionIdentifier PREFIX_LEN_JACCARD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "prefix-len-jaccard", 2);
+    public final static FunctionIdentifier PREFIX_LEN_JACCARD = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "prefix-len-jaccard", 2);
 
-    public final static FunctionIdentifier SIMILARITY_JACCARD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "similarity-jaccard", 2);
+    public final static FunctionIdentifier SIMILARITY_JACCARD = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "similarity-jaccard", 2);
     public final static FunctionIdentifier SIMILARITY_JACCARD_CHECK = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-check", 3);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "similarity-jaccard-check", 3);
     public final static FunctionIdentifier SIMILARITY_JACCARD_SORTED = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-sorted", 2);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-sorted", 2);
     public final static FunctionIdentifier SIMILARITY_JACCARD_SORTED_CHECK = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-sorted-check", 3);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-sorted-check", 3);
     public final static FunctionIdentifier SIMILARITY_JACCARD_PREFIX = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix", 6);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-prefix", 6);
     public final static FunctionIdentifier SIMILARITY_JACCARD_PREFIX_CHECK = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "similarity-jaccard-prefix-check", 6);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "similarity-jaccard-prefix-check", 6);
 
-    public final static FunctionIdentifier EDIT_DISTANCE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "edit-distance", 2);
-    public final static FunctionIdentifier EDIT_DISTANCE_CHECK = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "edit-distance-check", 3);
+    public final static FunctionIdentifier EDIT_DISTANCE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "edit-distance", 2);
+    public final static FunctionIdentifier EDIT_DISTANCE_CHECK = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "edit-distance-check", 3);
     public final static FunctionIdentifier EDIT_DISTANCE_LIST_IS_FILTERABLE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "edit-distance-list-is-filterable", 2);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "edit-distance-list-is-filterable", 2);
     public final static FunctionIdentifier EDIT_DISTANCE_STRING_IS_FILTERABLE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "edit-distance-string-is-filterable", 4);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "edit-distance-string-is-filterable", 4);
 
     // tokenizers:
-    public final static FunctionIdentifier WORD_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "word-tokens", 1);
-    public final static FunctionIdentifier HASHED_WORD_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "hashed-word-tokens", 1);
+    public final static FunctionIdentifier WORD_TOKENS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "word-tokens", 1);
+    public final static FunctionIdentifier HASHED_WORD_TOKENS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "hashed-word-tokens", 1);
     public final static FunctionIdentifier COUNTHASHED_WORD_TOKENS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "counthashed-word-tokens", 1);
-    public final static FunctionIdentifier GRAM_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "gram-tokens", 3);
-    public final static FunctionIdentifier HASHED_GRAM_TOKENS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "hashed-gram-tokens", 3);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "counthashed-word-tokens", 1);
+    public final static FunctionIdentifier GRAM_TOKENS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "gram-tokens", 3);
+    public final static FunctionIdentifier HASHED_GRAM_TOKENS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "hashed-gram-tokens", 3);
     public final static FunctionIdentifier COUNTHASHED_GRAM_TOKENS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "counthashed-gram-tokens", 3);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "counthashed-gram-tokens", 3);
 
-    public final static FunctionIdentifier TID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "tid", 0);
+    public final static FunctionIdentifier TID = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "tid", 0);
 
     // constructors:
-    public final static FunctionIdentifier BOOLEAN_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "boolean", 1);
-    public final static FunctionIdentifier NULL_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "null", 1);
-    public final static FunctionIdentifier STRING_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "string", 1);
-    public final static FunctionIdentifier INT8_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int8", 1);
-    public final static FunctionIdentifier INT16_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int16", 1);
-    public final static FunctionIdentifier INT32_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int32", 1);
-    public final static FunctionIdentifier INT64_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "int64", 1);
-    public final static FunctionIdentifier FLOAT_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "float", 1);
-    public final static FunctionIdentifier DOUBLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "double", 1);
-    public final static FunctionIdentifier POINT_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "point", 1);
-    public final static FunctionIdentifier POINT3D_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "point3d", 1);
-    public final static FunctionIdentifier LINE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "line", 1);
-    public final static FunctionIdentifier CIRCLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "circle", 1);
-    public final static FunctionIdentifier RECTANGLE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "rectangle", 1);
-    public final static FunctionIdentifier POLYGON_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "polygon", 1);
-    public final static FunctionIdentifier TIME_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "time", 1);
-    public final static FunctionIdentifier DATE_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "date", 1);
-    public final static FunctionIdentifier DATETIME_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "datetime", 1);
-    public final static FunctionIdentifier DURATION_CONSTRUCTOR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "duration", 1);
+    public final static FunctionIdentifier BOOLEAN_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "boolean", 1);
+    public final static FunctionIdentifier NULL_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "null", 1);
+    public final static FunctionIdentifier STRING_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "string", 1);
+    public final static FunctionIdentifier INT8_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "int8", 1);
+    public final static FunctionIdentifier INT16_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "int16", 1);
+    public final static FunctionIdentifier INT32_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "int32", 1);
+    public final static FunctionIdentifier INT64_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "int64", 1);
+    public final static FunctionIdentifier FLOAT_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "float", 1);
+    public final static FunctionIdentifier DOUBLE_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "double", 1);
+    public final static FunctionIdentifier POINT_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "point", 1);
+    public final static FunctionIdentifier POINT3D_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "point3d", 1);
+    public final static FunctionIdentifier LINE_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "line", 1);
+    public final static FunctionIdentifier CIRCLE_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "circle", 1);
+    public final static FunctionIdentifier RECTANGLE_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "rectangle", 1);
+    public final static FunctionIdentifier POLYGON_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "polygon", 1);
+    public final static FunctionIdentifier TIME_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "time", 1);
+    public final static FunctionIdentifier DATE_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "date", 1);
+    public final static FunctionIdentifier DATETIME_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "datetime", 1);
+    public final static FunctionIdentifier DURATION_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "duration", 1);
+
+    public final static FunctionIdentifier YEAR_MONTH_DURATION_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "year-month-duration", 1);
+    public final static FunctionIdentifier DAY_TIME_DURATION_CONSTRUCTOR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "day-time-duration", 1);
+
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_DATE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "interval-from-date", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-from-date", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_TIME = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "interval-from-time", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-from-time", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_DATETIME = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "interval-from-datetime", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-from-datetime", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_START_FROM_DATE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "interval-start-from-date", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-start-from-date", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_START_FROM_TIME = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "interval-start-from-time", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-start-from-time", 2);
     public final static FunctionIdentifier INTERVAL_CONSTRUCTOR_START_FROM_DATETIME = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "interval-start-from-datetime", 2);
-    public final static FunctionIdentifier INTERVAL_BEFORE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-before", 2);
-    public final static FunctionIdentifier INTERVAL_AFTER = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-after", 2);
-    public final static FunctionIdentifier INTERVAL_MEETS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-meets", 2);
-    public final static FunctionIdentifier INTERVAL_MET_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-met-by", 2);
-    public final static FunctionIdentifier INTERVAL_OVERLAPS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-overlaps", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-start-from-datetime", 2);
+    public final static FunctionIdentifier INTERVAL_BEFORE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-before", 2);
+    public final static FunctionIdentifier INTERVAL_AFTER = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-after", 2);
+    public final static FunctionIdentifier INTERVAL_MEETS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-meets", 2);
+    public final static FunctionIdentifier INTERVAL_MET_BY = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-met-by", 2);
+    public final static FunctionIdentifier INTERVAL_OVERLAPS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-overlaps", 2);
     public final static FunctionIdentifier INTERVAL_OVERLAPPED_BY = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "interval-overlapped-by", 2);
-    public final static FunctionIdentifier OVERLAP = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "overlap", 2);
-    public final static FunctionIdentifier INTERVAL_STARTS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-starts", 2);
-    public final static FunctionIdentifier INTERVAL_STARTED_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-started-by", 2);
-    public final static FunctionIdentifier INTERVAL_COVERS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-covers", 2);
-    public final static FunctionIdentifier INTERVAL_COVERED_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-covered-by", 2);
-    public final static FunctionIdentifier INTERVAL_ENDS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-ends", 2);
-    public final static FunctionIdentifier INTERVAL_ENDED_BY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-ended-by", 2);
-    public final static FunctionIdentifier CURRENT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "current-time", 0);
-    public final static FunctionIdentifier CURRENT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "current-date", 0);
-    public final static FunctionIdentifier CURRENT_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "current-datetime", 0);
-    public final static FunctionIdentifier DURATION_EQUAL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "duration-equal", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-overlapped-by", 2);
+    public final static FunctionIdentifier OVERLAP = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "overlap", 2);
+    public final static FunctionIdentifier INTERVAL_STARTS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-starts", 2);
+    public final static FunctionIdentifier INTERVAL_STARTED_BY = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-started-by", 2);
+    public final static FunctionIdentifier INTERVAL_COVERS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-covers", 2);
+    public final static FunctionIdentifier INTERVAL_COVERED_BY = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-covered-by", 2);
+    public final static FunctionIdentifier INTERVAL_ENDS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-ends", 2);
+    public final static FunctionIdentifier INTERVAL_ENDED_BY = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "interval-ended-by", 2);
+    public final static FunctionIdentifier CURRENT_TIME = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "current-time", 0);
+    public final static FunctionIdentifier CURRENT_DATE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "current-date", 0);
+    public final static FunctionIdentifier CURRENT_DATETIME = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "current-datetime", 0);
+    public final static FunctionIdentifier DURATION_EQUAL = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "duration-equal", 2);
     public final static FunctionIdentifier YEAR_MONTH_DURATION_GREATER_THAN = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "year-month-duration-greater-than", 2);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "year-month-duration-greater-than", 2);
     public final static FunctionIdentifier YEAR_MONTH_DURATION_LESS_THAN = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "year-month-duration-less-than", 2);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "year-month-duration-less-than", 2);
     public final static FunctionIdentifier DAY_TIME_DURATION_GREATER_THAN = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "day-time-duration-greater-than", 2);
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "day-time-duration-greater-than", 2);
     public final static FunctionIdentifier DAY_TIME_DURATION_LESS_THAN = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "day-time-duration-less-than", 2);
-    public final static FunctionIdentifier DURATION_FROM_MONTHS = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "duration-from-months", 1);
-    public final static FunctionIdentifier MONTHS_OF_YEAR_MONTH_DURATION = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "months-of-year-month-duration", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "day-time-duration-less-than", 2);
+    public final static FunctionIdentifier DURATION_FROM_MONTHS = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "duration-from-months", 1);
+    public final static FunctionIdentifier MONTHS_FROM_YEAR_MONTH_DURATION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "months-from-year-month-duration", 1);
     public final static FunctionIdentifier DURATION_FROM_MILLISECONDS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "duration-from-ms", 1);
-    public final static FunctionIdentifier MILLISECONDS_OF_DAY_TIME_DURATION = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "ms-of-day-time-duration", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "duration-from-ms", 1);
+    public final static FunctionIdentifier MILLISECONDS_FROM_DAY_TIME_DURATION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "ms-from-day-time-duration", 1);
+
+    public final static FunctionIdentifier GET_YEAR_MONTH_DURATION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-year-month-duration", 1);
+    public final static FunctionIdentifier GET_DAY_TIME_DURATION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-day-time-duration", 1);
 
     // spatial
-    public final static FunctionIdentifier CREATE_POINT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-point", 2);
-    public final static FunctionIdentifier CREATE_LINE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-line", 2);
-    public final static FunctionIdentifier CREATE_POLYGON = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-polygon", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier CREATE_CIRCLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-circle", 2);
-    public final static FunctionIdentifier CREATE_RECTANGLE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-rectangle", 2);
-    public final static FunctionIdentifier SPATIAL_INTERSECT = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-intersect", 2);
-    public final static FunctionIdentifier SPATIAL_AREA = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-area", 1);
-    public final static FunctionIdentifier SPATIAL_DISTANCE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-distance", 2);
-    public final static FunctionIdentifier CREATE_MBR = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "create-mbr", 3);
-    public final static FunctionIdentifier SPATIAL_CELL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "spatial-cell", 4);
-    public final static FunctionIdentifier SWITCH_CASE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "switch-case", FunctionIdentifier.VARARGS);
-    public final static FunctionIdentifier REG_EXP = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "reg-exp", 2);
+    public final static FunctionIdentifier CREATE_POINT = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-point", 2);
+    public final static FunctionIdentifier CREATE_LINE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-line", 2);
+    public final static FunctionIdentifier CREATE_POLYGON = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-polygon", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier CREATE_CIRCLE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-circle", 2);
+    public final static FunctionIdentifier CREATE_RECTANGLE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "create-rectangle", 2);
+    public final static FunctionIdentifier SPATIAL_INTERSECT = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-intersect", 2);
+    public final static FunctionIdentifier SPATIAL_AREA = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-area", 1);
+    public final static FunctionIdentifier SPATIAL_DISTANCE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-distance", 2);
+    public final static FunctionIdentifier CREATE_MBR = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "create-mbr", 3);
+    public final static FunctionIdentifier SPATIAL_CELL = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "spatial-cell", 4);
+    public final static FunctionIdentifier SWITCH_CASE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "switch-case", FunctionIdentifier.VARARGS);
+    public final static FunctionIdentifier REG_EXP = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "reg-exp", 2);
 
-    public final static FunctionIdentifier INJECT_FAILURE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "inject-failure", 2);
-    public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "cast-record", 1);
+    public final static FunctionIdentifier INJECT_FAILURE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "inject-failure", 2);
+    public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PRIVATE.name(), "cast-record", 1);
+    public final static FunctionIdentifier CAST_LIST = new FunctionIdentifier(FunctionNamespace.ASTERIX_PUBLIC.name(),
+            "cast-list", 1);
 
     // Spatial and temporal type accessors
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "year", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "year", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_MONTH = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "month", 1);
-    public static final FunctionIdentifier ACCESSOR_TEMPORAL_DAY = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "day", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "month", 1);
+    public static final FunctionIdentifier ACCESSOR_TEMPORAL_DAY = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "day", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_HOUR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "hour", 1);
-    public static final FunctionIdentifier ACCESSOR_TEMPORAL_MIN = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "minute", 1);
-    public static final FunctionIdentifier ACCESSOR_TEMPORAL_SEC = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "second", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "hour", 1);
+    public static final FunctionIdentifier ACCESSOR_TEMPORAL_MIN = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "minute", 1);
+    public static final FunctionIdentifier ACCESSOR_TEMPORAL_SEC = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "second", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_MILLISEC = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "millisecond", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "millisecond", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_INTERVAL_START = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "get-interval-start", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-interval-start", 1);
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_INTERVAL_END = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "get-interval-end", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-interval-end", 1);
 
     // Temporal functions
     public static final FunctionIdentifier DATE_FROM_UNIX_TIME_IN_DAYS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "date-from-unix-time-in-days", 1);
-    public static final FunctionIdentifier DATE_FROM_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "date-from-datetime", 1);
-    public final static FunctionIdentifier ADD_DATE_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "add-date-duration", 2);
-    public final static FunctionIdentifier SUBTRACT_DATE = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "subtract-date", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "date-from-unix-time-in-days", 1);
+    public static final FunctionIdentifier DATE_FROM_DATETIME = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "date-from-datetime", 1);
+    public final static FunctionIdentifier ADD_DATE_DURATION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "add-date-duration", 2);
+    public final static FunctionIdentifier SUBTRACT_DATE = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "subtract-date", 2);
     public final static FunctionIdentifier TIME_FROM_UNIX_TIME_IN_MS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "time-from-unix-time-in-ms", 1);
-    public final static FunctionIdentifier TIME_FROM_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "time-from-datetime", 1);
-    public final static FunctionIdentifier SUBTRACT_TIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "subtract-time", 2);
-    public final static FunctionIdentifier ADD_TIME_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "add-time-duration", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "time-from-unix-time-in-ms", 1);
+    public final static FunctionIdentifier TIME_FROM_DATETIME = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "time-from-datetime", 1);
+    public final static FunctionIdentifier SUBTRACT_TIME = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "subtract-time", 2);
+    public final static FunctionIdentifier ADD_TIME_DURATION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "add-time-duration", 2);
     public final static FunctionIdentifier DATETIME_FROM_UNIX_TIME_IN_MS = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "datetime-from-unix-time-in-ms", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "datetime-from-unix-time-in-ms", 1);
     public final static FunctionIdentifier DATETIME_FROM_DATE_TIME = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "datetime-from-date-time", 2);
-    public final static FunctionIdentifier SUBTRACT_DATETIME = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "subtract-datetime", 2);
-    public final static FunctionIdentifier ADD_DATETIME_DURATION = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "add-datetime-duration", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "datetime-from-date-time", 2);
+    public final static FunctionIdentifier SUBTRACT_DATETIME = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "subtract-datetime", 2);
+    public final static FunctionIdentifier ADD_DATETIME_DURATION = new FunctionIdentifier(
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "add-datetime-duration", 2);
     public final static FunctionIdentifier CALENDAR_DURATION_FROM_DATETIME = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "calendar-duration-from-datetime", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "calendar-duration-from-datetime", 2);
     public final static FunctionIdentifier CALENDAR_DURATION_FROM_DATE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "calendar-duration-from-date", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "calendar-duration-from-date", 2);
     public final static FunctionIdentifier ADJUST_TIME_FOR_TIMEZONE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "adjust-time-for-timezone", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "adjust-time-for-timezone", 2);
     public final static FunctionIdentifier ADJUST_DATETIME_FOR_TIMEZONE = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "adjust-datetime-for-timezone", 2);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "adjust-datetime-for-timezone", 2);
 
     public final static FunctionIdentifier GET_POINT_X_COORDINATE_ACCESSOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "get-x", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-x", 1);
     public final static FunctionIdentifier GET_POINT_Y_COORDINATE_ACCESSOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "get-y", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-y", 1);
     public final static FunctionIdentifier GET_CIRCLE_RADIUS_ACCESSOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "get-radius", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-radius", 1);
     public final static FunctionIdentifier GET_CIRCLE_CENTER_ACCESSOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "get-center", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-center", 1);
     public final static FunctionIdentifier GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR = new FunctionIdentifier(
-            FunctionConstants.ASTERIX_NS, "get-points", 1);
+            FunctionNamespace.ASTERIX_PUBLIC.name(), "get-points", 1);
 
     public static final FunctionIdentifier EQ = AlgebricksBuiltinFunctions.EQ;
     public static final FunctionIdentifier LE = AlgebricksBuiltinFunctions.LE;
@@ -540,11 +582,11 @@
     public static final FunctionIdentifier NUMERIC_ADD = AlgebricksBuiltinFunctions.NUMERIC_ADD;
     public static final FunctionIdentifier IS_NULL = AlgebricksBuiltinFunctions.IS_NULL;
 
-    public static final FunctionIdentifier NOT_NULL = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "not-null",
-            1);
+    public static final FunctionIdentifier NOT_NULL = new FunctionIdentifier(FunctionNamespace.ASTERIX_PRIVATE.name(),
+            "not-null", 1);
 
     public static IFunctionInfo getAsterixFunctionInfo(FunctionIdentifier fid) {
-        IFunctionInfo finfo = finfoRepo.get(fid);;
+        IFunctionInfo finfo = registeredFunctions.get(fid);
         if (finfo == null) {
             finfo = new AsterixFunctionInfo(fid);
         }
@@ -552,7 +594,7 @@
     }
 
     public static AsterixFunctionInfo lookupFunction(FunctionIdentifier fid) {
-        return (AsterixFunctionInfo) finfoRepo.get(fid);
+        return (AsterixFunctionInfo) registeredFunctions.get(fid);
     }
 
     static {
@@ -573,29 +615,15 @@
         // and then, Asterix builtin functions
         add(NOT_NULL, NotNullTypeComputer.INSTANCE);
         add(ANY_COLLECTION_MEMBER, NonTaggedCollectionMemberResultType.INSTANCE);
-        addPrivateFunction(AVG, OptionalADoubleTypeComputer.INSTANCE);
+        add(AVG, OptionalADoubleTypeComputer.INSTANCE);
         add(BOOLEAN_CONSTRUCTOR, UnaryBooleanOrNullFunctionTypeComputer.INSTANCE);
         add(CARET, NonTaggedNumericAddSubMulDivTypeComputer.INSTANCE);
         add(CIRCLE_CONSTRUCTOR, OptionalACircleTypeComputer.INSTANCE);
         add(CLOSED_RECORD_CONSTRUCTOR, ClosedRecordConstructorResultType.INSTANCE);
-        add(CONCAT_NON_NULL, new IResultTypeComputer() {
-            @Override
-            public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
-                    IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
-                AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
-                if (f.getArguments().size() < 1) {
-                    return BuiltinType.ANULL;
-                }
-                ILogicalExpression a0 = f.getArguments().get(0).getValue();
-                IAType t0 = (IAType) env.getType(a0);
-                if (TypeHelper.canBeNull(t0)) {
-                    return t0;
-                }
-                return AUnionType.createNullableType(t0);
-            }
-        });
+        add(CONCAT_NON_NULL, ConcatNonNullTypeComputer.INSTANCE);
+
         add(CONTAINS, ABooleanTypeComputer.INSTANCE);
-        addPrivateFunction(COUNT, AInt32TypeComputer.INSTANCE);
+        add(COUNT, AInt32TypeComputer.INSTANCE);
         add(COUNTHASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
         add(COUNTHASHED_WORD_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
         add(CREATE_CIRCLE, ACircleTypeComputer.INSTANCE);
@@ -609,6 +637,8 @@
         add(DATETIME_CONSTRUCTOR, OptionalADateTimeTypeComputer.INSTANCE);
         add(DOUBLE_CONSTRUCTOR, OptionalADoubleTypeComputer.INSTANCE);
         add(DURATION_CONSTRUCTOR, OptionalADurationTypeComputer.INSTANCE);
+        add(YEAR_MONTH_DURATION_CONSTRUCTOR, OptionalAYearMonthDurationTypeComputer.INSTANCE);
+        add(DAY_TIME_DURATION_CONSTRUCTOR, OptionalADayTimeDurationTypeComputer.INSTANCE);
         add(EDIT_DISTANCE, AInt32TypeComputer.INSTANCE);
         add(EDIT_DISTANCE_CHECK, OrderedListOfAnyTypeComputer.INSTANCE);
         add(EDIT_DISTANCE_STRING_IS_FILTERABLE, ABooleanTypeComputer.INSTANCE);
@@ -630,7 +660,7 @@
         add(GET_HANDLE, null); // TODO
         add(GET_ITEM, NonTaggedGetItemResultType.INSTANCE);
         add(GET_DATA, null); // TODO
-        addPrivateFunction(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
+        add(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
         add(GRAM_TOKENS, OrderedListOfAStringTypeComputer.INSTANCE);
         add(GLOBAL_AVG, OptionalADoubleTypeComputer.INSTANCE);
         add(HASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
@@ -651,7 +681,7 @@
         add(LIKE, BinaryBooleanOrNullFunctionTypeComputer.INSTANCE);
         add(LINE_CONSTRUCTOR, OptionalALineTypeComputer.INSTANCE);
         add(LISTIFY, OrderedListConstructorResultType.INSTANCE);
-        addPrivateFunction(LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
+        add(LOCAL_AVG, NonTaggedLocalAvgTypeComputer.INSTANCE);
         add(MAKE_FIELD_INDEX_HANDLE, null); // TODO
         add(MAKE_FIELD_NAME_HANDLE, null); // TODO
         add(MAX, NonTaggedSumTypeComputer.INSTANCE);
@@ -701,7 +731,7 @@
         add(RECTANGLE_CONSTRUCTOR, OptionalARectangleTypeComputer.INSTANCE);
         // add(RECORD_TYPE_CONSTRUCTOR, null);
         add(SCALAR_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
-        add(SCALAR_COUNT, ScalarVersionOfAggregateResultType.INSTANCE);
+        add(SCALAR_COUNT, AInt32TypeComputer.INSTANCE);
         add(SCALAR_GLOBAL_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
         add(SCALAR_LOCAL_AVG, ScalarVersionOfAggregateResultType.INSTANCE);
         add(SCALAR_MAX, ScalarVersionOfAggregateResultType.INSTANCE);
@@ -769,12 +799,13 @@
             }
         });
         add(SUBSTRING, SubstringTypeComputer.INSTANCE);
-        addPrivateFunction(SUM, NonTaggedSumTypeComputer.INSTANCE);
+        add(SUM, NonTaggedSumTypeComputer.INSTANCE);
         add(LOCAL_SUM, NonTaggedSumTypeComputer.INSTANCE);
         add(SWITCH_CASE, NonTaggedSwitchCaseComputer.INSTANCE);
         add(REG_EXP, ABooleanTypeComputer.INSTANCE);
         add(INJECT_FAILURE, InjectFailureTypeComputer.INSTANCE);
         add(CAST_RECORD, CastRecordResultTypeComputer.INSTANCE);
+        add(CAST_LIST, CastListResultTypeComputer.INSTANCE);
 
         add(TID, AInt32TypeComputer.INSTANCE);
         add(TIME_CONSTRUCTOR, OptionalATimeTypeComputer.INSTANCE);
@@ -840,8 +871,10 @@
         add(DURATION_EQUAL, OptionalABooleanTypeComputer.INSTANCE);
         add(DURATION_FROM_MONTHS, OptionalADurationTypeComputer.INSTANCE);
         add(DURATION_FROM_MILLISECONDS, OptionalADurationTypeComputer.INSTANCE);
-        add(MONTHS_OF_YEAR_MONTH_DURATION, OptionalAInt32TypeComputer.INSTANCE);
-        add(MILLISECONDS_OF_DAY_TIME_DURATION, OptionalAInt64TypeComputer.INSTANCE);
+        add(MONTHS_FROM_YEAR_MONTH_DURATION, OptionalAInt32TypeComputer.INSTANCE);
+        add(MILLISECONDS_FROM_DAY_TIME_DURATION, OptionalAInt64TypeComputer.INSTANCE);
+        add(GET_DAY_TIME_DURATION, OptionalADayTimeDurationTypeComputer.INSTANCE);
+        add(GET_YEAR_MONTH_DURATION, OptionalAYearMonthDurationTypeComputer.INSTANCE);
 
         // interval constructors
         add(INTERVAL_CONSTRUCTOR_DATE, OptionalAIntervalTypeComputer.INSTANCE);
@@ -929,6 +962,7 @@
 
     static {
         addUnnestFun(DATASET, false);
+        addUnnestFun(FEED_INGEST, false);
         addUnnestFun(RANGE, true);
         addUnnestFun(SCAN_COLLECTION, false);
         addUnnestFun(SUBSET_COLLECTION, false);
@@ -1032,14 +1066,7 @@
         IFunctionInfo functionInfo = getAsterixFunctionInfo(fi);
         builtinFunctionsSet.put(functionInfo, functionInfo);
         funTypeComputer.put(functionInfo, typeComputer);
-        finfoRepo.put(fi);
-    }
-
-    private static IFunctionInfo addPrivateFunction(FunctionIdentifier fi, IResultTypeComputer typeComputer) {
-        IFunctionInfo functionInfo = getAsterixFunctionInfo(fi);
-        builtinFunctionsSet.put(functionInfo, functionInfo);
-        funTypeComputer.put(functionInfo, typeComputer);
-        return functionInfo;
+        registeredFunctions.put(fi);
     }
 
     private static void addAgg(FunctionIdentifier fi) {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
index ab32b6b..84c9e6c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
@@ -93,6 +93,7 @@
     @Override
     public void set(byte[] b, int s, int len) {
         reset();
+        super.set(b, s, len);
 
         int numberOfitems = AInt32SerializerDeserializer.getInt(b, s + 6);
         int itemOffset;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java
index 494ea6f..ed9690c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/cast/ARecordCaster.java
@@ -99,7 +99,7 @@
             start = bos.size();
             dos.write(ATypeTag.NULL.serialize());
             end = bos.size();
-            nullTypeTag.set(bos.getByteArray(), start, end);
+            nullTypeTag.set(bos.getByteArray(), start, end - start);
         } catch (IOException e) {
             throw new IllegalStateException(e);
         }
@@ -150,7 +150,7 @@
         for (int i = 0; i < optionalFields.length; i++)
             optionalFields[i] = false;
 
-        bos.reset(nullReference.getStartOffset() + nullReference.getLength());
+        bos.reset(nullTypeTag.getStartOffset() + nullTypeTag.getLength());
         for (int i = 0; i < numSchemaFields; i++) {
             ATypeTag ftypeTag = fieldTypes[i].getTypeTag();
             String fname = fieldNames[i];
@@ -278,8 +278,14 @@
             // recursively casting, the result of casting can always be thought
             // as flat
             if (optionalFields[i]) {
-                nestedVisitorArg.second = ((AUnionType) fType).getUnionList().get(
-                        NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+                if (fieldTypeTags.size() <= i || fieldTypeTags.get(i) == null
+                        || fieldTypeTags.get(i).equals(nullTypeTag)) {
+                    //the field is optional in the input record
+                    nestedVisitorArg.second = ((AUnionType) fType).getUnionList().get(0);
+                } else {
+                    nestedVisitorArg.second = ((AUnionType) fType).getUnionList().get(
+                            NonTaggedFormatUtil.OPTIONAL_TYPE_INDEX_IN_UNION_LIST);
+                }
             }
             field.accept(visitor, nestedVisitorArg);
             recBuilder.addField(i, nestedVisitorArg.first);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java
index b78308d..cdbf5ca 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/APrintVisitor.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ACirclePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ADatePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ADateTimePrinter;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ADayTimeDurationPrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ADoublePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ADurationPrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AFloatPrinter;
@@ -40,6 +41,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ARectanglePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AStringPrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.ATimePrinter;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.AYearMonthDurationPrinter;
 import edu.uci.ics.asterix.om.pointables.AFlatValuePointable;
 import edu.uci.ics.asterix.om.pointables.AListPointable;
 import edu.uci.ics.asterix.om.pointables.ARecordPointable;
@@ -146,6 +148,14 @@
                     ADurationPrinter.INSTANCE.print(b, s, l, ps);
                     break;
                 }
+                case YEARMONTHDURATION: {
+                    AYearMonthDurationPrinter.INSTANCE.print(b, s, l, ps);
+                    break;
+                }
+                case DAYTIMEDURATION: {
+                    ADayTimeDurationPrinter.INSTANCE.print(b, s, l, ps);
+                    break;
+                }
                 case POINT: {
                     APointPrinter.INSTANCE.print(b, s, l, ps);
                     break;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java
index b2d5339..e88ed7e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/printer/json/APrintVisitor.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ACirclePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADatePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADateTimePrinter;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADayTimeDurationPrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADoublePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ADurationPrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AFloatPrinter;
@@ -39,6 +40,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ARectanglePrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AStringPrinter;
 import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.ATimePrinter;
+import edu.uci.ics.asterix.dataflow.data.nontagged.printers.json.AYearMonthDurationPrinter;
 import edu.uci.ics.asterix.om.pointables.AFlatValuePointable;
 import edu.uci.ics.asterix.om.pointables.AListPointable;
 import edu.uci.ics.asterix.om.pointables.ARecordPointable;
@@ -173,6 +175,14 @@
                     AStringPrinter.INSTANCE.print(b, s, l, ps);
                     break;
                 }
+                case YEARMONTHDURATION: {
+                    AYearMonthDurationPrinter.INSTANCE.print(b, s, l, ps);
+                    break;
+                }
+                case DAYTIMEDURATION: {
+                    ADayTimeDurationPrinter.INSTANCE.print(b, s, l, ps);
+                    break;
+                }
                 default: {
                     throw new NotImplementedException("No printer for type " + typeTag);
                 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java
new file mode 100644
index 0000000..4c3c56b
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
+import edu.uci.ics.asterix.om.types.IAType;
+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.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+/**
+ * The type computer for the cast-list function
+ * 
+ * @author yingyib
+ */
+public class CastListResultTypeComputer implements IResultTypeComputer {
+
+    public static final CastListResultTypeComputer INSTANCE = new CastListResultTypeComputer();
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) expression;
+        return TypeComputerUtilities.getRequiredType(funcExpr);
+    }
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/ConcatNonNullTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/ConcatNonNullTypeComputer.java
new file mode 100644
index 0000000..7bf2668
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/ConcatNonNullTypeComputer.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.AUnionType;
+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.core.algebra.base.ILogicalExpression;
+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.metadata.IMetadataProvider;
+
+/**
+ * The type computer for concat-not-null.
+ * Note that this function is only used for the if-then-else clause.
+ * 
+ * @author yingyib
+ */
+public class ConcatNonNullTypeComputer implements IResultTypeComputer {
+
+    public static final ConcatNonNullTypeComputer INSTANCE = new ConcatNonNullTypeComputer();
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expression;
+        if (f.getArguments().size() < 1) {
+            return BuiltinType.ANULL;
+        }
+        List<IAType> possibleTypes = new ArrayList<IAType>();
+        for (int i = 0; i < f.getArguments().size(); i++) {
+            ILogicalExpression arg = f.getArguments().get(i).getValue();
+            IAType type = (IAType) env.getType(arg);
+            if (type.getTypeTag() == ATypeTag.UNION) {
+                List<IAType> typeList = ((AUnionType) type).getUnionList();
+                for (IAType t : typeList) {
+                    if (t.getTypeTag() != ATypeTag.NULL) {
+                        //CONCAT_NON_NULL cannot return null because it's only used for if-else construct
+                        if (!possibleTypes.contains(t))
+                            possibleTypes.add(t);
+                    }
+                }
+            } else {
+                if (!possibleTypes.contains(type))
+                    possibleTypes.add(type);
+            }
+        }
+        if (possibleTypes.size() == 1) {
+            return possibleTypes.get(0);
+        } else {
+            throw new AlgebricksException("The two branches of the if-else clause should return the same type.");
+        }
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedFieldAccessByNameResultType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedFieldAccessByNameResultType.java
index db9d932..6d5880e 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedFieldAccessByNameResultType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/NonTaggedFieldAccessByNameResultType.java
@@ -70,6 +70,9 @@
                     if (t1.getTypeTag() == ATypeTag.RECORD) {
                         return (ARecordType) t1;
                     }
+                    if (t1.getTypeTag() == ATypeTag.ANY) {
+                        return DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+                    }
                 }
             }
             default: {
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/OptionalADayTimeDurationTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/OptionalADayTimeDurationTypeComputer.java
new file mode 100644
index 0000000..38f76e3
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/OptionalADayTimeDurationTypeComputer.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.AUnionType;
+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.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class OptionalADayTimeDurationTypeComputer implements IResultTypeComputer {
+
+    public static final OptionalADayTimeDurationTypeComputer INSTANCE = new OptionalADayTimeDurationTypeComputer();
+
+    private OptionalADayTimeDurationTypeComputer() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer#computeType(edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression, edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment, edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider)
+     */
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        List<IAType> unionList = new ArrayList<IAType>();
+        unionList.add(BuiltinType.ANULL);
+        unionList.add(BuiltinType.ADAYTIMEDURATION);
+        return new AUnionType(unionList, "OptionalDayTimeDuration");
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/OptionalAYearMonthDurationTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/OptionalAYearMonthDurationTypeComputer.java
new file mode 100644
index 0000000..4c08142
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/OptionalAYearMonthDurationTypeComputer.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.types.AUnionType;
+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.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+public class OptionalAYearMonthDurationTypeComputer implements IResultTypeComputer {
+
+    public static final OptionalAYearMonthDurationTypeComputer INSTANCE = new OptionalAYearMonthDurationTypeComputer();
+
+    private OptionalAYearMonthDurationTypeComputer() {
+
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer#computeType(edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression, edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment, edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider)
+     */
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        List<IAType> unionList = new ArrayList<IAType>();
+        unionList.add(BuiltinType.ANULL);
+        unionList.add(BuiltinType.AYEARMONTHDURATION);
+        return new AUnionType(unionList, "OptionalYearMonthDuration");
+    }
+
+}
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
index c8ba938..9126065 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/ScalarVersionOfAggregateResultType.java
@@ -1,6 +1,5 @@
 package edu.uci.ics.asterix.om.typecomputer.impl;
 
-
 import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.AUnionType;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
index ca7bc24..037cd29 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ARecordType.java
@@ -259,6 +259,11 @@
                 case FLOAT:
                 case DOUBLE:
                 case STRING:
+                case DATE:
+                case TIME:
+                case DATETIME:
+                case YEARMONTHDURATION:
+                case DAYTIMEDURATION:
                     break;
                 case UNION:
                     throw new AlgebricksException("The partitioning key \"" + fieldName + "\" cannot be nullable");
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java
index a0ad3e1..e88596d 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/ATypeTag.java
@@ -1,6 +1,4 @@
-/**
- * 
- */
+
 package edu.uci.ics.asterix.om.types;
 
 /**
@@ -44,7 +42,9 @@
     CIRCLE(32),
     RECTANGLE(33),
     INTERVAL(34),
-    SYSTEM_NULL(35);
+    SYSTEM_NULL(35),
+    YEARMONTHDURATION(36),
+    DAYTIMEDURATION(37);
 
     private byte value;
 
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
index a70f15b..17d98d4 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/types/BuiltinType.java
@@ -431,6 +431,60 @@
         }
     };
 
+    public final static BuiltinType AYEARMONTHDURATION = new LowerCaseConstructorType() {
+
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public String getDisplayName() {
+            return "AYearMonthDuration";
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.YEARMONTHDURATION;
+        }
+
+        @Override
+        public String getTypeName() {
+            return "year-month-duration";
+        }
+
+        @Override
+        public JSONObject toJSON() throws JSONException {
+            JSONObject type = new JSONObject();
+            type.put("type", "AYearMonthDuration");
+            return type;
+        }
+    };
+
+    public final static BuiltinType ADAYTIMEDURATION = new LowerCaseConstructorType() {
+
+        private static final long serialVersionUID = 1L;
+
+        @Override
+        public String getDisplayName() {
+            return "ADayTimeDuration";
+        }
+
+        @Override
+        public ATypeTag getTypeTag() {
+            return ATypeTag.DAYTIMEDURATION;
+        }
+
+        @Override
+        public String getTypeName() {
+            return "day-time-duration";
+        }
+
+        @Override
+        public JSONObject toJSON() throws JSONException {
+            JSONObject type = new JSONObject();
+            type.put("type", "ADayTimeDuration");
+            return type;
+        }
+    };
+
     public final static BuiltinType AINTERVAL = new LowerCaseConstructorType() {
 
         private static final long serialVersionUID = 1L;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java
index 57ae544..6d3edbc 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/AsterixRuntimeUtil.java
@@ -31,38 +31,32 @@
  */
 public class AsterixRuntimeUtil {
 
-	public static Set<String> getNodeControllersOnIP(String ipAddress)
-			throws Exception {
-		Map<String, Set<String>> nodeControllerInfo = getNodeControllerMap();
-		Set<String> nodeControllersAtLocation = nodeControllerInfo
-				.get(ipAddress);
-		return nodeControllersAtLocation;
-	}
+    public static Set<String> getNodeControllersOnIP(String ipAddress) throws Exception {
+        Map<String, Set<String>> nodeControllerInfo = getNodeControllerMap();
+        Set<String> nodeControllersAtLocation = nodeControllerInfo.get(ipAddress);
+        return nodeControllersAtLocation;
+    }
 
-	public static List<String> getAllNodeControllers() throws Exception {
-		Collection<Set<String>> nodeControllersCollection = getNodeControllerMap()
-				.values();
-		List<String> nodeControllers = new ArrayList<String>();
-		for (Set<String> ncCollection : nodeControllersCollection) {
-			nodeControllers.addAll(ncCollection);
-		}
-		return nodeControllers;
-	}
+    public static List<String> getAllNodeControllers() throws Exception {
+        Collection<Set<String>> nodeControllersCollection = getNodeControllerMap().values();
+        List<String> nodeControllers = new ArrayList<String>();
+        for (Set<String> ncCollection : nodeControllersCollection) {
+            nodeControllers.addAll(ncCollection);
+        }
+        return nodeControllers;
+    }
 
-	public static Map<String, Set<String>> getNodeControllerMap()
-			throws Exception {
-		Map<String, Set<String>> map = new HashMap<String, Set<String>>();
-		AsterixContextInfo.INSTANCE.getCCApplicationContext().getCCContext()
-				.getIPAddressNodeMap(map);
-		return map;
-	}
+    public static Map<String, Set<String>> getNodeControllerMap() throws Exception {
+        Map<String, Set<String>> map = new HashMap<String, Set<String>>();
+        AsterixContextInfo.INSTANCE.getCCApplicationContext().getCCContext().getIPAddressNodeMap(map);
+        return map;
+    }
 
-	public static String getIPAddress(String hostname)
-			throws UnknownHostException {
-		String address = InetAddress.getByName(hostname).getHostAddress();
-		if (address.equals("127.0.1.1")) {
-			address = "127.0.0.1";
-		}
-		return address;
-	}
+    public static String getIPAddress(String hostname) throws UnknownHostException {
+        String address = InetAddress.getByName(hostname).getHostAddress();
+        if (address.equals("127.0.1.1")) {
+            address = "127.0.0.1";
+        }
+        return address;
+    }
 }
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
index 047c459..b4cd23c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/util/NonTaggedFormatUtil.java
@@ -95,12 +95,14 @@
             case INT32:
             case FLOAT:
             case DATE:
+            case YEARMONTHDURATION:
                 return 4;
             case TIME:
                 return 4;
             case INT64:
             case DOUBLE:
             case DATETIME:
+            case DAYTIMEDURATION:
                 return 8;
             case DURATION:
                 return 12;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java
index 703b792..d83f72d 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/IOMVisitor.java
@@ -7,6 +7,7 @@
 import edu.uci.ics.asterix.om.base.ACircle;
 import edu.uci.ics.asterix.om.base.ADate;
 import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.ADayTimeDuration;
 import edu.uci.ics.asterix.om.base.ADouble;
 import edu.uci.ics.asterix.om.base.ADuration;
 import edu.uci.ics.asterix.om.base.AFloat;
@@ -26,6 +27,7 @@
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.base.ATime;
 import edu.uci.ics.asterix.om.base.AUnorderedList;
+import edu.uci.ics.asterix.om.base.AYearMonthDuration;
 import edu.uci.ics.asterix.om.types.IAType;
 
 public interface IOMVisitor {
@@ -47,6 +49,10 @@
 
     public void visitADuration(ADuration obj) throws AsterixException;
 
+    public void visitAYearMonthDuration(AYearMonthDuration obj) throws AsterixException;
+
+    public void visitADayTimeDuration(ADayTimeDuration obj) throws AsterixException;
+
     public void visitAInterval(AInterval obj) throws AsterixException;
 
     public void visitADate(ADate obj) throws AsterixException;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java
index e7856f0..60c8700 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/visitors/OMPrintToStringVisitor.java
@@ -8,6 +8,7 @@
 import edu.uci.ics.asterix.om.base.ACollectionCursor;
 import edu.uci.ics.asterix.om.base.ADate;
 import edu.uci.ics.asterix.om.base.ADateTime;
+import edu.uci.ics.asterix.om.base.ADayTimeDuration;
 import edu.uci.ics.asterix.om.base.ADouble;
 import edu.uci.ics.asterix.om.base.ADuration;
 import edu.uci.ics.asterix.om.base.AFloat;
@@ -27,6 +28,7 @@
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.base.ATime;
 import edu.uci.ics.asterix.om.base.AUnorderedList;
+import edu.uci.ics.asterix.om.base.AYearMonthDuration;
 import edu.uci.ics.asterix.om.base.IACursor;
 import edu.uci.ics.asterix.om.base.IAObject;
 import edu.uci.ics.asterix.om.types.ARecordType;
@@ -252,4 +254,16 @@
         }
     }
 
+    @Override
+    public void visitAYearMonthDuration(AYearMonthDuration obj) throws AsterixException {
+        // TODO Auto-generated method stub
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public void visitADayTimeDuration(ADayTimeDuration obj) throws AsterixException {
+        // TODO Auto-generated method stub
+        throw new NotImplementedException();
+    }
+
 }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
index 3de05f8..232eace 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/AbstractScalarAggregateDescriptor.java
@@ -20,21 +20,23 @@
 
     @Override
     public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) throws AlgebricksException {
+
+        // The aggregate function will get a SingleFieldFrameTupleReference that points to the result of the ScanCollection.
+        // The list-item will always reside in the first field (column) of the SingleFieldFrameTupleReference.
+        ICopyEvaluatorFactory[] aggFuncArgs = new ICopyEvaluatorFactory[1];
+        aggFuncArgs[0] = new ColumnAccessEvalFactory(0);
+        // Create aggregate function from this scalar version.
+        FunctionIdentifier fid = AsterixBuiltinFunctions.getAggregateFunction(getIdentifier());
+        IFunctionManager mgr = FunctionManagerHolder.getFunctionManager();
+        IFunctionDescriptor fd = mgr.lookupFunction(fid);
+        AbstractAggregateFunctionDynamicDescriptor aggFuncDesc = (AbstractAggregateFunctionDynamicDescriptor) fd;
+        final ICopyAggregateFunctionFactory aggFuncFactory = aggFuncDesc.createAggregateFunctionFactory(aggFuncArgs);
+
         return new ICopyEvaluatorFactory() {
             private static final long serialVersionUID = 1L;
 
             @Override
             public ICopyEvaluator createEvaluator(IDataOutputProvider output) throws AlgebricksException {
-                // The aggregate function will get a SingleFieldFrameTupleReference that points to the result of the ScanCollection.
-                // The list-item will always reside in the first field (column) of the SingleFieldFrameTupleReference.
-                ICopyEvaluatorFactory[] aggFuncArgs = new ICopyEvaluatorFactory[1];
-                aggFuncArgs[0] = new ColumnAccessEvalFactory(0);
-                // Create aggregate function from this scalar version.
-                FunctionIdentifier fid = AsterixBuiltinFunctions.getAggregateFunction(getIdentifier());
-                IFunctionManager mgr = FunctionManagerHolder.getFunctionManager();
-                IFunctionDescriptor fd = mgr.lookupFunction(fid);
-                AbstractAggregateFunctionDynamicDescriptor aggFuncDesc = (AbstractAggregateFunctionDynamicDescriptor) fd;
-                ICopyAggregateFunctionFactory aggFuncFactory = aggFuncDesc.createAggregateFunctionFactory(aggFuncArgs);
                 // Use ScanCollection to iterate over list items.
                 ScanCollectionUnnestingFunctionFactory scanCollectionFactory = new ScanCollectionUnnestingFunctionFactory(
                         args[0]);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
index ae2a485..fa8f930 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarAvgAggregateDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.asterix.runtime.aggregates.scalar;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -8,7 +8,7 @@
 public class ScalarAvgAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg", 1);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SCALAR_AVG;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ScalarAvgAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
index 6f3baa6..1f362c9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarCountAggregateDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.asterix.runtime.aggregates.scalar;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -8,7 +8,7 @@
 public class ScalarCountAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "count", 1);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SCALAR_COUNT;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ScalarCountAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
index a71eb3c..26cb74e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMaxAggregateDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.asterix.runtime.aggregates.scalar;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -8,7 +8,7 @@
 public class ScalarMaxAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "max", 1);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SCALAR_MAX;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ScalarMaxAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
index 4beae60..4d2bbd4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarMinAggregateDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.asterix.runtime.aggregates.scalar;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -8,7 +8,7 @@
 public class ScalarMinAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "min", 1);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SCALAR_MIN;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ScalarMinAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
index f3d9d1a..c82e3e2 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/scalar/ScalarSumAggregateDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.asterix.runtime.aggregates.scalar;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -8,7 +8,7 @@
 public class ScalarSumAggregateDescriptor extends AbstractScalarAggregateDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "sum", 1);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.SCALAR_SUM;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new ScalarSumAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
index 6badf0f..c3ee985 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/serializable/std/SerializableLocalSumAggregateDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.asterix.runtime.aggregates.serializable.std;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.runtime.aggregates.base.AbstractSerializableAggregateFunctionDynamicDescriptor;
@@ -13,8 +13,7 @@
 public class SerializableLocalSumAggregateDescriptor extends AbstractSerializableAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "local-sum-serial", 1);
+    private final static FunctionIdentifier FID = AsterixBuiltinFunctions.SERIAL_LOCAL_SUM;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new SerializableLocalSumAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateFunction.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateFunction.java
index e4d015b..82c1e8a 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateFunction.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/CountAggregateFunction.java
@@ -6,7 +6,6 @@
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AMutableInt32;
-import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.EnumDeserializer;
@@ -20,19 +19,15 @@
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
 /**
- * count(NULL) returns NULL.
+ * COUNT returns the number of items in the given list. Note that COUNT(NULL) is not allowed.
  */
 public class CountAggregateFunction implements ICopyAggregateFunction {
     private AMutableInt32 result = new AMutableInt32(-1);
     @SuppressWarnings("unchecked")
     private ISerializerDeserializer<AInt32> int32Serde = AqlSerializerDeserializerProvider.INSTANCE
             .getSerializerDeserializer(BuiltinType.AINT32);
-    @SuppressWarnings("unchecked")
-    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
-            .getSerializerDeserializer(BuiltinType.ANULL);
     private ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
     private ICopyEvaluator eval;
-    private boolean metNull;
     private int cnt;
     private DataOutput out;
 
@@ -44,7 +39,6 @@
     @Override
     public void init() {
         cnt = 0;
-        metNull = false;
     }
 
     @Override
@@ -53,9 +47,7 @@
         eval.evaluate(tuple);
         ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(inputVal.getByteArray()[0]);
         // Ignore SYSTEM_NULL.
-        if (typeTag == ATypeTag.NULL) {
-            metNull = true;
-        } else {
+        if (typeTag != ATypeTag.SYSTEM_NULL) {
             cnt++;
         }
     }
@@ -63,12 +55,8 @@
     @Override
     public void finish() throws AlgebricksException {
         try {
-            if (metNull) {
-                nullSerde.serialize(ANull.NULL, out);
-            } else {
-                result.setValue(cnt);
-                int32Serde.serialize(result, out);
-            }
+            result.setValue(cnt);
+            int32Serde.serialize(result, out);
         } catch (IOException e) {
             throw new AlgebricksException(e);
         }
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
index 59287d5..e7346a8 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMaxAggregateDescriptor.java
@@ -1,6 +1,6 @@
 package edu.uci.ics.asterix.runtime.aggregates.std;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
@@ -14,8 +14,8 @@
 public class LocalMaxAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-max",
-            1);
+    private final static FunctionIdentifier FID = AsterixBuiltinFunctions.LOCAL_MAX;
+
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LocalMaxAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
index ca32e2f..9e6747d 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalMinAggregateDescriptor.java
@@ -1,6 +1,7 @@
 package edu.uci.ics.asterix.runtime.aggregates.std;
 
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
@@ -14,8 +15,7 @@
 public class LocalMinAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-min",
-            1);
+    private final static FunctionIdentifier FID = AsterixBuiltinFunctions.LOCAL_MIN;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LocalMinAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
index c133e09..01f6d28 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/std/LocalSumAggregateDescriptor.java
@@ -1,6 +1,7 @@
 package edu.uci.ics.asterix.runtime.aggregates.std;
 
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
@@ -14,8 +15,7 @@
 public class LocalSumAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    private final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sum",
-            1);
+    private final static FunctionIdentifier FID = AsterixBuiltinFunctions.LOCAL_SUM;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new LocalSumAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
index 587a113..28d3c5e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/aggregates/stream/EmptyStreamAggregateDescriptor.java
@@ -5,6 +5,7 @@
 import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.ABoolean;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.BuiltinType;
@@ -23,8 +24,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "empty-stream", 0);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.EMPTY_STREAM;
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         public IFunctionDescriptor createFunctionDescriptor() {
             return new EmptyStreamAggregateDescriptor();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
index 577a1fb..fe21cbb 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleCenterAccessor.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.asterix.om.base.AMutablePoint;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.APoint;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -43,7 +44,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    private static final FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-center", 1);
+    private static final FunctionIdentifier FID = AsterixBuiltinFunctions.GET_CIRCLE_CENTER_ACCESSOR;
     private static final byte SER_CICLE_TAG = ATypeTag.CIRCLE.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
index ffaae9e..0669f4c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/CircleRadiusAccessor.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.asterix.om.base.ADouble;
 import edu.uci.ics.asterix.om.base.AMutableDouble;
 import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -42,7 +43,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    private static final FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-radius", 1);
+    private static final FunctionIdentifier FID = AsterixBuiltinFunctions.GET_CIRCLE_RADIUS_ACCESSOR;
     private static final byte SER_CICLE_TAG = ATypeTag.CIRCLE.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
index d7f87df..8dbdf75 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/LineRectanglePolygonAccessor.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.asterix.om.base.AMutablePoint;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.APoint;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
@@ -49,7 +50,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    private static final FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-points", 1);
+    private static final FunctionIdentifier FID = AsterixBuiltinFunctions.GET_POINTS_LINE_RECTANGLE_POLYGON_ACCESSOR;
     private static final byte SER_LINE_TAG = ATypeTag.LINE.serialize();
     private static final byte SER_RECTANGLE_TAG = ATypeTag.RECTANGLE.serialize();
     private static final byte SER_POLYGON_TAG = ATypeTag.POLYGON.serialize();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
index 7fcfd20..c42a1b4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointXCoordinateAccessor.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.asterix.om.base.ADouble;
 import edu.uci.ics.asterix.om.base.AMutableDouble;
 import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -43,7 +44,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    private static final FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-x", 1);
+    private static final FunctionIdentifier FID = AsterixBuiltinFunctions.GET_POINT_X_COORDINATE_ACCESSOR;
     private static final byte SER_POINT_TAG = ATypeTag.POINT.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
index 1c47efa..02c6b30 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/accessors/PointYCoordinateAccessor.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.asterix.om.base.ADouble;
 import edu.uci.ics.asterix.om.base.AMutableDouble;
 import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -43,7 +44,7 @@
 
     private static final long serialVersionUID = 1L;
 
-    private static final FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "get-y", 1);
+    private static final FunctionIdentifier FID = AsterixBuiltinFunctions.GET_POINT_Y_COORDINATE_ACCESSOR;
     private static final byte SER_POINT_TAG = ATypeTag.POINT.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java
index 6f834a7..127d4a6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/common/FunctionManagerImpl.java
@@ -37,7 +37,11 @@
     @Override
     public synchronized IFunctionDescriptor lookupFunction(FunctionIdentifier fid) throws AlgebricksException {
         Pair<FunctionIdentifier, Integer> key = new Pair<FunctionIdentifier, Integer>(fid, fid.getArity());
-        return functions.get(key).createFunctionDescriptor();
+        IFunctionDescriptorFactory factory = functions.get(key);
+        if (factory == null) {
+            throw new AlgebricksException("Inappropriate use of function " + "'" + fid.getName() + "'");
+        }
+        return factory.createFunctionDescriptor();
     }
 
     @Override
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
index 1565cb9..06a7823 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/AbstractComparisonEvaluator.java
@@ -2,8 +2,14 @@
 
 import java.io.DataOutput;
 
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ADateOrTimeAscBinaryComparatorFactory;
-import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ADateTimeAscBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ACirclePartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ADurationPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.AIntervalPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ALinePartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.APoint3DPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.APointPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.APolygonPartialBinaryComparatorFactory;
+import edu.uci.ics.asterix.dataflow.data.nontagged.comparators.ARectanglePartialBinaryComparatorFactory;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
@@ -49,9 +55,21 @@
             .getSerializerDeserializer(BuiltinType.ANULL);
     protected IBinaryComparator strBinaryComp = AqlBinaryComparatorFactoryProvider.UTF8STRING_POINTABLE_INSTANCE
             .createBinaryComparator();
-    protected IBinaryComparator dateTimeBinaryComp = ADateTimeAscBinaryComparatorFactory.INSTANCE
+    protected IBinaryComparator circleBinaryComp = ACirclePartialBinaryComparatorFactory.INSTANCE
             .createBinaryComparator();
-    protected IBinaryComparator dateOrTimeBinaryComp = ADateOrTimeAscBinaryComparatorFactory.INSTANCE
+    protected IBinaryComparator durationBinaryComp = ADurationPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    protected IBinaryComparator intervalBinaryComp = AIntervalPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    protected IBinaryComparator lineBinaryComparator = ALinePartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    protected IBinaryComparator pointBinaryComparator = APointPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    protected IBinaryComparator point3DBinaryComparator = APoint3DPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    protected IBinaryComparator polygonBinaryComparator = APolygonPartialBinaryComparatorFactory.INSTANCE
+            .createBinaryComparator();
+    protected IBinaryComparator rectangleBinaryComparator = ARectanglePartialBinaryComparatorFactory.INSTANCE
             .createBinaryComparator();
 
     public AbstractComparisonEvaluator(DataOutput out, ICopyEvaluatorFactory evalLeftFactory,
@@ -68,6 +86,25 @@
         evalRight.evaluate(tuple);
     }
 
+    protected void checkComparable() throws AlgebricksException {
+        if (outLeft.getLength() != 0) {
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(outLeft.getByteArray()[0]);
+            switch (typeTag) {
+                case DURATION:
+                case INTERVAL:
+                case LINE:
+                case POINT:
+                case POINT3D:
+                case POLYGON:
+                case CIRCLE:
+                case RECTANGLE:
+                    throw new AlgebricksException("Inequality comparison for " + typeTag + " is not defined.");
+                default:
+                    return;
+            }
+        }
+    }
+
     protected ComparisonResult compareResults() throws AlgebricksException {
         boolean isLeftNull = false;
         boolean isRightNull = false;
@@ -119,50 +156,73 @@
             case BOOLEAN: {
                 return compareBooleanWithArg(typeTag2);
             }
-            case DATETIME: {
-                return compareDateTimeWithArg(typeTag2);
-            }
-            case DATE:
-            case TIME: {
-                return compareDateOrTimeWithArg(typeTag2);
-            }
+
             default: {
-                throw new AlgebricksException("Comparison is undefined between types " + typeTag1 + " and " + typeTag2
-                        + " .");
+                return compareStrongTypedWithArg(typeTag1, typeTag2);
             }
         }
     }
 
-    private ComparisonResult compareDateOrTimeWithArg(ATypeTag typeTag2) throws AlgebricksException {
-        if (typeTag2 == ATypeTag.NULL) {
-            return ComparisonResult.GREATER_THAN;
-        } else if (typeTag2 == ATypeTag.DATE || typeTag2 == ATypeTag.TIME) {
-            int result = dateOrTimeBinaryComp.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
-                    outRight.getByteArray(), 1, outRight.getLength() - 1);
-            if (result == 0)
-                return ComparisonResult.EQUAL;
-            else if (result < 0)
-                return ComparisonResult.LESS_THAN;
-            else
-                return ComparisonResult.GREATER_THAN;
+    private ComparisonResult compareStrongTypedWithArg(ATypeTag expectedTypeTag, ATypeTag actualTypeTag)
+            throws AlgebricksException {
+        if (expectedTypeTag != actualTypeTag) {
+            throw new AlgebricksException("Comparison is undefined between " + expectedTypeTag + " and "
+                    + actualTypeTag + ".");
         }
-        throw new AlgebricksException("Comparison is undefined between types Date/Time and " + typeTag2 + " .");
-    }
-
-    private ComparisonResult compareDateTimeWithArg(ATypeTag typeTag2) throws AlgebricksException {
-        if (typeTag2 == ATypeTag.NULL) {
-            return ComparisonResult.GREATER_THAN;
-        } else if (typeTag2 == ATypeTag.DATETIME) {
-            int result = dateTimeBinaryComp.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
-                    outRight.getByteArray(), 1, outRight.getLength() - 1);
-            if (result == 0)
-                return ComparisonResult.EQUAL;
-            else if (result < 0)
-                return ComparisonResult.LESS_THAN;
-            else
-                return ComparisonResult.GREATER_THAN;
+        int result = 0;
+        switch (actualTypeTag) {
+            case YEARMONTHDURATION:
+            case TIME:
+            case DATE:
+                result = Integer.compare(AInt32SerializerDeserializer.getInt(outLeft.getByteArray(), 1),
+                        AInt32SerializerDeserializer.getInt(outRight.getByteArray(), 1));
+                break;
+            case DAYTIMEDURATION:
+            case DATETIME:
+                result = Long.compare(AInt64SerializerDeserializer.getLong(outLeft.getByteArray(), 1),
+                        AInt64SerializerDeserializer.getLong(outRight.getByteArray(), 1));
+                break;
+            case CIRCLE:
+                result = circleBinaryComp.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            case LINE:
+                result = lineBinaryComparator.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            case POINT:
+                result = pointBinaryComparator.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            case POINT3D:
+                result = point3DBinaryComparator.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            case POLYGON:
+                result = polygonBinaryComparator.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            case DURATION:
+                result = durationBinaryComp.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            case INTERVAL:
+                result = intervalBinaryComp.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            case RECTANGLE:
+                result = rectangleBinaryComparator.compare(outLeft.getByteArray(), 1, outLeft.getLength() - 1,
+                        outRight.getByteArray(), 1, outRight.getLength() - 1);
+                break;
+            default:
+                throw new AlgebricksException("Comparison for " + actualTypeTag + " is not supported.");
         }
-        throw new AlgebricksException("Comparison is undefined between types Datetime and " + typeTag2 + " .");
+        if (result == 0)
+            return ComparisonResult.EQUAL;
+        else if (result < 0)
+            return ComparisonResult.LESS_THAN;
+        else
+            return ComparisonResult.GREATER_THAN;
     }
 
     private ComparisonResult compareBooleanWithArg(ATypeTag typeTag2) throws AlgebricksException {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java
index ed77276..8d57c2e 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/comparisons/ComparisonEvalFactory.java
@@ -121,6 +121,7 @@
         @Override
         public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
             evalInputs(tuple);
+            checkComparable();
             ComparisonResult r = compareResults();
             if (r == ComparisonResult.UNKNOWN) {
                 try {
@@ -150,6 +151,7 @@
         @Override
         public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
             evalInputs(tuple);
+            checkComparable();
             ComparisonResult r = compareResults();
             if (r == ComparisonResult.UNKNOWN) {
                 try {
@@ -178,6 +180,7 @@
         @Override
         public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
             evalInputs(tuple);
+            checkComparable();
             ComparisonResult r = compareResults();
             if (r == ComparisonResult.UNKNOWN) {
                 try {
@@ -207,6 +210,7 @@
         @Override
         public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
             evalInputs(tuple);
+            checkComparable();
             ComparisonResult r = compareResults();
             if (r == ComparisonResult.UNKNOWN) {
                 try {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
new file mode 100644
index 0000000..0d43c3d
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADayTimeDurationConstructorDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.constructors;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.ADayTimeDuration;
+import edu.uci.ics.asterix.om.base.AMutableDayTimeDuration;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class ADayTimeDurationConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new ADayTimeDurationConstructorDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+
+                    private ArrayBackedValueStorage outInput = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(outInput);
+                    private String errorMessage = "This can not be an instance of day-time-duration";
+                    private AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ADAYTIMEDURATION);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            outInput.reset();
+                            eval.evaluate(tuple);
+                            byte[] serString = outInput.getByteArray();
+
+                            if (serString[0] == SER_STRING_TYPE_TAG) {
+
+                                int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
+
+                                ADurationParserFactory.parseDuration(serString, 3, stringLength, aDayTimeDuration,
+                                        ADurationParseOption.DAY_TIME);
+
+                                dayTimeDurationSerde.serialize(aDayTimeDuration, out);
+                            } else if (serString[0] == SER_NULL_TYPE_TAG) {
+                                nullSerde.serialize(ANull.NULL, out);
+                            } else {
+                                throw new AlgebricksException(errorMessage);
+                            }
+                        } catch (Exception e1) {
+                            throw new AlgebricksException(e1);
+                        }
+                    }
+
+                };
+            }
+        };
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+     */
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.DAY_TIME_DURATION_CONSTRUCTOR;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
index a84ae1d..669c4f7 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/ADurationConstructorDescriptor.java
@@ -21,6 +21,7 @@
 import edu.uci.ics.asterix.om.base.AMutableDuration;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -82,7 +83,8 @@
 
                                 int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
 
-                                ADurationParserFactory.parseDuration(serString, 3, stringLength, aDuration);
+                                ADurationParserFactory.parseDuration(serString, 3, stringLength, aDuration,
+                                        ADurationParseOption.All);
 
                                 durationSerde.serialize(aDuration, out);
                             } else if (serString[0] == SER_NULL_TYPE_TAG) {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateConstructorDescriptor.java
index 8daf9a6..4c8c60f 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateConstructorDescriptor.java
@@ -17,7 +17,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.AInterval;
@@ -25,6 +24,7 @@
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.temporal.ADateParserFactory;
 import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -43,8 +43,7 @@
 public class AIntervalFromDateConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-from-date", 2);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.INTERVAL_CONSTRUCTOR_DATE;
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_DATE_TYPE_TAG = ATypeTag.DATE.serialize();
@@ -123,7 +122,7 @@
                                         + ": expects NULL/STRING/DATE for the second argument, but got "
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0]));
                             }
-                            
+
                             if (intervalEnd < intervalStart) {
                                 throw new AlgebricksException(FID.getName()
                                         + ": interval end must not be less than the interval start.");
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java
index 78d2e4a..42479f4 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromDateTimeConstructorDescriptor.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.temporal.ADateParserFactory;
 import edu.uci.ics.asterix.om.base.temporal.ATimeParserFactory;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -43,8 +44,7 @@
 public class AIntervalFromDateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-from-datetime", 2);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.INTERVAL_CONSTRUCTOR_DATETIME;
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_DATETIME_TYPE_TAG = ATypeTag.DATETIME.serialize();
@@ -145,7 +145,7 @@
                                         + ": expects NULL/STRING/DATETIME for the second argument, but got "
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut1.getByteArray()[0]));
                             }
-                            
+
                             if (intervalEnd < intervalStart) {
                                 throw new AlgebricksException(FID.getName()
                                         + ": interval end must not be less than the interval start.");
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromTimeConstructorDescriptor.java
index 8816160..eb69037 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalFromTimeConstructorDescriptor.java
@@ -25,6 +25,7 @@
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.temporal.ATimeParserFactory;
 import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -43,8 +44,7 @@
 public class AIntervalFromTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-from-time", 2);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.INTERVAL_CONSTRUCTOR_TIME;
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_TIME_TYPE_TAG = ATypeTag.TIME.serialize();
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
index 7ea8e90..2802f29 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateConstructorDescriptor.java
@@ -17,7 +17,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -27,8 +26,10 @@
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.temporal.ADateParserFactory;
 import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
 import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -47,8 +48,7 @@
 public class AIntervalStartFromDateConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-start-from-date", 2);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.INTERVAL_CONSTRUCTOR_START_FROM_DATE;
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_DATE_TYPE_TAG = ATypeTag.DATE.serialize();
@@ -128,8 +128,8 @@
                                 int stringLength = (argOut1.getByteArray()[1] & 0xff << 8)
                                         + (argOut1.getByteArray()[2] & 0xff << 0);
 
-                                ADurationParserFactory
-                                        .parseDuration(argOut1.getByteArray(), 3, stringLength, aDuration);
+                                ADurationParserFactory.parseDuration(argOut1.getByteArray(), 3, stringLength,
+                                        aDuration, ADurationParseOption.All);
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
                                         aDuration.getMonths(), aDuration.getMilliseconds());
                             } else {
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
index f16dccb..0cab83b 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromDateTimeConstructorDescriptor.java
@@ -17,7 +17,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -29,6 +28,8 @@
 import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
 import edu.uci.ics.asterix.om.base.temporal.ATimeParserFactory;
 import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -47,8 +48,7 @@
 public class AIntervalStartFromDateTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-start-from-datetime", 2);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.INTERVAL_CONSTRUCTOR_START_FROM_DATETIME;
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_DATETIME_TYPE_TAG = ATypeTag.DATETIME.serialize();
@@ -141,8 +141,8 @@
                                 int stringLength = (argOut1.getByteArray()[1] & 0xff << 8)
                                         + (argOut1.getByteArray()[2] & 0xff << 0);
 
-                                ADurationParserFactory
-                                        .parseDuration(argOut1.getByteArray(), 3, stringLength, aDuration);
+                                ADurationParserFactory.parseDuration(argOut1.getByteArray(), 3, stringLength,
+                                        aDuration, ADurationParseOption.All);
 
                                 intervalEnd = DurationArithmeticOperations.addDuration(intervalStart,
                                         aDuration.getMonths(), aDuration.getMilliseconds());
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
index 35bfe17..40db774 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AIntervalStartFromTimeConstructorDescriptor.java
@@ -17,7 +17,6 @@
 import java.io.DataOutput;
 import java.io.IOException;
 
-import edu.uci.ics.asterix.common.functions.FunctionConstants;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
@@ -29,6 +28,8 @@
 import edu.uci.ics.asterix.om.base.temporal.ATimeParserFactory;
 import edu.uci.ics.asterix.om.base.temporal.DurationArithmeticOperations;
 import edu.uci.ics.asterix.om.base.temporal.GregorianCalendarSystem;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.types.ATypeTag;
@@ -47,8 +48,7 @@
 public class AIntervalStartFromTimeConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private static final long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
-            "interval-start-from-time", 2);
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.INTERVAL_CONSTRUCTOR_START_FROM_TIME;
     private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
     private final static byte SER_TIME_TYPE_TAG = ATypeTag.TIME.serialize();
@@ -138,8 +138,8 @@
                                 int stringLength = (argOut1.getByteArray()[1] & 0xff << 8)
                                         + (argOut1.getByteArray()[2] & 0xff << 0);
 
-                                ADurationParserFactory
-                                        .parseDuration(argOut1.getByteArray(), 3, stringLength, aDuration);
+                                ADurationParserFactory.parseDuration(argOut1.getByteArray(), 3, stringLength,
+                                        aDuration, ADurationParseOption.All);
 
                                 if (aDuration.getMonths() != 0) {
                                     throw new AlgebricksException(FID.getName()
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
new file mode 100644
index 0000000..bc6099c
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/constructors/AYearMonthDurationConstructorDescriptor.java
@@ -0,0 +1,114 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.runtime.evaluators.constructors;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
+import edu.uci.ics.asterix.om.base.AMutableYearMonthDuration;
+import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.AYearMonthDuration;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory;
+import edu.uci.ics.asterix.om.base.temporal.ADurationParserFactory.ADurationParseOption;
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.BuiltinType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.api.dataflow.value.ISerializerDeserializer;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class AYearMonthDurationConstructorDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    private static final long serialVersionUID = 1L;
+    private final static byte SER_STRING_TYPE_TAG = ATypeTag.STRING.serialize();
+    private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new AYearMonthDurationConstructorDescriptor();
+        }
+    };
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                return new ICopyEvaluator() {
+
+                    private DataOutput out = output.getDataOutput();
+
+                    private ArrayBackedValueStorage outInput = new ArrayBackedValueStorage();
+                    private ICopyEvaluator eval = args[0].createEvaluator(outInput);
+                    private String errorMessage = "This can not be an instance of year-month-duration";
+                    private AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.AYEARMONTHDURATION);
+                    @SuppressWarnings("unchecked")
+                    private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ANULL);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            outInput.reset();
+                            eval.evaluate(tuple);
+                            byte[] serString = outInput.getByteArray();
+
+                            if (serString[0] == SER_STRING_TYPE_TAG) {
+
+                                int stringLength = (serString[1] & 0xff << 8) + (serString[2] & 0xff << 0);
+
+                                ADurationParserFactory.parseDuration(serString, 3, stringLength, aYearMonthDuration,
+                                        ADurationParseOption.YEAR_MONTH);
+
+                                yearMonthDurationSerde.serialize(aYearMonthDuration, out);
+                            } else if (serString[0] == SER_NULL_TYPE_TAG) {
+                                nullSerde.serialize(ANull.NULL, out);
+                            } else {
+                                throw new AlgebricksException(errorMessage);
+                            }
+                        } catch (Exception e1) {
+                            throw new AlgebricksException(e1);
+                        }
+                    }
+
+                };
+            }
+        };
+    }
+
+    /* (non-Javadoc)
+     * @see edu.uci.ics.asterix.om.functions.AbstractFunctionDescriptor#getIdentifier()
+     */
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.YEAR_MONTH_DURATION_CONSTRUCTOR;
+    }
+
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java
new file mode 100644
index 0000000..c1817e5
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java
@@ -0,0 +1,105 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.pointables.PointableAllocator;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.pointables.cast.ACastVisitor;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * The runtime function for casting a list(unordered list or ordered list)
+ * 
+ * @author yingyib
+ *
+ */
+public class CastListDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new CastListDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private AbstractCollectionType reqType;
+    private AbstractCollectionType inputType;
+
+    public void reset(AbstractCollectionType reqType, AbstractCollectionType inputType) {
+        this.reqType = reqType;
+        this.inputType = inputType;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.CAST_LIST;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        final ICopyEvaluatorFactory recordEvalFactory = args[0];
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                final DataOutput out = output.getDataOutput();
+                final ArrayBackedValueStorage recordBuffer = new ArrayBackedValueStorage();
+                final ICopyEvaluator recEvaluator = recordEvalFactory.createEvaluator(recordBuffer);
+
+                return new ICopyEvaluator() {
+                    // pointable allocator
+                    private PointableAllocator allocator = new PointableAllocator();
+                    final IVisitablePointable recAccessor = allocator.allocateListValue(inputType);
+                    final IVisitablePointable resultAccessor = allocator.allocateListValue(reqType);
+                    final ACastVisitor castVisitor = new ACastVisitor();
+                    final Triple<IVisitablePointable, IAType, Boolean> arg = new Triple<IVisitablePointable, IAType, Boolean>(
+                            resultAccessor, reqType, Boolean.FALSE);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            recordBuffer.reset();
+                            recEvaluator.evaluate(tuple);
+                            recAccessor.set(recordBuffer);
+                            recAccessor.accept(castVisitor, arg);
+                            out.write(resultAccessor.getByteArray(), resultAccessor.getStartOffset(),
+                                    resultAccessor.getLength());
+                        } catch (Exception ioe) {
+                            throw new AlgebricksException(ioe);
+                        }
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java
index b41347d..c30fb6d 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/GetItemDescriptor.java
@@ -109,7 +109,7 @@
                         }
                         if (itemIndex < 0)
                             throw new AlgebricksException(AsterixBuiltinFunctions.GET_ITEM.getName()
-                                    + ": item index can be negative!");
+                                    + ": item index cannot be negative!");
 
                         itemTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(serOrderedList[1]);
                         if (itemTag == ATypeTag.ANY)
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
similarity index 84%
copy from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java
copy to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
index 97fa94a..17c6ba9 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/GetDayTimeDurationDescriptor.java
@@ -18,8 +18,8 @@
 
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.AInt64;
-import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.ADayTimeDuration;
+import edu.uci.ics.asterix.om.base.AMutableDayTimeDuration;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
@@ -38,10 +38,10 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class MillisecondsOfDayTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class GetDayTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private final static long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.MILLISECONDS_OF_DAY_TIME_DURATION;
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.GET_DAY_TIME_DURATION;
 
     // allowed input types
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
@@ -51,7 +51,7 @@
 
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new MillisecondsOfDayTimeDurationDescriptor();
+            return new GetDayTimeDurationDescriptor();
         }
     };
 
@@ -74,10 +74,10 @@
                     private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.ANULL);
                     @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.AINT64);
+                    private ISerializerDeserializer<ADayTimeDuration> dayTimeDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.ADAYTIMEDURATION);
 
-                    AMutableInt64 aInt64 = new AMutableInt64(0);
+                    AMutableDayTimeDuration aDayTimeDuration = new AMutableDayTimeDuration(0);
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -95,9 +95,10 @@
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
                             }
 
-                            aInt64.setValue(ADurationSerializerDeserializer.getDayTime(argOut0.getByteArray(), 1));
+                            aDayTimeDuration.setMilliseconds(ADurationSerializerDeserializer.getDayTime(
+                                    argOut0.getByteArray(), 1));
 
-                            int64Serde.serialize(aInt64, out);
+                            dayTimeDurationSerde.serialize(aDayTimeDuration, out);
 
                         } catch (HyracksDataException hex) {
                             throw new AlgebricksException(hex);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
similarity index 84%
copy from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java
copy to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
index 97fa94a..b7b4278 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/GetYearMonthDurationDescriptor.java
@@ -18,9 +18,9 @@
 
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
-import edu.uci.ics.asterix.om.base.AInt64;
-import edu.uci.ics.asterix.om.base.AMutableInt64;
+import edu.uci.ics.asterix.om.base.AMutableYearMonthDuration;
 import edu.uci.ics.asterix.om.base.ANull;
+import edu.uci.ics.asterix.om.base.AYearMonthDuration;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
@@ -38,10 +38,10 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class MillisecondsOfDayTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class GetYearMonthDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
-    private final static long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.MILLISECONDS_OF_DAY_TIME_DURATION;
+    private static final long serialVersionUID = 1L;
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.GET_YEAR_MONTH_DURATION;
 
     // allowed input types
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
@@ -51,7 +51,7 @@
 
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new MillisecondsOfDayTimeDurationDescriptor();
+            return new GetYearMonthDurationDescriptor();
         }
     };
 
@@ -74,10 +74,10 @@
                     private ISerializerDeserializer<ANull> nullSerde = AqlSerializerDeserializerProvider.INSTANCE
                             .getSerializerDeserializer(BuiltinType.ANULL);
                     @SuppressWarnings("unchecked")
-                    private ISerializerDeserializer<AInt64> int64Serde = AqlSerializerDeserializerProvider.INSTANCE
-                            .getSerializerDeserializer(BuiltinType.AINT64);
+                    private ISerializerDeserializer<AYearMonthDuration> yearMonthDurationSerde = AqlSerializerDeserializerProvider.INSTANCE
+                            .getSerializerDeserializer(BuiltinType.AYEARMONTHDURATION);
 
-                    AMutableInt64 aInt64 = new AMutableInt64(0);
+                    AMutableYearMonthDuration aYearMonthDuration = new AMutableYearMonthDuration(0);
 
                     @Override
                     public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
@@ -95,9 +95,10 @@
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
                             }
 
-                            aInt64.setValue(ADurationSerializerDeserializer.getDayTime(argOut0.getByteArray(), 1));
+                            aYearMonthDuration.setMonths(ADurationSerializerDeserializer.getYearMonth(
+                                    argOut0.getByteArray(), 1));
 
-                            int64Serde.serialize(aInt64, out);
+                            yearMonthDurationSerde.serialize(aYearMonthDuration, out);
 
                         } catch (HyracksDataException hex) {
                             throw new AlgebricksException(hex);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
similarity index 88%
rename from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java
rename to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
index 97fa94a..14e16c6 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsOfDayTimeDurationDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MillisecondsFromDayTimeDurationDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.io.DataOutput;
 
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.AInt64;
 import edu.uci.ics.asterix.om.base.AMutableInt64;
@@ -38,20 +38,20 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class MillisecondsOfDayTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class MillisecondsFromDayTimeDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private final static long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.MILLISECONDS_OF_DAY_TIME_DURATION;
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.MILLISECONDS_FROM_DAY_TIME_DURATION;
 
     // allowed input types
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
-    private final static byte SER_DURATION_TYPE_TAG = ATypeTag.DURATION.serialize();
+    private final static byte SER_DAY_TIME_DURATION_TYPE_TAG = ATypeTag.DAYTIMEDURATION.serialize();
 
     public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
 
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new MillisecondsOfDayTimeDurationDescriptor();
+            return new MillisecondsFromDayTimeDurationDescriptor();
         }
     };
 
@@ -90,12 +90,12 @@
                                 return;
                             }
 
-                            if (argOut0.getByteArray()[0] != SER_DURATION_TYPE_TAG) {
-                                throw new AlgebricksException(FID.getName() + ": expects NULL/DURATION, but got "
+                            if (argOut0.getByteArray()[0] != SER_DAY_TIME_DURATION_TYPE_TAG) {
+                                throw new AlgebricksException(FID.getName() + ": expects NULL/DAY-TIME-DURATION, but got "
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
                             }
 
-                            aInt64.setValue(ADurationSerializerDeserializer.getDayTime(argOut0.getByteArray(), 1));
+                            aInt64.setValue(ADayTimeDurationSerializerDeserializer.getDayTime(argOut0.getByteArray(), 1));
 
                             int64Serde.serialize(aInt64, out);
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsOfYearMonthDurationDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
similarity index 87%
rename from asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsOfYearMonthDurationDescriptor.java
rename to asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
index 55173a5..9a66d45 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsOfYearMonthDurationDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/temporal/MonthsFromYearMonthDurationDescriptor.java
@@ -16,7 +16,7 @@
 
 import java.io.DataOutput;
 
-import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationerializerDeserializer;
 import edu.uci.ics.asterix.formats.nontagged.AqlSerializerDeserializerProvider;
 import edu.uci.ics.asterix.om.base.AInt32;
 import edu.uci.ics.asterix.om.base.AMutableInt32;
@@ -38,20 +38,20 @@
 import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-public class MonthsOfYearMonthDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+public class MonthsFromYearMonthDurationDescriptor extends AbstractScalarFunctionDynamicDescriptor {
 
     private final static long serialVersionUID = 1L;
-    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.MONTHS_OF_YEAR_MONTH_DURATION;
+    public final static FunctionIdentifier FID = AsterixBuiltinFunctions.MONTHS_FROM_YEAR_MONTH_DURATION;
 
     // allowed input types
     private final static byte SER_NULL_TYPE_TAG = ATypeTag.NULL.serialize();
-    private final static byte SER_DURATION_TYPE_TAG = ATypeTag.DURATION.serialize();
+    private final static byte SER_YEARMONTHDURATION_TYPE_TAG = ATypeTag.YEARMONTHDURATION.serialize();
 
     public final static IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
 
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
-            return new MonthsOfYearMonthDurationDescriptor();
+            return new MonthsFromYearMonthDurationDescriptor();
         }
     };
 
@@ -90,12 +90,14 @@
                                 return;
                             }
 
-                            if (argOut0.getByteArray()[0] != SER_DURATION_TYPE_TAG) {
-                                throw new AlgebricksException(FID.getName() + ": expects NULL/DURATION, but got "
+                            if (argOut0.getByteArray()[0] != SER_YEARMONTHDURATION_TYPE_TAG) {
+                                throw new AlgebricksException(FID.getName()
+                                        + ": expects NULL/YEAR-MONTH-DURATION, but got "
                                         + EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(argOut0.getByteArray()[0]));
                             }
 
-                            aInt32.setValue(ADurationSerializerDeserializer.getYearMonth(argOut0.getByteArray(), 1));
+                            aInt32.setValue(AYearMonthDurationerializerDeserializer.getYearMonth(
+                                    argOut0.getByteArray(), 1));
 
                             int32Serde.serialize(aInt32, out);
 
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 84142be..2c81b7c 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -35,12 +35,14 @@
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
 import edu.uci.ics.asterix.om.functions.IFunctionManager;
+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.AOrderedListType;
 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.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.runtime.aggregates.collections.ListifyAggregateDescriptor;
@@ -67,6 +69,11 @@
 import edu.uci.ics.asterix.runtime.aggregates.std.SumAggregateDescriptor;
 import edu.uci.ics.asterix.runtime.aggregates.stream.EmptyStreamAggregateDescriptor;
 import edu.uci.ics.asterix.runtime.aggregates.stream.NonEmptyStreamAggregateDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalDayAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalHourAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalIntervalEndAccessor;
@@ -76,11 +83,6 @@
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalMonthAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalSecondAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalYearAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.common.CreateMBREvalFactory;
 import edu.uci.ics.asterix.runtime.evaluators.common.FieldAccessByIndexEvalFactory;
 import edu.uci.ics.asterix.runtime.evaluators.common.FunctionManagerImpl;
@@ -88,6 +90,7 @@
 import edu.uci.ics.asterix.runtime.evaluators.constructors.ACircleConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.constructors.ADateConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.constructors.ADateTimeConstructorDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.constructors.ADayTimeDurationConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.constructors.ADoubleConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.constructors.ADurationConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.constructors.AFloatConstructorDescriptor;
@@ -109,8 +112,10 @@
 import edu.uci.ics.asterix.runtime.evaluators.constructors.ARectangleConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.constructors.AStringConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.constructors.ATimeConstructorDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.CastListDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.CastRecordDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.ClosedRecordConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.CodePointToStringDescriptor;
@@ -170,10 +175,6 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StartsWithDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StringConcatDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StringEndWithDescrtiptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StringEqualDescriptor;
@@ -189,6 +190,10 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.Substring2Descriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringAfterDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringBeforeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDateDurationDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDatetimeDurationDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddTimeDurationDescriptor;
@@ -207,6 +212,8 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DurationEqualDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DurationFromMillisecondsDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.DurationFromMonthsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.GetDayTimeDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.GetYearMonthDurationDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalAfterDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalBeforeDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalCoveredByDescriptor;
@@ -215,13 +222,16 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalEndsDecriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalMeetsDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalMetByDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MillisecondsOfDayTimeDurationDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MonthsOfYearMonthDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MillisecondsFromDayTimeDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MonthsFromYearMonthDurationDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalOverlappedByDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MillisecondsFromDayTimeDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MonthsFromYearMonthDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDateDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDatetimeDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractTimeDescriptor;
@@ -416,6 +426,8 @@
         temp.add(ADateConstructorDescriptor.FACTORY);
         temp.add(ADateTimeConstructorDescriptor.FACTORY);
         temp.add(ADurationConstructorDescriptor.FACTORY);
+        temp.add(AYearMonthDurationConstructorDescriptor.FACTORY);
+        temp.add(ADayTimeDurationConstructorDescriptor.FACTORY);
 
         // Spatial
         temp.add(CreatePointDescriptor.FACTORY);
@@ -462,6 +474,7 @@
         temp.add(SwitchCaseDescriptor.FACTORY);
         temp.add(RegExpDescriptor.FACTORY);
         temp.add(InjectFailureDescriptor.FACTORY);
+        temp.add(CastListDescriptor.FACTORY);
         temp.add(CastRecordDescriptor.FACTORY);
         temp.add(NotNullDescriptor.FACTORY);
 
@@ -515,9 +528,11 @@
         temp.add(YearMonthDurationComparatorDecriptor.LESS_THAN_FACTORY);
         temp.add(DayTimeDurationComparatorDescriptor.GREATER_THAN_FACTORY);
         temp.add(DayTimeDurationComparatorDescriptor.LESS_THAN_FACTORY);
-        temp.add(MonthsOfYearMonthDurationDescriptor.FACTORY);
-        temp.add(MillisecondsOfDayTimeDurationDescriptor.FACTORY);
+        temp.add(MonthsFromYearMonthDurationDescriptor.FACTORY);
+        temp.add(MillisecondsFromDayTimeDurationDescriptor.FACTORY);
         temp.add(DurationEqualDescriptor.FACTORY);
+        temp.add(GetYearMonthDurationDescriptor.FACTORY);
+        temp.add(GetDayTimeDurationDescriptor.FACTORY);
 
         // Interval constructor
         temp.add(AIntervalFromDateConstructorDescriptor.FACTORY);
@@ -689,9 +704,22 @@
             }
         }
         if (fd.getIdentifier().equals(AsterixBuiltinFunctions.CAST_RECORD)) {
-            ARecordType rt = (ARecordType) TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) expr);
-            ARecordType it = (ARecordType) TypeComputerUtilities.getInputType((AbstractFunctionCallExpression) expr);
-            ((CastRecordDescriptor) fd).reset(rt, it);
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            ARecordType rt = (ARecordType) TypeComputerUtilities.getRequiredType(funcExpr);
+            IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
+            if (it.getTypeTag().equals(ATypeTag.ANY)) {
+                it = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+            }
+            ((CastRecordDescriptor) fd).reset(rt, (ARecordType) it);
+        }
+        if (fd.getIdentifier().equals(AsterixBuiltinFunctions.CAST_LIST)) {
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            AbstractCollectionType rt = (AbstractCollectionType) TypeComputerUtilities.getRequiredType(funcExpr);
+            IAType it = (IAType) context.getType(funcExpr.getArguments().get(0).getValue());
+            if (it.getTypeTag().equals(ATypeTag.ANY)) {
+                it = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+            }
+            ((CastListDescriptor) fd).reset(rt, (AbstractCollectionType) it);
         }
         if (fd.getIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)) {
             ARecordType rt = (ARecordType) context.getType(expr);
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
index b32110b..35c0529 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/operators/file/ADMDataParser.java
@@ -33,6 +33,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ACircleSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADateTimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADayTimeDurationSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ADurationSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AIntervalSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ALineSerializerDeserializer;
@@ -41,6 +42,7 @@
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
 import edu.uci.ics.asterix.dataflow.data.nontagged.serde.ATimeSerializerDeserializer;
+import edu.uci.ics.asterix.dataflow.data.nontagged.serde.AYearMonthDurationerializerDeserializer;
 import edu.uci.ics.asterix.om.base.ABoolean;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.types.AOrderedListType;
@@ -288,6 +290,14 @@
                 parseConstructor(ATypeTag.DURATION, objectType, out);
                 break;
             }
+            case AdmLexer.TOKEN_YEAR_MONTH_DURATION_CONS: {
+                parseConstructor(ATypeTag.YEARMONTHDURATION, objectType, out);
+                break;
+            }
+            case AdmLexer.TOKEN_DAY_TIME_DURATION_CONS: {
+                parseConstructor(ATypeTag.DAYTIMEDURATION, objectType, out);
+                break;
+            }
             case AdmLexer.TOKEN_POINT_CONS: {
                 parseConstructor(ATypeTag.POINT, objectType, out);
                 break;
@@ -363,6 +373,22 @@
 
     }
 
+    private void parseYearMonthDuration(String duration, DataOutput out) throws AsterixException {
+        try {
+            AYearMonthDurationerializerDeserializer.parse(duration, out);
+        } catch (HyracksDataException e) {
+            throw new AsterixException(e);
+        }
+    }
+
+    private void parseDayTimeDuration(String duration, DataOutput out) throws AsterixException {
+        try {
+            ADayTimeDurationSerializerDeserializer.parse(duration, out);
+        } catch (HyracksDataException e) {
+            throw new AsterixException(e);
+        }
+    }
+
     private IAType getComplexType(IAType aObjectType, ATypeTag tag) {
 
         if (aObjectType == null) {
@@ -762,6 +788,16 @@
                                         admLexer.getLastTokenImage().substring(1,
                                                 admLexer.getLastTokenImage().length() - 1), out);
                                 break;
+                            case DAYTIMEDURATION:
+                                parseDayTimeDuration(
+                                        admLexer.getLastTokenImage().substring(1,
+                                                admLexer.getLastTokenImage().length() - 1), out);
+                                break;
+                            case YEARMONTHDURATION:
+                                parseYearMonthDuration(
+                                        admLexer.getLastTokenImage().substring(1,
+                                                admLexer.getLastTokenImage().length() - 1), out);
+                                break;
                             case POINT:
                                 parsePoint(
                                         admLexer.getLastTokenImage().substring(1,
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index c70cdf8..f29d6be 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -22,6 +22,8 @@
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
 import edu.uci.ics.asterix.runtime.evaluators.common.AsterixListAccessor;
 import edu.uci.ics.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -73,12 +75,19 @@
                 private ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
                 private ICopyEvaluator argEval = listEvalFactory.createEvaluator(inputVal);
                 private int itemIndex;
+                private boolean metNull = false;
 
                 @Override
                 public void init(IFrameTupleReference tuple) throws AlgebricksException {
                     try {
                         inputVal.reset();
                         argEval.evaluate(tuple);
+                        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                                .deserialize(inputVal.getByteArray()[0]);
+                        if (typeTag == ATypeTag.NULL) {
+                            metNull = true;
+                            return;
+                        }
                         listAccessor.reset(inputVal.getByteArray(), 0);
                         itemIndex = 0;
                     } catch (AsterixException e) {
@@ -89,10 +98,12 @@
                 @Override
                 public boolean step() throws AlgebricksException {
                     try {
-                        if (itemIndex < listAccessor.size()) {
-                            listAccessor.writeItem(itemIndex, out);
-                            ++itemIndex;
-                            return true;
+                        if (!metNull) {
+                            if (itemIndex < listAccessor.size()) {
+                                listAccessor.writeItem(itemIndex, out);
+                                ++itemIndex;
+                                return true;
+                            }
                         }
                     } catch (IOException e) {
                         throw new AlgebricksException(e);
diff --git a/asterix-runtime/src/main/resources/adm.grammar b/asterix-runtime/src/main/resources/adm.grammar
index 4d4158f..80e2cd4 100644
--- a/asterix-runtime/src/main/resources/adm.grammar
+++ b/asterix-runtime/src/main/resources/adm.grammar
@@ -29,6 +29,8 @@
 INTERVAL_TIME_CONS      = string(interval-time)
 INTERVAL_DATE_CONS      = string(interval-date)
 INTERVAL_DATETIME_CONS  = string(interval-datetime)
+YEAR_MONTH_DURATION_CONS = string(year-month-duration)
+DAY_TIME_DURATION_CONS   = string(day-time-duration) 
 
 NULL_LITERAL   = string(null)
 TRUE_LITERAL   = string(true)
diff --git a/asterix-test-framework/.gitignore b/asterix-test-framework/.gitignore
new file mode 100644
index 0000000..ea8c4bf
--- /dev/null
+++ b/asterix-test-framework/.gitignore
@@ -0,0 +1 @@
+/target
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
index d662cba..9da0a8f 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/IndexOperationTracker.java
@@ -16,6 +16,8 @@
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import java.util.concurrent.atomic.AtomicInteger;
+
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
 import edu.uci.ics.hyracks.storage.am.common.api.ISearchOperationCallback;
@@ -31,7 +33,7 @@
 public class IndexOperationTracker implements ILSMOperationTracker {
 
     // Number of active operations on a ILSMIndex instance.
-    private int numActiveOperations = 0;
+    private AtomicInteger numActiveOperations;
     private long lastLSN;
     private long firstLSN;
     private final ILSMIndex index;
@@ -39,6 +41,7 @@
     private ILSMIndexAccessor accessor;
 
     public IndexOperationTracker(ILSMIndex index, ILSMIOOperationCallbackFactory ioOpCallbackFactory) {
+        this.numActiveOperations = new AtomicInteger(0);
         this.index = index;
         //TODO 
         //This code is added to avoid NullPointException when the index's comparatorFactory is null.
@@ -55,7 +58,7 @@
     public void beforeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
         if (opType != LSMOperationType.FORCE_MODIFICATION) {
-            numActiveOperations++;
+            numActiveOperations.incrementAndGet();
 
             // Increment transactor-local active operations count.
             AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
@@ -77,7 +80,6 @@
     @Override
     public void completeOperation(LSMOperationType opType, ISearchOperationCallback searchCallback,
             IModificationOperationCallback modificationCallback) throws HyracksDataException {
-        numActiveOperations--;
 
         // Decrement transactor-local active operations count.
         AbstractOperationCallback opCallback = getOperationCallback(searchCallback, modificationCallback);
@@ -86,7 +88,7 @@
         }
         // If we need a flush, and this is the last completing operation, then schedule the flush.
         // Once the flush has completed notify all waiting operations.
-        if (index.getFlushStatus() && numActiveOperations == 0 && opType != LSMOperationType.FLUSH) {
+        if (index.getFlushStatus() && numActiveOperations.decrementAndGet() == 0 && opType != LSMOperationType.FLUSH) {
             if (accessor == null) {
                 accessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
                         NoOpOperationCallback.INSTANCE);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
index 603e678..e06b5df 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMBTreeLocalResourceMetadata.java
@@ -55,4 +55,4 @@
         return lsmBTree;
     }
 
-}
\ No newline at end of file
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
index bad91f7..d56c6ee 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMInvertedIndexLocalResourceMetadata.java
@@ -78,4 +78,4 @@
             throw new HyracksDataException(e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
index b91b3d9..95fab45 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/resource/LSMRTreeLocalResourceMetadata.java
@@ -71,4 +71,4 @@
             throw new HyracksDataException(e);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
index dbe76ff..bd414de 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockWaiterManager.java
@@ -223,45 +223,35 @@
      */
     private void shrink() {
         int i;
-        boolean bContiguous = true;
-        int decreaseCount = 0;
+        int removeCount = 0;
         int size = pArray.size();
         int maxDecreaseCount = size / 2;
         ChildLockWaiterArrayManager child;
-        for (i = size - 1; i >= 0; i--) {
-            child = pArray.get(i);
-            if (child.isEmpty() || child.isDeinitialized()) {
-                if (bContiguous) {
-                    pArray.remove(i);
-                    if (++decreaseCount == maxDecreaseCount) {
-                        break;
-                    }
-                } else {
-                    bContiguous = false;
-                    if (child.isEmpty()) {
-                        child.deinitialize();
-                        if (++decreaseCount == maxDecreaseCount) {
-                            break;
-                        }
-                    }
-                }
-            } else {
-                bContiguous = false;
+
+        //The first buffer never be deinitialized.
+        for (i = 1; i < size; i++) {
+            if (pArray.get(i).isEmpty()) {
+                pArray.get(i).deinitialize();
             }
         }
 
-        //reset allocChild when the child is removed or deinitialized.
-        size = pArray.size();
-        if (allocChild >= size || pArray.get(allocChild).isDeinitialized()) {
-            //set allocChild to any initialized one.
-            //It is guaranteed that there is at least one initialized child.
-            for (i = 0; i < size; i++) {
-                if (!pArray.get(i).isDeinitialized()) {
-                    allocChild = i;
+        //remove the empty buffers from the end
+        for (i = size - 1; i >= 1; i--) {
+            child = pArray.get(i);
+            if (child.isDeinitialized()) {
+                pArray.remove(i);
+                if (++removeCount == maxDecreaseCount) {
                     break;
                 }
+            } else {
+                break;
             }
         }
+        
+        //reset allocChild to the first buffer
+        allocChild = 0;
+
+        isShrinkTimerOn = false;
     }
 
     public String prettyPrint() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
index 6926518..f452631 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
@@ -23,6 +23,7 @@
 import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
 import edu.uci.ics.asterix.common.transactions.ILogCursor;
 import edu.uci.ics.asterix.common.transactions.ILogFilter;
+import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
 import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 
@@ -30,27 +31,16 @@
 
     private final LogManager logManager;
     private final ILogFilter logFilter;
+    private final int logPageSize;
     private IBuffer readOnlyBuffer;
     private LogicalLogLocator logicalLogLocator = null;
-    private long bufferIndex = 0;
-    private boolean firstNext = true;
-    private boolean readMemory = false;
-    private long readLSN = 0;
     private boolean needReloadBuffer = true;
 
-    /**
-     * @param logFilter
-     */
-    public LogCursor(final LogManager logManager, ILogFilter logFilter) throws ACIDException {
+    public LogCursor(final LogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter,
+            int logPageSize) throws IOException, ACIDException {
         this.logFilter = logFilter;
         this.logManager = logManager;
-
-    }
-
-    public LogCursor(final LogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter)
-            throws IOException, ACIDException {
-        this.logFilter = logFilter;
-        this.logManager = logManager;
+        this.logPageSize = logPageSize;
         initialize(startingPhysicalLogLocator);
     }
 
@@ -64,7 +54,8 @@
         File file = new File(filePath);
         if (file.exists()) {
             return FileUtil.getFileBasedBuffer(filePath, lsn
-                    % logManager.getLogManagerProperties().getLogPartitionSize(), size);
+                    % logManager.getLogManagerProperties().getLogPartitionSize(), size, logManager
+                    .getLogManagerProperties().getDiskSectorSize());
         } else {
             return null;
         }
@@ -94,8 +85,7 @@
             return false;
         }
 
-        //if the lsn to read is greater than the last flushed lsn, then read from memory
-        if (logicalLogLocator.getLsn() > logManager.getLastFlushedLsn().get()) {
+        if (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
             return readFromMemory(currentLogLocator);
         }
 
@@ -103,10 +93,9 @@
         //needReloadBuffer is set to true if the log record is read from the memory log page.
         if (needReloadBuffer) {
             //log page size doesn't exceed integer boundary
-            int offset = (int)(logicalLogLocator.getLsn() % logManager.getLogManagerProperties().getLogPageSize());
+            int offset = (int) (logicalLogLocator.getLsn() % logPageSize);
             long adjustedLSN = logicalLogLocator.getLsn() - offset;
-            readOnlyBuffer = getReadOnlyBuffer(adjustedLSN, logManager.getLogManagerProperties()
-                    .getLogPageSize());
+            readOnlyBuffer = getReadOnlyBuffer(adjustedLSN, logPageSize);
             logicalLogLocator.setBuffer(readOnlyBuffer);
             logicalLogLocator.setMemoryOffset(offset);
             needReloadBuffer = false;
@@ -117,14 +106,14 @@
         while (logicalLogLocator.getMemoryOffset() <= readOnlyBuffer.getSize()
                 - logManager.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
             integerRead = readOnlyBuffer.readInt(logicalLogLocator.getMemoryOffset());
-            if (integerRead == logManager.getLogManagerProperties().LOG_MAGIC_NUMBER) {
+            if (integerRead == LogManagerProperties.LOG_MAGIC_NUMBER) {
                 logRecordBeginPosFound = true;
                 break;
             }
             logicalLogLocator.increaseMemoryOffset(1);
             logicalLogLocator.incrementLsn();
             bytesSkipped++;
-            if (bytesSkipped > logManager.getLogManagerProperties().getLogPageSize()) {
+            if (bytesSkipped > logPageSize) {
                 return false; // the maximum size of a log record is limited to
                 // a log page size. If we have skipped as many
                 // bytes without finding a log record, it
@@ -140,10 +129,9 @@
             // need to reload the buffer
             // TODO
             // reduce IO by reading more pages(equal to logBufferSize) at a time.
-            long lsnpos = ((logicalLogLocator.getLsn() / logManager.getLogManagerProperties().getLogPageSize()) + 1)
-                    * logManager.getLogManagerProperties().getLogPageSize();
+            long lsnpos = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
 
-            readOnlyBuffer = getReadOnlyBuffer(lsnpos, logManager.getLogManagerProperties().getLogPageSize());
+            readOnlyBuffer = getReadOnlyBuffer(lsnpos, logPageSize);
             if (readOnlyBuffer != null) {
                 logicalLogLocator.setBuffer(readOnlyBuffer);
                 logicalLogLocator.setLsn(lsnpos);
@@ -197,13 +185,12 @@
         IFileBasedBuffer logPage = logManager.getLogPage(pageIndex);
         synchronized (logPage) {
             // need to check again if the log record in the log buffer or has reached the disk
-            if (lsn > logManager.getLastFlushedLsn().get()) {
+            if (logManager.isMemoryRead(lsn)) {
 
                 //find the magic number to identify the start of the log record
                 //----------------------------------------------------------------
                 int readNumber = -1;
-                int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
-                int logMagicNumber = logManager.getLogManagerProperties().LOG_MAGIC_NUMBER;
+                int logMagicNumber = LogManagerProperties.LOG_MAGIC_NUMBER;
                 int bytesSkipped = 0;
                 boolean logRecordBeginPosFound = false;
                 //check whether the currentOffset has enough space to have new log record by comparing
@@ -230,7 +217,8 @@
                     // need to read the next log page
                     readOnlyBuffer = null;
                     logicalLogLocator.setBuffer(null);
-                    logicalLogLocator.setLsn(lsn / logPageSize + 1);
+                    lsn = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
+                    logicalLogLocator.setLsn(lsn);
                     logicalLogLocator.setMemoryOffset(0);
                     return next(currentLogLocator);
                 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index eceb526..318996a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -27,7 +27,6 @@
 import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -46,8 +45,7 @@
 import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManager.PageOwnershipStatus;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManager.PageState;
+import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
@@ -59,6 +57,9 @@
     private final TransactionSubsystem provider;
     private LogManagerProperties logManagerProperties;
     private LogPageFlushThread logPageFlusher;
+    private final int logPageSize;
+    private long statLogSize;
+    private long statLogCount;
 
     /*
      * the array of log pages. The number of log pages is configurable. Pages
@@ -73,47 +74,6 @@
      */
     private int numLogPages;
 
-    /*
-     * Initially all pages have an owner count of 1 that is the LogManager. When
-     * a transaction requests to write in a log page, the owner count is
-     * incremented. The log manager reserves space in the log page and puts in
-     * the log header but leaves the space for the content and the checksum
-     * (covering the whole log record). When the content has been put, the log
-     * manager computes the checksum and puts it after the content. At this
-     * point, the ownership count is decremented as the transaction is done with
-     * using the page. When a page is requested to be flushed, logPageFlusher
-     * set the count to 0(LOG_FLUSHER: meaning that the page is being flushed)
-     * only if the count is 1(LOG_WRITER: meaning that there is no other
-     * transactions who own the page to write logs.) After flushing the page,
-     * logPageFlusher set this count to 1.
-     */
-    private AtomicInteger[] logPageOwnerCount;
-
-    static class PageOwnershipStatus {
-        public static final int LOG_WRITER = 1;
-        public static final int LOG_FLUSHER = 0;
-    }
-
-    /*
-     * LogPageStatus: A page is either ACTIVE or INACTIVE. The status for each
-     * page is maintained in logPageStatus. A page is ACTIVE when the LogManager
-     * can allocate space in the page for writing a log record. Initially all
-     * pages are ACTIVE. As transactions fill up space by writing log records, a
-     * page may not have sufficient space left for serving a request by a
-     * transaction. When this happens, the page is flushed to disk by calling
-     * logPageFlusher.requestFlush(). In the requestFlush(), after
-     * groupCommitWaitTime, the page status is set to INACTIVE. Then, there is
-     * no more writer on the page(meaning the corresponding logPageOwnerCount is
-     * 1), the page is flushed by the logPageFlusher and the status is reset to
-     * ACTIVE by the logPageFlusher.
-     */
-    private AtomicInteger[] logPageStatus;
-
-    static class PageState {
-        public static final int INACTIVE = 0;
-        public static final int ACTIVE = 1;
-    }
-
     private AtomicLong lastFlushedLSN = new AtomicLong(-1);
 
     /*
@@ -140,10 +100,6 @@
         return lastFlushedLSN;
     }
 
-    public AtomicInteger getLogPageStatus(int pageIndex) {
-        return logPageStatus[pageIndex];
-    }
-
     public AtomicLong getCurrentLsn() {
         return lsn;
     }
@@ -155,13 +111,19 @@
     public LogManager(TransactionSubsystem provider) throws ACIDException {
         this.provider = provider;
         initLogManagerProperties(this.provider.getId());
+        logPageSize = logManagerProperties.getLogPageSize();
         initLogManager();
+        statLogSize = 0;
+        statLogCount = 0;
     }
 
     public LogManager(TransactionSubsystem provider, String nodeId) throws ACIDException {
         this.provider = provider;
         initLogManagerProperties(nodeId);
+        logPageSize = logManagerProperties.getLogPageSize();
         initLogManager();
+        statLogSize = 0;
+        statLogCount = 0;
     }
 
     /*
@@ -197,10 +159,8 @@
     private void initLogManager() throws ACIDException {
         logRecordHelper = new LogRecordHelper(this);
         numLogPages = logManagerProperties.getNumLogPages();
-        logPageOwnerCount = new AtomicInteger[numLogPages];
-        logPageStatus = new AtomicInteger[numLogPages];
-
         activeTxnCountMaps = new ArrayList<HashMap<ITransactionContext, Integer>>(numLogPages);
+
         for (int i = 0; i < numLogPages; i++) {
             activeTxnCountMaps.add(new HashMap<ITransactionContext, Integer>());
         }
@@ -210,20 +170,12 @@
         /*
          * place the log anchor at the end of the last log record written.
          */
-        PhysicalLogLocator nextPhysicalLsn = initLSN();
-
-        /*
-         * initialize meta data for each log page.
-         */
-        for (int i = 0; i < numLogPages; i++) {
-            logPageOwnerCount[i] = new AtomicInteger(PageOwnershipStatus.LOG_WRITER);
-            logPageStatus[i] = new AtomicInteger(PageState.ACTIVE);
-        }
+        initLSN();
 
         /*
          * initialize the log pages.
          */
-        initializeLogPages(nextPhysicalLsn);
+        initializeLogPages(startingLSN);
 
         /*
          * Instantiate and begin the LogFlusher thread. The Log Flusher thread
@@ -237,7 +189,7 @@
     }
 
     public int getLogPageIndex(long lsnValue) {
-        return (int) (((lsnValue - startingLSN) / logManagerProperties.getLogPageSize()) % numLogPages);
+        return (int) (((lsnValue - startingLSN) / logPageSize) % numLogPages);
     }
 
     /*
@@ -253,28 +205,7 @@
      * record is (to be) placed.
      */
     public int getLogPageOffset(long lsnValue) {
-        return (int) ((lsnValue - startingLSN) % logManagerProperties.getLogPageSize());
-    }
-
-    /*
-     * a transaction thread under certain scenarios is required to wait until
-     * the page where it has to write a log record becomes available for writing
-     * a log record.
-     */
-    private void waitUntillPageIsAvailableForWritingLog(int pageIndex) throws ACIDException {
-        if (logPageStatus[pageIndex].get() == PageState.ACTIVE
-                && logPageOwnerCount[pageIndex].get() >= PageOwnershipStatus.LOG_WRITER) {
-            return;
-        }
-        try {
-            synchronized (logPages[pageIndex]) {
-                while (!(logPageStatus[pageIndex].get() == PageState.ACTIVE && logPageOwnerCount[pageIndex].get() >= PageOwnershipStatus.LOG_WRITER)) {
-                    logPages[pageIndex].wait();
-                }
-            }
-        } catch (InterruptedException e) {
-            throw new ACIDException(" thread interrupted while waiting for page " + pageIndex + " to be available ", e);
-        }
+        return (int) (lsnValue % logPageSize);
     }
 
     /*
@@ -288,7 +219,6 @@
      * @param logType: the type of log record.
      */
     private long getLsn(int entrySize, byte logType) throws ACIDException {
-        long pageSize = logManagerProperties.getLogPageSize();
 
         while (true) {
             boolean forwardPage = false;
@@ -305,9 +235,9 @@
 
             // check if the log record will cross page boundaries, a case that
             // is not allowed.
-            if ((next - 1) / pageSize != old / pageSize || (next % pageSize == 0)) {
+            if ((next - 1) / logPageSize != old / logPageSize || (next % logPageSize == 0)) {
 
-                if ((old != 0 && old % pageSize == 0)) {
+                if ((old != 0 && old % logPageSize == 0)) {
                     // On second thought, this shall never be the case as it
                     // means that the lsn is
                     // currently at the beginning of a page and we still need to
@@ -320,7 +250,7 @@
 
                 } else {
                     // set the lsn to point to the beginning of the next page.
-                    retVal = ((old / pageSize) + 1) * pageSize;
+                    retVal = ((old / logPageSize) + 1) * logPageSize;
                 }
 
                 next = retVal;
@@ -334,20 +264,6 @@
                 pageIndex = getNextPageInSequence(pageIndex);
             }
 
-            /*
-             * we do not want to keep allocating LSNs if the corresponding page
-             * is unavailable. Consider a scenario when the log flusher thread
-             * is incredibly slow in flushing pages. Transaction threads will
-             * acquire an lsn each for writing their next log record. When a
-             * page has been made available, mulltiple transaction threads that
-             * were waiting can continue to write their log record at the
-             * assigned LSNs. Two transaction threads may get LSNs that are on
-             * the same log page but actually differ by the size of the log
-             * buffer. This would be erroneous. Transaction threads are made to
-             * wait upfront for avoiding this situation.
-             */
-            waitUntillPageIsAvailableForWritingLog(pageIndex);
-
             if (!lsn.compareAndSet(old, next)) {
                 // Atomic call -> returns true only when the value represented
                 // by lsn is same as
@@ -356,6 +272,17 @@
             }
 
             if (forwardPage) {
+                logPages[prevPage].acquireReadLatch();
+                // increment the counter as the transaction thread now holds a
+                // space in the log page and hence is an owner.
+                logPages[prevPage].incRefCnt();
+                logPages[prevPage].releaseReadLatch();
+
+                // forward the nextWriteOffset in the log page
+                logPages[prevPage].setBufferNextWriteOffset(logPageSize);
+
+                logPages[prevPage].decRefCnt();
+
                 addFlushRequest(prevPage, old, false);
 
                 // The transaction thread that discovers the need to forward a
@@ -363,21 +290,18 @@
                 continue;
 
             } else {
-                // the transaction thread has been given a space in a log page,
-                // but is made to wait until the page is available.
-                // (Is this needed? when does this wait happen?)
-                waitUntillPageIsAvailableForWritingLog(pageIndex);
-
+                logPages[pageIndex].acquireReadLatch();
                 // increment the counter as the transaction thread now holds a
                 // space in the log page and hence is an owner.
-                logPageOwnerCount[pageIndex].incrementAndGet();
+                logPages[pageIndex].incRefCnt();
+                logPages[pageIndex].releaseReadLatch();
 
                 // Before the count is incremented, if the flusher flushed the
                 // allocated page,
                 // then retry to get new LSN. Otherwise, the log with allocated
                 // lsn will be lost.
                 if (lastFlushedLSN.get() >= retVal) {
-                    logPageOwnerCount[pageIndex].decrementAndGet();
+                    logPages[pageIndex].decRefCnt();
                     continue;
                 }
             }
@@ -407,10 +331,10 @@
         int totalLogSize = logRecordHelper.getLogRecordSize(logType, logContentSize);
 
         // check for the total space requirement to be less than a log page.
-        if (totalLogSize > logManagerProperties.getLogPageSize()) {
+        if (totalLogSize > logPageSize) {
             throw new ACIDException(
                     " Maximum Log Content Size is "
-                            + (logManagerProperties.getLogPageSize() - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
+                            + (logPageSize - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
                                     .getLogChecksumSize()));
         }
 
@@ -493,16 +417,21 @@
             logPages[pageIndex].writeLong(pageOffset + logRecordHelper.getLogHeaderSize(logType) + logContentSize,
                     checksum);
 
-            if (IS_DEBUG_MODE) {
-                System.out.println("--------------> LSN(" + currentLSN + ") is written");
+            // forward the nextWriteOffset in the log page
+            int bufferNextWriteOffset = (int) ((currentLSN + totalLogSize) % logPageSize);
+            if (bufferNextWriteOffset == 0) {
+                bufferNextWriteOffset = logPageSize;
             }
+            logPages[pageIndex].setBufferNextWriteOffset(bufferNextWriteOffset);
 
-            // release the ownership as the log record has been placed in
-            // created space.
-            logPageOwnerCount[pageIndex].decrementAndGet();
+            if (logType != LogType.ENTITY_COMMIT) {
+                // release the ownership as the log record has been placed in
+                // created space.
+                logPages[pageIndex].decRefCnt();
 
-            // indicating that the transaction thread has released ownership
-            decremented = true;
+                // indicating that the transaction thread has released ownership
+                decremented = true;
+            }
 
             if (logType == LogType.ENTITY_COMMIT) {
                 map = activeTxnCountMaps.get(pageIndex);
@@ -513,18 +442,42 @@
                 } else {
                     map.put(txnCtx, 1);
                 }
+                //------------------------------------------------------------------------------
+                // [Notice]
+                // reference count should be decremented 
+                // after activeTxnCount is incremented, but before addFlushRequest() is called. 
+                //------------------------------------------------------------------------------
+                // release the ownership as the log record has been placed in
+                // created space.
+                logPages[pageIndex].decRefCnt();
+
+                // indicating that the transaction thread has released ownership
+                decremented = true;
+
                 addFlushRequest(pageIndex, currentLSN, false);
             } else if (logType == LogType.COMMIT) {
+
                 addFlushRequest(pageIndex, currentLSN, true);
+                if (IS_DEBUG_MODE) {
+                    System.out.println("Running sum of log size: " + statLogSize + ", log count: " + statLogCount);
+                }
             }
 
+            if (IS_DEBUG_MODE) {
+                System.out.println("--------------> LSN(" + currentLSN + ") is written");
+            }
+
+            //collect statistics
+            statLogSize += totalLogSize;
+            statLogCount++;
+
         } catch (Exception e) {
             e.printStackTrace();
             throw new ACIDException(txnCtx, "Thread: " + Thread.currentThread().getName()
                     + " logger encountered exception", e);
         } finally {
             if (!decremented) {
-                logPageOwnerCount[pageIndex].decrementAndGet();
+                logPages[pageIndex].decRefCnt();
             }
         }
     }
@@ -537,20 +490,13 @@
 
         String filePath = LogUtil.getLogFilePath(logManagerProperties, getLogFileId(lsn));
 
-        logPages[pageIndex].reset(filePath, LogUtil.getFileOffset(this, nextWritePosition),
-                logManagerProperties.getLogPageSize());
-    }
-
-    @Override
-    public ILogCursor readLog(ILogFilter logFilter) throws ACIDException {
-        LogCursor cursor = new LogCursor(this, logFilter);
-        return cursor;
+        logPages[pageIndex].reset(filePath, LogUtil.getFileOffset(this, nextWritePosition), logPageSize);
     }
 
     @Override
     public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
             ACIDException {
-        LogCursor cursor = new LogCursor(this, physicalLogLocator, logFilter);
+        LogCursor cursor = new LogCursor(this, physicalLogLocator, logFilter, logPageSize);
         return cursor;
     }
 
@@ -561,7 +507,7 @@
         String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, lsnValue));
         long fileOffset = LogUtil.getFileOffset(this, lsnValue);
 
-        ByteBuffer buffer = ByteBuffer.allocate(logManagerProperties.getLogPageSize());
+        ByteBuffer buffer = ByteBuffer.allocate(logPageSize);
         RandomAccessFile raf = null;
         FileChannel fileChannel = null;
         try {
@@ -614,7 +560,7 @@
         }
 
         /* check if the log record in the log buffer or has reached the disk. */
-        if (lsnValue > getLastFlushedLsn().get()) {
+        if (isMemoryRead(lsnValue)) {
             int pageIndex = getLogPageIndex(lsnValue);
             int pageOffset = getLogPageOffset(lsnValue);
 
@@ -622,7 +568,7 @@
             // minimize memory allocation overhead. current code allocates the
             // log page size per reading a log record.
 
-            byte[] pageContent = new byte[logManagerProperties.getLogPageSize()];
+            byte[] pageContent = new byte[logPageSize];
 
             // take a lock on the log page so that the page is not flushed to
             // disk interim
@@ -630,8 +576,8 @@
 
                 // need to check again (this thread may have got de-scheduled
                 // and must refresh!)
-                if (lsnValue > getLastFlushedLsn().get()) {
 
+                if (isMemoryRead(lsnValue)) {
                     // get the log record length
                     logPages[pageIndex].getBytes(pageContent, 0, pageContent.length);
                     byte logType = pageContent[pageOffset + 4];
@@ -667,6 +613,20 @@
         readDiskLog(lsnValue, logicalLogLocator);
     }
 
+    public boolean isMemoryRead(long currentLSN) {
+        long flushLSN = lastFlushedLSN.get();
+        if ((flushLSN + 1) % logPageSize == 0) {
+            return false;
+        }
+        long logPageBeginOffset = flushLSN - (flushLSN % logPageSize);
+        long logPageEndOffset = logPageBeginOffset + logPageSize;
+        if (currentLSN > flushLSN || (currentLSN >= logPageBeginOffset && currentLSN < logPageEndOffset)) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
     public void renewLogFiles() throws ACIDException {
         List<String> logFileNames = LogUtil.getLogFiles(logManagerProperties);
         for (String name : logFileNames) {
@@ -681,7 +641,7 @@
         logPageFlusher.renew();
     }
 
-    private PhysicalLogLocator initLSN() throws ACIDException {
+    private void initLSN() throws ACIDException {
         PhysicalLogLocator nextPhysicalLsn = LogUtil.initializeLogAnchor(this);
         startingLSN = nextPhysicalLsn.getLsn();
         lastFlushedLSN.set(startingLSN - 1);
@@ -689,7 +649,6 @@
             LOGGER.info(" Starting lsn is : " + startingLSN);
         }
         lsn.set(startingLSN);
-        return nextPhysicalLsn;
     }
 
     private void closeLogPages() throws ACIDException {
@@ -706,9 +665,7 @@
         try {
             String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, startingLSN));
             for (int i = 0; i < numLogPages; i++) {
-                logPages[i].open(filePath,
-                        LogUtil.getFileOffset(this, startingLSN) + i * logManagerProperties.getLogPageSize(),
-                        logManagerProperties.getLogPageSize());
+                logPages[i].open(filePath, LogUtil.getFileOffset(this, startingLSN) + i * logPageSize, logPageSize);
             }
         } catch (Exception e) {
             throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
@@ -721,33 +678,25 @@
     }
 
     /*
-     * This method shall be called by the Buffer manager when it needs to evict
-     * a page from the cache. TODO: Change the implementation from a looping
-     * logic to event based when log manager support is integrated with the
-     * Buffer Manager.
-     */
-    @Override
-    public synchronized void flushLog(LogicalLogLocator logicalLogLocator) throws ACIDException {
-        if (logicalLogLocator.getLsn() > lsn.get()) {
-            throw new ACIDException(" invalid lsn " + logicalLogLocator.getLsn());
-        }
-        while (lastFlushedLSN.get() < logicalLogLocator.getLsn());
-    }
-
-    /*
      * Map each log page to cover a physical byte range over a log file. When a
      * page is flushed, the page contents are put to disk in the corresponding
      * byte range.
      */
-    private void initializeLogPages(PhysicalLogLocator physicalLogLocator) throws ACIDException {
+    private void initializeLogPages(long beginLsn) throws ACIDException {
         try {
-            String filePath = LogUtil.getLogFilePath(logManagerProperties,
-                    LogUtil.getFileId(this, physicalLogLocator.getLsn()));
+            String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, beginLsn));
+            long nextDiskWriteOffset = LogUtil.getFileOffset(this, beginLsn);
+            long nextBufferWriteOffset = nextDiskWriteOffset % logPageSize;
+            long bufferBeginOffset = nextDiskWriteOffset - nextBufferWriteOffset;
+
             for (int i = 0; i < numLogPages; i++) {
-                logPages[i] = FileUtil.getFileBasedBuffer(
-                        filePath,
-                        LogUtil.getFileOffset(this, physicalLogLocator.getLsn()) + i
-                                * logManagerProperties.getLogPageSize(), logManagerProperties.getLogPageSize());
+                logPages[i] = FileUtil.getFileBasedBuffer(filePath, bufferBeginOffset + i * logPageSize, logPageSize,
+                        logManagerProperties.getDiskSectorSize());
+                if (i == 0) {
+                    logPages[i].setBufferLastFlushOffset((int) nextBufferWriteOffset);
+                    logPages[i].setBufferNextWriteOffset((int) nextBufferWriteOffset);
+                    logPages[i].setDiskNextWriteOffset(nextDiskWriteOffset);
+                }
             }
         } catch (Exception e) {
             e.printStackTrace();
@@ -775,10 +724,6 @@
         return logPages[pageIndex];
     }
 
-    public AtomicInteger getLogPageOwnershipCount(int pageIndex) {
-        return logPageOwnerCount[pageIndex];
-    }
-
     public IFileBasedBuffer[] getLogPages() {
         return logPages;
     }
@@ -840,7 +785,7 @@
      */
     private final LinkedBlockingQueue<Object>[] flushRequestQueue;
     private final Object[] flushRequests;
-    private int pageToFlush;
+    private int flushPageIndex;
     private final long groupCommitWaitPeriod;
     private boolean isRenewRequest;
 
@@ -854,14 +799,14 @@
             flushRequestQueue[i] = new LinkedBlockingQueue<Object>(1);
             flushRequests[i] = new Object();
         }
-        this.pageToFlush = -1;
+        this.flushPageIndex = 0;
         groupCommitWaitPeriod = logManager.getLogManagerProperties().getGroupCommitWaitPeriod();
         isRenewRequest = false;
     }
 
     public void renew() {
         isRenewRequest = true;
-        pageToFlush = -1;
+        flushPageIndex = 0;
         this.interrupt();
         isRenewRequest = false;
     }
@@ -897,15 +842,19 @@
 
     @Override
     public void run() {
+        int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
+        int logBufferSize = logManager.getLogManagerProperties().getLogBufferSize();
+        int beforeFlushOffset = 0;
+        int afterFlushOffset = 0;
+        boolean resetFlushPageIndex = false;
+
         while (true) {
             try {
-                pageToFlush = logManager.getNextPageInSequence(pageToFlush);
-
                 // A wait call on the linkedBLockingQueue. The flusher thread is
                 // notified when an object is added to the queue. Please note
                 // that each page has an associated blocking queue.
                 try {
-                    flushRequestQueue[pageToFlush].take();
+                    flushRequestQueue[flushPageIndex].take();
                 } catch (InterruptedException ie) {
                     while (isRenewRequest) {
                         sleep(1);
@@ -913,58 +862,67 @@
                     continue;
                 }
 
-                synchronized (logManager.getLogPage(pageToFlush)) {
-
-                    // #. sleep during the groupCommitWaitTime
+                //if the log page is already full, don't wait. 
+                if (logManager.getLogPage(flushPageIndex).getBufferNextWriteOffset() < logPageSize
+                        - logManager.getLogRecordHelper().getCommitLogSize()) {
+                    // #. sleep for the groupCommitWaitTime
                     sleep(groupCommitWaitPeriod);
+                }
 
-                    // #. set the logPageStatus to INACTIVE in order to prevent
-                    // other txns from writing on this page.
-                    logManager.getLogPageStatus(pageToFlush).set(PageState.INACTIVE);
+                synchronized (logManager.getLogPage(flushPageIndex)) {
+                    logManager.getLogPage(flushPageIndex).acquireWriteLatch();
+                    try {
 
-                    // #. need to wait until the logPageOwnerCount reaches 1
-                    // (LOG_WRITER)
-                    // meaning every one has finished writing logs on this page.
-                    while (logManager.getLogPageOwnershipCount(pageToFlush).get() != PageOwnershipStatus.LOG_WRITER) {
-                        sleep(0);
+                        // #. need to wait until the reference count reaches 0
+                        while (logManager.getLogPage(flushPageIndex).getRefCnt() != 0) {
+                            sleep(0);
+                        }
+
+                        beforeFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
+
+                        // put the content to disk (the thread still has a lock on the log page)
+                        logManager.getLogPage(flushPageIndex).flush();
+
+                        afterFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
+
+                        // increment the last flushed lsn
+                        logManager.incrementLastFlushedLsn(afterFlushOffset - beforeFlushOffset);
+
+                        // increment currentLSN if currentLSN is less than flushLSN.
+                        if (logManager.getLastFlushedLsn().get() + 1 > logManager.getCurrentLsn().get()) {
+                            logManager.getCurrentLsn().set(logManager.getLastFlushedLsn().get() + 1);
+                        }
+
+                        // Map the log page to a new region in the log file if the flushOffset reached the logPageSize
+                        if (afterFlushOffset == logPageSize) {
+                            long diskNextWriteOffset = logManager.getLogPages()[flushPageIndex]
+                                    .getDiskNextWriteOffset() + logBufferSize;
+                            logManager.resetLogPage(logManager.getLastFlushedLsn().get() + 1 + logBufferSize,
+                                    diskNextWriteOffset, flushPageIndex);
+                            resetFlushPageIndex = true;
+                        }
+
+                        // decrement activeTxnCountOnIndexes
+                        logManager.decrementActiveTxnCountOnIndexes(flushPageIndex);
+
+                    } finally {
+                        logManager.getLogPage(flushPageIndex).releaseWriteLatch();
                     }
 
-                    // #. set the logPageOwnerCount to 0 (LOG_FLUSHER)
-                    // meaning it is flushing.
-                    logManager.getLogPageOwnershipCount(pageToFlush).set(PageOwnershipStatus.LOG_FLUSHER);
-
-                    // put the content to disk (the thread still has a lock on
-                    // the log page)
-                    logManager.getLogPage(pageToFlush).flush();
-
-                    // Map the log page to a new region in the log file.
-                    long nextWritePosition = logManager.getLogPages()[pageToFlush].getNextWritePosition()
-                            + logManager.getLogManagerProperties().getLogBufferSize();
-
-                    logManager.resetLogPage(logManager.getLastFlushedLsn().get() + 1
-                            + logManager.getLogManagerProperties().getLogBufferSize(), nextWritePosition, pageToFlush);
-
-                    // increment the last flushed lsn and lastFlushedPage
-                    logManager.incrementLastFlushedLsn(logManager.getLogManagerProperties().getLogPageSize());
-
-                    // decrement activeTxnCountOnIndexes
-                    logManager.decrementActiveTxnCountOnIndexes(pageToFlush);
-
-                    // reset the count to 1
-                    logManager.getLogPageOwnershipCount(pageToFlush).set(PageOwnershipStatus.LOG_WRITER);
-
-                    // mark the page as ACTIVE
-                    logManager.getLogPageStatus(pageToFlush).set(LogManager.PageState.ACTIVE);
-
                     // #. checks the queue whether there is another flush
                     // request on the same log buffer
                     // If there is another request, then simply remove it.
-                    if (flushRequestQueue[pageToFlush].peek() != null) {
-                        flushRequestQueue[pageToFlush].take();
+                    if (flushRequestQueue[flushPageIndex].peek() != null) {
+                        flushRequestQueue[flushPageIndex].take();
                     }
 
                     // notify all waiting (transaction) threads.
-                    logManager.getLogPage(pageToFlush).notifyAll();
+                    logManager.getLogPage(flushPageIndex).notifyAll();
+
+                    if (resetFlushPageIndex) {
+                        flushPageIndex = logManager.getNextPageInSequence(flushPageIndex);
+                        resetFlushPageIndex = false;
+                    }
                 }
             } catch (IOException ioe) {
                 ioe.printStackTrace();
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
index 5418ade..ed30b88 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
@@ -17,6 +17,7 @@
 import edu.uci.ics.asterix.common.transactions.ILogManager;
 import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
 import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
 
@@ -54,6 +55,9 @@
 public class LogRecordHelper implements ILogRecordHelper {
 
     private final int LOG_CHECKSUM_SIZE = 8;
+    private final int LOG_HEADER_PART1_SIZE = 17;
+    private final int LOG_HEADER_PART2_SIZE = 21;
+    private final int COMMIT_LOG_SIZE = LOG_HEADER_PART1_SIZE + LOG_CHECKSUM_SIZE;
 
     private final int MAGIC_NO_POS = 0;
     private final int LOG_TYPE_POS = 4;
@@ -122,7 +126,11 @@
 
     @Override
     public int getLogContentSize(LogicalLogLocator logicalLogLocater) {
-        return logicalLogLocater.getBuffer().readInt(logicalLogLocater.getMemoryOffset() + LOG_RECORD_SIZE_POS);
+        if (getLogType(logicalLogLocater) == LogType.COMMIT || getLogType(logicalLogLocater) == LogType.ENTITY_COMMIT) {
+            return 0;
+        } else {
+            return logicalLogLocater.getBuffer().readInt(logicalLogLocater.getMemoryOffset() + LOG_RECORD_SIZE_POS);
+        }
     }
 
     @Override
@@ -182,7 +190,7 @@
 
         /* magic no */
         (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + MAGIC_NO_POS,
-                logManager.getLogManagerProperties().LOG_MAGIC_NUMBER);
+                LogManagerProperties.LOG_MAGIC_NUMBER);
 
         /* log type */
         (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS, logType);
@@ -234,18 +242,18 @@
     @Override
     public int getLogRecordSize(byte logType, int logBodySize) {
         if (logType == LogType.UPDATE) {
-            return 46 + logBodySize;
+            return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE + LOG_CHECKSUM_SIZE + logBodySize;
         } else {
-            return 25;
+            return COMMIT_LOG_SIZE;
         }
     }
 
     @Override
     public int getLogHeaderSize(byte logType) {
         if (logType == LogType.UPDATE) {
-            return 38;
+            return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE;
         } else {
-            return 17;
+            return LOG_HEADER_PART1_SIZE;
         }
     }
 
@@ -253,4 +261,8 @@
     public int getLogChecksumSize() {
         return LOG_CHECKSUM_SIZE;
     }
-}
+
+    public int getCommitLogSize() {
+        return COMMIT_LOG_SIZE;
+    }
+}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/StorageContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/StorageContext.java
deleted file mode 100644
index f5e69ee..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/StorageContext.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * 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.transaction.management.service.transaction;
-
-import edu.uci.ics.asterix.common.dataflow.IStorageContext;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManagerProvider;
-import edu.uci.ics.hyracks.storage.common.IStorageManagerInterface;
-
-/*
- * Acts as an holder class for IndexRegistryProvider, AsterixStorageManager
- * instances that are accessed from the NCs. In addition an instance of ICCApplicationContext 
- * is stored for access by the CC.
- */
-public class StorageContext implements IStorageContext {
-
-    private static IStorageContext INSTANCE = new StorageContext();;
-
-    private StorageContext() {
-
-    }
-
-    public static IStorageContext getInstance() {
-        return INSTANCE;
-    }
-
-    @Override
-    public IStorageManagerInterface getStorageManagerInterface() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
-    }
-
-    @Override
-    public IIndexLifecycleManagerProvider getIndexLifecycleManagerProvider() {
-        return AsterixRuntimeComponentsProvider.NOINDEX_PROVIDER;
-    }
-}
diff --git a/pom.xml b/pom.xml
index 20efdb3..f4c46c6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -84,6 +84,7 @@
                 <module>asterix-server</module>
                 <module>asterix-installer</module>
                 <module>asterix-events</module>
+                <module>asterix-doc</module>
         </modules>
 
 	<repositories>