Merge branch 'chenli/master/wiki_changes' of https://code.google.com/p/asterixdb into chenli/master/wiki_changes
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 d5169c7..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;
@@ -103,6 +104,7 @@
         List<IAlgebraicRewriteRule> typeInfer = new LinkedList<IAlgebraicRewriteRule>();
         typeInfer.add(new InlineUnnestFunctionRule());
         typeInfer.add(new InferTypesRule());
+        typeInfer.add(new CheckFilterExpressionTypeRule());
         return typeInfer;
     }
 
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 1f242b8..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
@@ -26,7 +26,10 @@
 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;
@@ -37,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;
 
@@ -88,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);
-        IAType inputRecordType = (IAType) 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));
-        cast.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputRecordVar)));
-        TypeComputerUtilities.setRequiredAndInputTypes(cast, requiredRecordType, inputRecordType);
-        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);
-
-        context.computeAndSetTypeEnvironmentForOperator(newAssignOperator);
-        context.computeAndSetTypeEnvironmentForOperator(projectOperator);
-        context.computeAndSetTypeEnvironmentForOperator(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/typecast/StaticTypeCastUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index 6e3ab37..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
@@ -189,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);
     }
 
     /**
@@ -245,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();
@@ -341,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
@@ -385,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.");
+                }
             }
         }
 
@@ -472,6 +480,7 @@
                 arguments.add(expRef);
             }
         }
+        return true;
     }
 
     /**
@@ -505,7 +514,7 @@
      *            the input type
      * @return true if the two types are compatiable; false otherwise
      */
-    private static boolean compatible(IAType reqType, IAType inputType) {
+    public static boolean compatible(IAType reqType, IAType inputType) {
         if (reqType.getTypeTag() == ATypeTag.ANY || inputType.getTypeTag() == ATypeTag.ANY) {
             return true;
         }
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.1.ddl.aql
new file mode 100644
index 0000000..73d63c9
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using year-month-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:year-month-duration,
+dur:year-month-duration,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dur);
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.2.update.aql
new file mode 100644
index 0000000..bd96595
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using year-month-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":year-month-duration("P16Y"), "dur":year-month-duration("-P23Y"), "name": "John"})
+insert into dataset Employee({"id":year-month-duration("-P37M"), "dur":year-month-duration("P1Y48M"), "name": "Alex"})
+insert into dataset Employee({"id":year-month-duration("P2013Y"), "dur":year-month-duration("P7M"), "name": "Bob"})
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.3.query.aql
new file mode 100644
index 0000000..8244c6d
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_0/issue_363_temporal_sec_key_0.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : create a dataset using year-month-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dur > year-month-duration("P1Y")
+return $x
+
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.1.ddl.aql
new file mode 100644
index 0000000..9697c32
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using datetime for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:datetime,
+dt:datetime,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dt);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.2.update.aql
new file mode 100644
index 0000000..d66da92
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using datetime for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":datetime("1900-01-01T00:00:00"), "dt":datetime("1900-01-01T00:00:00"), "name": "John"})
+insert into dataset Employee({"id":datetime("2000-01-01T00:00:00"), "dt":datetime("2000-01-01T00:00:00"), "name": "Alex"})
+insert into dataset Employee({"id":datetime("2013-01-01T00:00:00"), "dt":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_sec_key_1/issue_363_temporal_sec_key_1.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.3.query.aql
new file mode 100644
index 0000000..eb92aea
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_1/issue_363_temporal_sec_key_1.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using datetime for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dt > 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_sec_key_2/issue_363_temporal_sec_key_2.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.1.ddl.aql
new file mode 100644
index 0000000..d3c19f4
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using time for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:time,
+tm:time,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(tm);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.2.update.aql
new file mode 100644
index 0000000..6f44d6c
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using time for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":time("03:10:00.493Z"), "tm":time("03:10:00.493Z"), "name": "John"})
+insert into dataset Employee({"id":time("20:37:19+08:00"), "tm":time("20:37:19+08:00"), "name": "Alex"})
+insert into dataset Employee({"id":time("21:39:17.948-04:00"), "tm":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_sec_key_2/issue_363_temporal_sec_key_2.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.aql
new file mode 100644
index 0000000..f0392b1
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_2/issue_363_temporal_sec_key_2.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using time for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.tm > time("07:07:07.777Z")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.1.ddl.aql
new file mode 100644
index 0000000..ab88978
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using date for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:date,
+dt:date,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dt);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.2.update.aql
new file mode 100644
index 0000000..b8d415e
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using date for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":date("2010-01-01"), "dt":date("2010-01-01"), "name": "John"})
+insert into dataset Employee({"id":date("-1912-10-11"), "dt":date("-1912-10-11"), "name": "Alex"})
+insert into dataset Employee({"id":date("0732-02-02"), "dt":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_sec_key_3/issue_363_temporal_sec_key_3.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.3.query.aql
new file mode 100644
index 0000000..128a2a6
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_3/issue_363_temporal_sec_key_3.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using date for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dt > date("2007-07-07")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.1.ddl.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.1.ddl.aql
new file mode 100644
index 0000000..c6a8836
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.1.ddl.aql
@@ -0,0 +1,19 @@
+/*
+ * Description  : create a dataset using day-time-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type Emp as open {
+id:day-time-duration,
+dur:day-time-duration,
+name:string
+}
+
+create dataset Employee(Emp) primary key id;
+create index TestSecondIndex on Employee(dt);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.2.update.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.2.update.aql
new file mode 100644
index 0000000..d5aafe9
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.2.update.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using day-time-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+
+use dataverse test;
+
+insert into dataset Employee({"id":day-time-duration("P380DT983M"), "dur":day-time-duration("P380DT983M"), "name": "John"})
+insert into dataset Employee({"id":day-time-duration("-P3829H849.392S"), "dur":day-time-duration("-P3829H849.392S"), "name": "Alex"})
+insert into dataset Employee({"id":day-time-duration("PT93847M0.392S"), "dur":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_sec_key_4/issue_363_temporal_sec_key_4.3.query.aql b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.3.query.aql
new file mode 100644
index 0000000..dbca351
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/queries/basic/issue_363_temporal_sec_key_4/issue_363_temporal_sec_key_4.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : create a dataset using day-time-duration for the secondary index 
+ * Expected Res : Success
+ * Date         : 26 May 2013
+ * Issue        : 461
+ */
+ 
+use dataverse test;
+
+for $x in dataset('Employee')
+where $x.dur > day-time-duration("P350D")
+return $x
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_0/issue363_temporal_sec_key_0.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_0/issue363_temporal_sec_key_0.1.adm
new file mode 100644
index 0000000..f1a3b14
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_0/issue363_temporal_sec_key_0.1.adm
@@ -0,0 +1 @@
+{ "id": year-month-duration("-P3Y1M"), "dur": year-month-duration("P5Y"), "name": "Alex" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm
new file mode 100644
index 0000000..1703ccd
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_1/issue363_temporal_sec_key_1.1.adm
@@ -0,0 +1 @@
+{ "id": datetime("2013-01-01T00:00:00.000Z"), "dt": 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_sec_key_2/issue363_temporal_sec_key_2.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm
new file mode 100644
index 0000000..1e32e45
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_2/issue363_temporal_sec_key_2.1.adm
@@ -0,0 +1,2 @@
+{ "id": time("12:37:19.000Z"), "tm": time("12:37:19.000Z"), "name": "Alex" }
+{ "id": time("01:39:17.948Z"), "tm": 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_sec_key_3/issue363_temporal_sec_key_3.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_3/issue363_temporal_sec_key_3.1.adm
new file mode 100644
index 0000000..fb4a286
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_3/issue363_temporal_sec_key_3.1.adm
@@ -0,0 +1 @@
+{ "id": date("2010-01-01"), "dt": 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_sec_key_4/issue363_temporal_sec_key_4.1.adm b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_4/issue363_temporal_sec_key_4.1.adm
new file mode 100644
index 0000000..dacaf15
--- /dev/null
+++ b/asterix-app/src/test/resources/metadata/results/basic/issue_363_temporal_sec_key_4/issue363_temporal_sec_key_4.1.adm
@@ -0,0 +1 @@
+{ "id": day-time-duration("P380DT16H23M"), "dur": day-time-duration("P380DT16H23M"), "name": "John" }
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/metadata/testsuite.xml b/asterix-app/src/test/resources/metadata/testsuite.xml
index 63487d4..e9ed34d 100644
--- a/asterix-app/src/test/resources/metadata/testsuite.xml
+++ b/asterix-app/src/test/resources/metadata/testsuite.xml
@@ -175,6 +175,31 @@
         <output-dir compare="Text">issue_363_temporal_key_4</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_0">
+        <output-dir compare="Text">issue_363_temporal_sec_key_0</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_1">
+        <output-dir compare="Text">issue_363_temporal_sec_key_1</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_2">
+        <output-dir compare="Text">issue_363_temporal_sec_key_2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_3">
+        <output-dir compare="Text">issue_363_temporal_sec_key_3</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="basic">
+      <compilation-unit name="issue_363_temporal_sec_key_4">
+        <output-dir compare="Text">issue_363_temporal_sec_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/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/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/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/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-issue377/query-issue377.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue377/query-issue377.3.query.aql
index 81d6cf6..02ff97d 100644
--- 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
@@ -11,6 +11,7 @@
 set simthreshold "3";
 
 for $fbu in dataset FacebookUsers
+order by $fbu.id
 return {
     "id": $fbu.id,
     "name": $fbu.name,
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/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/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/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/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-issue377/query-issue377.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue377/query-issue377.1.adm
index c3bb80f..1a7caf5 100644
--- 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
@@ -1,23 +1,18 @@
 { "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": 10001080, "similar-users": [  ], "name": "GarrettBode" }
+{ "id": 10179538, "similar-users": [  ], "name": "OrlandoBaxter" }
 { "id": 10272571, "similar-users": [  ], "name": "JarrettGoldvogel" }
+{ "id": 10307032, "similar-users": [  ], "name": "QuentinSauter" }
 { "id": 10361965, "similar-users": [  ], "name": "ArlenFlick" }
+{ "id": 10394488, "similar-users": [  ], "name": "OswaldRay" }
+{ "id": 10423588, "similar-users": [  ], "name": "ShirleneRuch" }
+{ "id": 10495420, "similar-users": [  ], "name": "WendyMcloskey" }
 { "id": 10498285, "similar-users": [  ], "name": "KileyBridger" }
 { "id": 10733617, "similar-users": [  ], "name": "LeonardoKight" }
 { "id": 10874791, "similar-users": [  ], "name": "HaydeeGarratt" }
@@ -25,5 +20,10 @@
 { "id": 11061631, "similar-users": [  ], "name": "MaxeneKellogg" }
 { "id": 11068231, "similar-users": [  ], "name": "DinahSwink" }
 { "id": 11140213, "similar-users": [  ], "name": "MontgomeryWhittier" }
+{ "id": 11307946, "similar-users": [  ], "name": "HelgaStough" }
 { "id": 11381089, "similar-users": [  ], "name": "EarleneAmmons" }
+{ "id": 11447332, "similar-users": [  ], "name": "SherisseMaugham" }
+{ "id": 11570326, "similar-users": [  ], "name": "LindenFilby" }
 { "id": 11675221, "similar-users": [  ], "name": "CalantheGearhart" }
+{ "id": 11951098, "similar-users": [  ], "name": "TeraByers" }
+{ "id": 11954992, "similar-users": [  ], "name": "CaitlinLangston" }
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 b9c6dc7..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
@@ -1,21 +1,21 @@
-2
-4
-6
-8
-10
-12
-14
-16
-18
-20
 1
+2
 3
+4
 5
+6
 7
+8
 9
+10
 11
+12
 13
+14
 15
+16
 17
+18
 19
+20
 21
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 3c52897..9e6c308 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2786,7 +2786,28 @@
     </test-case>
     <test-case FilePath="open-closed">
       <compilation-unit name="query-issue410">
-        <output-dir compare="Text">query-issue40</output-dir>
+        <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>
@@ -4334,4 +4355,16 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <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-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md b/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md
index 7e49a0f..7319094 100644
--- a/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md
+++ b/asterix-doc/src/site/markdown/AccessingExternalDataInAsterixDB.md
@@ -14,7 +14,7 @@
 _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'.
+Paste the following in the query textbox on the webpage at http://127.0.0.1:19001 and hit 'Execute'.
 
 
         create dataverse ExternalFileDemo;
@@ -100,7 +100,7 @@
         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'.
+In your web-browser, navigate to 127.0.0.1:19001 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.
 
diff --git a/asterix-doc/src/site/markdown/AsterixDataTypes.md b/asterix-doc/src/site/markdown/AsterixDBDataModel.md
similarity index 94%
rename from asterix-doc/src/site/markdown/AsterixDataTypes.md
rename to asterix-doc/src/site/markdown/AsterixDBDataModel.md
index 6674006..ffefd09 100644
--- a/asterix-doc/src/site/markdown/AsterixDataTypes.md
+++ b/asterix-doc/src/site/markdown/AsterixDBDataModel.md
@@ -23,7 +23,12 @@
 
 
 ### Int8 / Int16 / Int32 / Int64 ###
-Integer types using 8, 16, 32, or 64 bits.
+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:
 
@@ -40,7 +45,7 @@
 
 
 ### Float ###
-`Float` represents approximate numeric data values using 4 bytes.
+`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:
 
@@ -57,7 +62,7 @@
 
 
 ### Double ###
-`Double` represents approximate numeric data values using 8 bytes.
+`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:
 
diff --git a/asterix-doc/src/site/markdown/AsterixDataTypesAndFunctions.md b/asterix-doc/src/site/markdown/AsterixDBFunctions.md
similarity index 86%
rename from asterix-doc/src/site/markdown/AsterixDataTypesAndFunctions.md
rename to asterix-doc/src/site/markdown/AsterixDBFunctions.md
index 7576648..d47752c 100644
--- a/asterix-doc/src/site/markdown/AsterixDataTypesAndFunctions.md
+++ b/asterix-doc/src/site/markdown/AsterixDBFunctions.md
@@ -1,290 +1,3 @@
-# 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.
-
- * 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.
-
- * 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.
-
- * 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]}}
-
-
 # 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.
 
diff --git a/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md b/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md
index c9ed3e2..1ed876e 100644
--- a/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md
+++ b/asterix-doc/src/site/markdown/AsterixQueryLanguageReference.md
@@ -3,38 +3,79 @@
 
 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
+    PrimaryExpr ::= Literal
+                  | VariableRef
+                  | ParenthesizedExpression
+                  | FunctionCallExpr
+                  | DatasetAccessExpression
+                  | ListConstructor
+                  | RecordConstructor
+                  
 
 #### Literals
 
-    Literal ::= StringLiteral | <INTEGER_LITERAL> | <FLOAT_LITERAL> | <DOUBLE_LITERAL> | <NULL> | <TRUE> | <FALSE>
+    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 ::= <LEFTPAREN> Expression <RIGHTPAREN>
+    ParenthesizedExpression ::= "(" Expression ")"
+
+##### Example
+
+    ( 1 + 1 )
+
 
 #### Function Calls
 
-    FunctionCallExpr ::= FunctionOrTypeName <LEFTPAREN> ( Expression ( "," Expression )* )? <RIGHTPAREN>
-    
+    FunctionCallExpr ::= FunctionOrTypeName "(" ( Expression ( "," Expression )* )? ")"
+
+##### Example
+
+    string-length("a string")
+
+
 #### Dataset Access
 
-    DatasetAccessExpression ::= <DATASET> ( ( Identifier ( "." Identifier )? )
-                              | ( <LEFTPAREN> Expression ( "," Expression )* <RIGHTPAREN> ) )
+    DatasetAccessExpression ::= "dataset" ( ( Identifier ( "." Identifier )? )
+                              | ( "(" Expression ")" ) )
     Identifier              ::= <IDENTIFIER> | StringLiteral
 
+##### Examples
+
+    dataset customers
+    dataset (string-join("customers", $country))
+    
+
 #### Constructors
 
     ListConstructor          ::= ( OrderedListConstructor | UnorderedListConstructor )
@@ -43,20 +84,51 @@
     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
 
@@ -64,6 +136,11 @@
     MultExpr ::= UnaryExpr ( ( "*" | "/" | "%" | <CARET> | "idiv" ) UnaryExpr )*
     UnaryExpr ::= ( ( "+" | "-" ) )? ValueExpr
 
+##### Example
+
+    3 ^ 2 + 4 ^ 2
+
+
 ###  FLWOGR Expression   
     
     FLWOGR         ::= ( ForClause | LetClause ) ( Clause )* "return" Expression
@@ -81,16 +158,99 @@
     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" <LEFTPAREN> Expression <RIGHTPAREN> "then" Expression "else" 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
 
@@ -103,7 +263,6 @@
                       | SetStatement
                       | InsertStatement
                       | DeleteStatement
-                      | FeedStatement
                       | Query
     
 ### Declarations    
@@ -111,21 +270,48 @@
     DataverseDeclaration ::= "use" "dataverse" Identifier
     SetStatement         ::= "set" Identifier StringLiteral
     FunctionDeclaration  ::= "declare" "function" Identifier ParameterList "{" Expression "}"
-    ParameterList        ::= <LEFTPAREN> ( <VARIABLE> ( "," <VARIABLE> )* )? <RIGHTPAREN>
+    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" ( TypeSpecification | DatasetSpecification | IndexSpecification | DataverseSpecification | FunctionSpecification )
+    CreateStatement ::= "create" ( DataverseSpecification
+                                 | TypeSpecification
+                                 | DatasetSpecification
+                                 | IndexSpecification
+                                 | FunctionSpecification )
 
-    DropStatement       ::= "drop" ( <DATASET> QualifiedName IfExists
-                                   | "index" DoubleQualifiedName IfExists
-                                   | "type" FunctionOrTypeName IfExists
-                                   | "dataverse" Identifier IfExists
-                                   | "function" FunctionSignature IfExists )
-    IfExists            ::= ( "if" "exists" )?
     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
@@ -137,55 +323,140 @@
     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 ::= "external" <DATASET> QualifiedName <LEFTPAREN> Identifier <RIGHTPAREN> IfNotExists 
-                                 "using" AdapterName Configuration ( "hints" Properties )? 
-                           | "feed" <DATASET> QualifiedName <LEFTPAREN> Identifier <RIGHTPAREN> IfNotExists
-                                 "using" AdapterName Configuration ( ApplyFunction )? PrimaryKey ( "on" Identifier )? ( "hints" Properties )? 
-                           | "internal"? <DATASET> QualifiedName <LEFTPAREN> Identifier <RIGHTPAREN> IfNotExists
-                             PrimaryKey ( "on" Identifier )? ( "hints" Properties )?
+    DatasetSpecification ::= "internal"? "dataset" QualifiedName "(" Identifier ")" IfNotExists
+                             PrimaryKey ( "on" Identifier )? ( "hints" Properties )? 
+                           | "external" "dataset" QualifiedName "(" Identifier ")" IfNotExists 
+                             "using" AdapterName Configuration ( "hints" Properties )?
     AdapterName          ::= Identifier
-    Configuration        ::= <LEFTPAREN> ( KeyValuePair ( "," KeyValuePair )* )? <RIGHTPAREN>
-    KeyValuePair         ::= <LEFTPAREN> StringLiteral "=" StringLiteral <RIGHTPAREN>
-    Properties           ::= ( <LEFTPAREN> Property ( "," Property )* <RIGHTPAREN> )?
+    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 <LEFTPAREN> ( Identifier ) ( "," Identifier )* <RIGHTPAREN> ( "type" IndexType )?
-    IndexType          ::= "btree" | "rtree" | "keyword" | "fuzzy keyword" | "ngram" <LEFTPAREN> <INTEGER_LITERAL> <RIGHTPAREN> | "fuzzy ngram" <LEFTPAREN> <INTEGER_LITERAL> <RIGHTPAREN>
+    IndexSpecification ::= "index" Identifier IfNotExists "on" QualifiedName 
+                           "(" ( Identifier ) ( "," Identifier )* ")" ( "type" IndexType )?
+    IndexType          ::= "btree"
+                         | "rtree"
+                         | "keyword"
+                         | "fuzzy keyword"
+                         | "ngram" "(" <INTEGER_LITERAL> ")"
+                         | "fuzzy ngram" "(" <INTEGER_LITERAL> ")"
 
-#### Dataverses
+##### Example
 
-    DataverseSpecification ::= "dataverse" Identifier IfNotExists ( "with format" StringLiteral )?
+    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 "}"    
+    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" )?
+    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 )?
+    InsertStatement ::= "insert" "into" "dataset" QualifiedName Query
+    DeleteStatement ::= "delete" Variable "from" "dataset" QualifiedName ( "where" Expression )?
+    
+##### Example
 
-### Feed Management Statements
+    insert into dataset UsersCopy (for $user in dataset FacebookUsers return $user)
 
-    FeedStatement ::= "begin" "feed" QualifiedName
-                    | "suspend" "feed" QualifiedName
-                    | "resume" "feed" QualifiedName
-                    | "end" "feed" QualifiedName
-                    | "alter" "feed" QualifiedName "set" Configuration
+##### 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/index.md b/asterix-doc/src/site/markdown/index.md
new file mode 100644
index 0000000..cba2fdc
--- /dev/null
+++ b/asterix-doc/src/site/markdown/index.md
@@ -0,0 +1 @@
+# AsterixDB
diff --git a/asterix-doc/src/site/site.xml b/asterix-doc/src/site/site.xml
new file mode 100644
index 0000000..6724153
--- /dev/null
+++ b/asterix-doc/src/site/site.xml
@@ -0,0 +1,46 @@
+<?xml version="1.0" encoding="ISO-8859-1"?>
+
+<project name="AsterixDB" xmlns="http://maven.apache.org/DECORATION/1.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/DECORATION/1.0.0 http://maven.apache.org/xsd/decoration-1.0.0.xsd">
+  <!--
+  <bannerLeft>
+    <name>Asterix</name>
+    <src>http://asterix.ics.uci.edu/pic/img9.jpg</src>
+    <href>http://asterix.ics.uci.edu/</href>
+  </bannerLeft>
+  -->
+
+  <skin>
+    <groupId>org.apache.maven.skins</groupId>
+    <artifactId>maven-fluido-skin</artifactId>
+    <version>1.2.1</version>
+  </skin>
+  <custom>
+    <fluidoSkin>
+      <sideBarEnabled>true</sideBarEnabled>
+      <!-- <googlePlusOne /> -->
+    </fluidoSkin>
+  </custom>
+
+  <body>
+    <links>
+      <item name="Home" href="index.html"/>
+    </links>
+
+    <menu name="Documentation">
+      <item name="AsterixDB: A Big Data Management System" href="AsterixAlphaRelease.html"/>
+      <item name="Installing Asterix using Managix" href="InstallingAsterixUsingManagix.html"/>
+      <item name="AsterixDB 101: An ADM and AQL Primer" href="AdmAql101.html"/>
+      <item name="Asterix Data Model (ADM)" href="AsterixDBDataModel.html"/>
+      <item name="AsterixDB Functions" href="AsterixDBFunctions.html"/>
+      <item name="The Asterix Query Language" href="AsterixQueryLanguageReference.html"/>
+      <item name="AsterixDB Support of Similarity Queries" href="AsterixSimilarityQueries.html"/>
+      <item name="Accessing External Data in AsterixDB" href="AccessingExternalDataInAsterixDB.html"/>
+      <item name="REST API to AsterixDB" href="AsterixDBRestAPI.html"/>
+      <item name="(old AQL doc)" href="AsterixQueryLanguage.html"/>      
+    </menu>
+
+    <menu ref="reports"/>
+  </body>
+</project>
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
index e4c7ba2..0ed3c78 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlDataSource.java
@@ -17,7 +17,6 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
@@ -28,6 +27,7 @@
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.ListSet;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSource;
 import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IDataSourcePropertiesProvider;
@@ -219,7 +219,7 @@
                     if (n < 2) {
                         pp = new RandomPartitioningProperty(domain);
                     } else {
-                        Set<LogicalVariable> pvars = new HashSet<LogicalVariable>();
+                        Set<LogicalVariable> pvars = new ListSet<LogicalVariable>();
                         int i = 0;
                         for (LogicalVariable v : scanVariables) {
                             pvars.add(v);
@@ -239,7 +239,7 @@
                     if (n < 2) {
                         pp = new RandomPartitioningProperty(domain);
                     } else {
-                        Set<LogicalVariable> pvars = new HashSet<LogicalVariable>();
+                        Set<LogicalVariable> pvars = new ListSet<LogicalVariable>();
                         int i = 0;
                         for (LogicalVariable v : scanVariables) {
                             pvars.add(v);
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/AObjectAscBinaryComparatorFactory.java b/asterix-om/src/main/java/edu/uci/ics/asterix/dataflow/data/nontagged/comparators/AObjectAscBinaryComparatorFactory.java
index 865ab94..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
@@ -62,8 +62,13 @@
                         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);
                     }
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 495b41f..0946f1b 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
@@ -1,14 +1,8 @@
 package edu.uci.ics.asterix.dataflow.data.nontagged.hash;
 
-import edu.uci.ics.asterix.om.types.ATypeTag;
-import edu.uci.ics.asterix.om.types.EnumDeserializer;
 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;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class AObjectBinaryHashFunctionFactory implements IBinaryHashFunctionFactory {
 
@@ -22,56 +16,12 @@
     @Override
     public IBinaryHashFunction createBinaryHashFunction() {
         return new IBinaryHashFunction() {
-
-            private IBinaryHashFunction boolHash = BooleanBinaryHashFunctionFactory.INSTANCE.createBinaryHashFunction();
-            private IBinaryHashFunction intHash = new PointableBinaryHashFunctionFactory(IntegerPointable.FACTORY)
-                    .createBinaryHashFunction();
-            private IBinaryHashFunction longHash = LongBinaryHashFunctionFactory.INSTANCE.createBinaryHashFunction();
-            private IBinaryHashFunction floatHash = new PointableBinaryHashFunctionFactory(FloatPointable.FACTORY)
-                    .createBinaryHashFunction();
-            private IBinaryHashFunction stringHash = new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY)
-                    .createBinaryHashFunction();
-
-            private IBinaryHashFunction doubleHash = DoubleBinaryHashFunctionFactory.INSTANCE
-                    .createBinaryHashFunction();
-
             private IBinaryHashFunction genericBinaryHash = MurmurHash3BinaryHashFunctionFamily.INSTANCE
                     .createBinaryHashFunction(0);
 
             @Override
             public int hash(byte[] bytes, int offset, int length) {
-                ATypeTag tag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(bytes[offset]);
-                switch (tag) {
-                    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);
-                    }
-                    case FLOAT: {
-                        return floatHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case DOUBLE: {
-                        return doubleHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case STRING: {
-                        return stringHash.hash(bytes, offset + 1, length - 1);
-                    }
-                    case NULL: {
-                        return 0;
-                    }
-                    default: {
-                        return genericBinaryHash.hash(bytes, offset + 1, length - 1);
-                    }
-                }
+                return genericBinaryHash.hash(bytes, offset, length);
             }
         };
     }
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/AqlBinaryHashFunctionFactoryProvider.java b/asterix-om/src/main/java/edu/uci/ics/asterix/formats/nontagged/AqlBinaryHashFunctionFactoryProvider.java
index 33698bd..76dd07d 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
@@ -2,11 +2,6 @@
 
 import java.io.Serializable;
 
-import edu.uci.ics.asterix.dataflow.data.nontagged.hash.AObjectBinaryHashFunctionFactory;
-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.om.types.IAType;
 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;
@@ -16,7 +11,6 @@
 import edu.uci.ics.hyracks.data.std.primitive.FloatPointable;
 import edu.uci.ics.hyracks.data.std.primitive.IntegerPointable;
 import edu.uci.ics.hyracks.data.std.primitive.RawUTF8StringPointable;
-import edu.uci.ics.hyracks.data.std.primitive.UTF8StringPointable;
 
 public class AqlBinaryHashFunctionFactoryProvider implements IBinaryHashFunctionFactoryProvider, Serializable {
 
@@ -39,81 +33,6 @@
 
     @Override
     public IBinaryHashFunctionFactory getBinaryHashFunctionFactory(Object type) {
-        if (type == null) {
-            return AObjectBinaryHashFunctionFactory.INSTANCE;
-        }
-        IAType aqlType = (IAType) type;
-        switch (aqlType.getTypeTag()) {
-            case ANY:
-            case UNION: { // we could do smth better for nullable fields
-                return AObjectBinaryHashFunctionFactory.INSTANCE;
-            }
-            case NULL: {
-                return new IBinaryHashFunctionFactory() {
-
-                    private static final long serialVersionUID = 1L;
-
-                    @Override
-                    public IBinaryHashFunction createBinaryHashFunction() {
-                        return new IBinaryHashFunction() {
-
-                            @Override
-                            public int hash(byte[] bytes, int offset, int length) {
-                                return 0;
-                            }
-                        };
-                    }
-                };
-            }
-            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);
-            }
-            case FLOAT: {
-                return addOffset(new PointableBinaryHashFunctionFactory(FloatPointable.FACTORY));
-            }
-            case DOUBLE: {
-                return addOffset(DoubleBinaryHashFunctionFactory.INSTANCE);
-            }
-            case STRING: {
-                return addOffset(new PointableBinaryHashFunctionFactory(UTF8StringPointable.FACTORY));
-            }
-            default: {
-                return addOffsetForGenericBinaryHash();
-            }
-        }
-    }
-
-    private IBinaryHashFunctionFactory addOffset(final IBinaryHashFunctionFactory inst) {
-        return new IBinaryHashFunctionFactory() {
-
-            private static final long serialVersionUID = 1L;
-
-            @Override
-            public IBinaryHashFunction createBinaryHashFunction() {
-                final IBinaryHashFunction bhf = inst.createBinaryHashFunction();
-                return new IBinaryHashFunction() {
-
-                    @Override
-                    public int hash(byte[] bytes, int offset, int length) {
-                        return bhf.hash(bytes, offset + 1, length);
-                    }
-                };
-            }
-        };
-    }
-
-    private IBinaryHashFunctionFactory addOffsetForGenericBinaryHash() {
         return new IBinaryHashFunctionFactory() {
 
             private static final long serialVersionUID = 1L;
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 38af5e1..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
@@ -29,6 +29,7 @@
 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;
@@ -86,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;
@@ -620,22 +620,8 @@
         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);
         add(COUNT, AInt32TypeComputer.INSTANCE);
         add(COUNTHASHED_GRAM_TOKENS, OrderedListOfAInt32TypeComputer.INSTANCE);
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/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-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)