merged back master e4baf34
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/base/RuleCollections.java
index f14fff8..d5169c7 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
@@ -36,6 +36,7 @@
 import edu.uci.ics.asterix.optimizer.rules.IntroduceStaticTypeCastRule;
 import edu.uci.ics.asterix.optimizer.rules.LoadRecordFieldsRule;
 import edu.uci.ics.asterix.optimizer.rules.NestGroupByRule;
+import edu.uci.ics.asterix.optimizer.rules.NestedSubplanToJoinRule;
 import edu.uci.ics.asterix.optimizer.rules.PullPositionalVariableFromUnnestRule;
 import edu.uci.ics.asterix.optimizer.rules.PushAggFuncIntoStandaloneAggregateRule;
 import edu.uci.ics.asterix.optimizer.rules.PushAggregateIntoGroupbyRule;
@@ -192,6 +193,7 @@
         consolidation.add(new CountVarToCountOneRule());
         consolidation.add(new RemoveUnusedAssignAndAggregateRule());
         consolidation.add(new RemoveRedundantGroupByDecorVars());
+        consolidation.add(new NestedSubplanToJoinRule());
         return consolidation;
     }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
index 2dce5f6..5cff07b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceDynamicTypeCastRule.java
@@ -24,6 +24,7 @@
 import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.metadata.declared.AqlDataSource;
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
 import edu.uci.ics.asterix.om.types.ARecordType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -120,11 +121,8 @@
         // assign
         AbstractFunctionCallExpression cast = new ScalarFunctionCallExpression(
                 FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_RECORD));
-        ARecordType[] types = new ARecordType[2];
-        types[0] = requiredRecordType;
-        types[1] = inputRecordType;
         cast.getArguments().add(new MutableObject<ILogicalExpression>(new VariableReferenceExpression(inputRecordVar)));
-        cast.setOpaqueParameters(types);
+        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));
@@ -142,6 +140,10 @@
         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);
         return true;
     }
 
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index c99e4bc..6985753 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -64,10 +64,16 @@
         }
 
         FunctionIdentifier fid = null;
+        /** find the record variable */
+        InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
+        ILogicalExpression recordExpr = insertOp.getPayloadExpression().getValue();
+        List<LogicalVariable> recordVar = new ArrayList<LogicalVariable>();
+        /** assume the payload is always a single variable expression */
+        recordExpr.getUsedVariables(recordVar);
+
         /** op2 is the assign operator which extract primary keys from the record variable */
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op1.getInputs().get(0).getValue();
-        List<LogicalVariable> recordVar = new ArrayList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(op2, recordVar);
+
         if (recordVar.size() == 0) {
             /**
              * For the case primary key-assignment expressions are constant expressions,
@@ -92,7 +98,6 @@
             AssignOperator assignOp2 = (AssignOperator) op2;
             recordVar.addAll(assignOp2.getVariables());
         }
-        InsertDeleteOperator insertOp = (InsertDeleteOperator) op1;
         AqlDataSource datasetSource = (AqlDataSource) insertOp.getDataSource();
         AqlMetadataProvider mp = (AqlMetadataProvider) context.getMetadataProvider();
         String dataverseName = datasetSource.getId().getDataverseName();
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
index 3aae2dd..f42782b 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/IntroduceStaticTypeCastRule.java
@@ -42,7 +42,7 @@
 /**
  * Statically cast a constant from its type to a specified required type, in a
  * recursive way. It enables: 1. bag-based fields in a record, 2. bidirectional
- * cast of a open field and a matched closed field, and 3. put in null fields
+ * cast of an open field and a matched closed field, and 3. put in null fields
  * when necessary. It should be fired before the constant folding rule.
  * This rule is not responsible for type casting between primitive types.
  * Here is an example: A record { "hobby": {{"music", "coding"}}, "id": "001",
@@ -90,9 +90,6 @@
         InsertDeleteOperator insertDeleteOp = (InsertDeleteOperator) op2;
         if (insertDeleteOp.getOperation() == InsertDeleteOperator.Kind.DELETE)
             return false;
-        AbstractLogicalOperator assignOp = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
-        if (assignOp.getOperatorTag() != LogicalOperatorTag.ASSIGN)
-            return false;
         /**
          * get required record type
          */
@@ -101,21 +98,21 @@
         IAType[] schemaTypes = (IAType[]) dataSource.getSchemaTypes();
         IAType requiredRecordType = schemaTypes[schemaTypes.length - 1];
 
-        AssignOperator topAssignOperator = (AssignOperator) assignOp;
         List<LogicalVariable> usedVariables = new ArrayList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(topAssignOperator, usedVariables);
+        insertDeleteOperator.getPayloadExpression().getValue().getUsedVariables(usedVariables);
 
         // the used variable should contain the record that will be inserted
         // but it will not fail in many cases even if the used variable set is
         // empty
         if (usedVariables.size() == 0)
             return false;
+
         oldRecordVariable = usedVariables.get(0);
         LogicalVariable inputRecordVar = usedVariables.get(0);
-        IVariableTypeEnvironment env = topAssignOperator.computeOutputTypeEnvironment(context);
+        IVariableTypeEnvironment env = insertDeleteOperator.computeOutputTypeEnvironment(context);
         IAType inputRecordType = (IAType) env.getVarType(inputRecordVar);
 
-        AbstractLogicalOperator currentOperator = assignOp;
+        AbstractLogicalOperator currentOperator = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
         /**
          * find the assign operator for the "input record" to the insert_delete
          * operator
@@ -123,6 +120,7 @@
         do {
             context.addToDontApplySet(this, currentOperator);
             if (currentOperator.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                AssignOperator assignOp = (AssignOperator) currentOperator;
                 producedVariables.clear();
                 VariableUtilities.getProducedVariables(currentOperator, producedVariables);
                 int position = producedVariables.indexOf(oldRecordVariable);
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
new file mode 100644
index 0000000..c5d415e
--- /dev/null
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/NestedSubplanToJoinRule.java
@@ -0,0 +1,131 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.operators.logical.LeftOuterJoinOperator;
+import edu.uci.ics.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
+import edu.uci.ics.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * replace Subplan operators with nested loop joins where the join condition is true, if the Subplan
+ * does not contain free variables (does not have correlations to the input stream).
+ * 
+ * @author yingyib
+ */
+public class NestedSubplanToJoinRule implements IAlgebraicRewriteRule {
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context) throws AlgebricksException {
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        if (context.checkIfInDontApplySet(this, opRef.getValue()))
+            return false;
+        context.addToDontApplySet(this, opRef.getValue());
+
+        ILogicalOperator op1 = opRef.getValue();
+        if (op1.getInputs().size() == 0) {
+            return false;
+        }
+
+        boolean rewritten = false;
+        for (int index = 0; index < op1.getInputs().size(); index++) {
+            AbstractLogicalOperator child = (AbstractLogicalOperator) op1.getInputs().get(index).getValue();
+            if (child.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+                continue;
+            }
+
+            AbstractOperatorWithNestedPlans subplan = (AbstractOperatorWithNestedPlans) child;
+            Set<LogicalVariable> freeVars = new HashSet<LogicalVariable>();
+            OperatorPropertiesUtil.getFreeVariablesInSubplans(subplan, freeVars);
+            if (!freeVars.isEmpty()) {
+                /**
+                 * the subplan is correlated with the outer plan, other rules can deal with it
+                 */
+                continue;
+            }
+
+            /** get the input operator of the subplan operator */
+            ILogicalOperator subplanInput = subplan.getInputs().get(0).getValue();
+
+            /** get all nested top operators */
+            List<ILogicalPlan> nestedPlans = subplan.getNestedPlans();
+            List<Mutable<ILogicalOperator>> nestedRoots = new ArrayList<Mutable<ILogicalOperator>>();
+            for (ILogicalPlan nestedPlan : nestedPlans) {
+                nestedRoots.addAll(nestedPlan.getRoots());
+            }
+            if (nestedRoots.size() == 0) {
+                /** there is no nested top operators */
+                return false;
+            }
+
+            /** expend the input and roots into a DAG of nested loop joins */
+            Mutable<ILogicalExpression> expr = new MutableObject<ILogicalExpression>(ConstantExpression.TRUE);
+            Mutable<ILogicalOperator> nestedRootRef = nestedRoots.get(0);
+            ILogicalOperator join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(subplanInput),
+                    nestedRootRef);
+
+            /** rewrite the nested tuple source to be empty tuple source */
+            rewriteNestedTupleSource(nestedRootRef);
+
+            for (int i = 1; i < nestedRoots.size(); i++) {
+                join = new LeftOuterJoinOperator(expr, new MutableObject<ILogicalOperator>(join), nestedRoots.get(i));
+            }
+            op1.getInputs().get(index).setValue(join);
+            context.computeAndSetTypeEnvironmentForOperator(join);
+            rewritten = true;
+        }
+        return rewritten;
+    }
+
+    /**
+     * rewrite NestedTupleSource operators to EmptyTupleSource operators
+     * 
+     * @param nestedRootRef
+     */
+    private void rewriteNestedTupleSource(Mutable<ILogicalOperator> nestedRootRef) {
+        AbstractLogicalOperator nestedRoot = (AbstractLogicalOperator) nestedRootRef.getValue();
+        if (nestedRoot.getOperatorTag() == LogicalOperatorTag.NESTEDTUPLESOURCE) {
+            nestedRootRef.setValue(new EmptyTupleSourceOperator());
+        }
+        List<Mutable<ILogicalOperator>> inputs = nestedRoot.getInputs();
+        for (Mutable<ILogicalOperator> input : inputs) {
+            rewriteNestedTupleSource(input);
+        }
+    }
+}
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
index c5a1cb0..bee8c40 100644
--- a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
+++ b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
@@ -111,19 +111,7 @@
         
         // The assign now just "renames" the variable to make sure the upstream plan still works.
         srcAssignExprRef.setValue(new VariableReferenceExpression(aggVar));
-
-        // Create a new assign for a TRUE variable.
-        LogicalVariable trueVar = context.newVar();
-        AssignOperator trueAssignOp = new AssignOperator(trueVar, new MutableObject<ILogicalExpression>(ConstantExpression.TRUE));
         
-        ILogicalOperator aggInput = aggOp.getInputs().get(0).getValue();
-        aggOp.getInputs().get(0).setValue(trueAssignOp);
-        trueAssignOp.getInputs().add(new MutableObject<ILogicalOperator>(aggInput));
-        
-        // Set partitioning variable.
-        aggOp.setPartitioningVariable(trueVar);
-        
-        context.computeAndSetTypeEnvironmentForOperator(trueAssignOp);
         context.computeAndSetTypeEnvironmentForOperator(aggOp);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
         
diff --git a/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterix-algebra/src/main/java/edu/uci/ics/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index 3ba3e96..6cb4d5c 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
@@ -22,6 +22,7 @@
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 
+import edu.uci.ics.asterix.aql.util.FunctionUtils;
 import edu.uci.ics.asterix.om.base.ANull;
 import edu.uci.ics.asterix.om.base.AString;
 import edu.uci.ics.asterix.om.constants.AsterixConstantValue;
@@ -39,10 +40,12 @@
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 
 /**
  * This class is utility to do type cast.
@@ -124,6 +127,11 @@
      */
     public static boolean rewriteFuncExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
             IVariableTypeEnvironment env) throws AlgebricksException {
+        /**
+         * sanity check: if there are list(ordered or unordered)/record variable expressions in the funcExpr, we will not do STATIC type casting
+         * because they are not "statically cast-able".
+         * instead, the record will be dynamically casted at the runtime
+         */
         if (funcExpr.getFunctionIdentifier() == AsterixBuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR) {
             if (reqType.equals(BuiltinType.ANY)) {
                 reqType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
@@ -399,32 +407,86 @@
         for (int i = 0; i < openFields.length; i++) {
             if (openFields[i]) {
                 arguments.add(originalArguments.get(2 * i));
-                Mutable<ILogicalExpression> fExprRef = originalArguments.get(2 * i + 1);
-                ILogicalExpression argExpr = fExprRef.getValue();
-
+                Mutable<ILogicalExpression> expRef = originalArguments.get(2 * i + 1);
+                ILogicalExpression argExpr = expRef.getValue();
+                List<LogicalVariable> parameterVars = new ArrayList<LogicalVariable>();
+                argExpr.getUsedVariables(parameterVars);
                 // we need to handle open fields recursively by their default
                 // types
                 // for list, their item type is any
                 // for record, their
-                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                boolean castInjected = false;
+                if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+                        || argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
                     IAType reqFieldType = inputFieldTypes[i];
+                    // do not enforce nested type in the case of no-used variables
                     if (inputFieldTypes[i].getTypeTag() == ATypeTag.RECORD) {
                         reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+                            //inject dynamic type casting
+                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_RECORD),
+                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
+                            castInjected = true;
+                        }
                     }
                     if (inputFieldTypes[i].getTypeTag() == ATypeTag.ORDEREDLIST) {
                         reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+                            //inject dynamic type casting
+                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
+                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
+                            castInjected = true;
+                        }
                     }
                     if (inputFieldTypes[i].getTypeTag() == ATypeTag.UNORDEREDLIST) {
                         reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+                        if (!inputFieldTypes[i].equals(reqFieldType) && parameterVars.size() > 0) {
+                            //inject dynamic type casting
+                            injectCastFunction(FunctionUtils.getFunctionInfo(AsterixBuiltinFunctions.CAST_LIST),
+                                    reqFieldType, inputFieldTypes[i], expRef, argExpr);
+                            castInjected = true;
+                        }
                     }
-                    if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null) {
-                        ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
-                        rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
+                    if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        //recursively rewrite function arguments
+                        if (TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) argExpr) == null
+                                && reqFieldType != null) {
+                            if (castInjected) {
+                                //rewrite the arg expression inside the dynamic cast
+                                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+                                rewriteFuncExpr(argFunc, inputFieldTypes[i], inputFieldTypes[i], env);
+                            } else {
+                                //rewrite arg
+                                ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
+                                rewriteFuncExpr(argFunc, reqFieldType, inputFieldTypes[i], env);
+                            }
+                        }
                     }
                 }
-                arguments.add(fExprRef);
+                arguments.add(expRef);
             }
         }
     }
 
+    /**
+     * Inject a dynamic cast function wrapping an existing expression
+     * 
+     * @param funcInfo
+     *            the cast function
+     * @param reqType
+     *            the required type
+     * @param inputType
+     *            the original type
+     * @param exprRef
+     *            the expression reference
+     * @param argExpr
+     *            the original expression
+     */
+    private static void injectCastFunction(IFunctionInfo funcInfo, IAType reqType, IAType inputType,
+            Mutable<ILogicalExpression> exprRef, ILogicalExpression argExpr) {
+        ScalarFunctionCallExpression cast = new ScalarFunctionCallExpression(funcInfo);
+        cast.getArguments().add(new MutableObject<ILogicalExpression>(argExpr));
+        exprRef.setValue(cast);
+        TypeComputerUtilities.setRequiredAndInputTypes(cast, reqType, inputType);
+    }
 }
diff --git a/asterix-app/src/test/resources/logging.properties b/asterix-app/src/test/resources/logging.properties
index 6f8cd00..f04eb3de 100644
--- a/asterix-app/src/test/resources/logging.properties
+++ b/asterix-app/src/test/resources/logging.properties
@@ -60,7 +60,8 @@
 # For example, set the com.xyz.foo logger to only log SEVERE
 # messages:
 
+
 edu.uci.ics.asterix.test.level = INFO
 #edu.uci.ics.asterix.level = FINE
-#edu.uci.ics.algebricks.level = FINE
+#edu.uci.ics.hyracks.algebricks.level = FINE
 #edu.uci.ics.hyracks.level = INFO
diff --git a/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql b/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql
new file mode 100644
index 0000000..f1b2678
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/queries/limit-issue353.aql
@@ -0,0 +1,44 @@
+/*
+ * Description  : This test case is to verify the fix for issue353
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=353
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+drop dataverse tpch if exists;
+create dataverse tpch;
+
+use dataverse tpch;
+
+create type LineItemType as closed {
+  l_orderkey: int32, 
+  l_partkey: int32, 
+  l_suppkey: int32, 
+  l_linenumber: int32, 
+  l_quantity: double, 
+  l_extendedprice: double,
+  l_discount: double, 
+  l_tax: double,
+  l_returnflag: string, 
+  l_linestatus: string, 
+  l_shipdate: string,
+  l_commitdate: string, 
+  l_receiptdate: string, 
+  l_shipinstruct: string, 
+  l_shipmode: string, 
+  l_comment: string
+}
+
+create dataset LineItem(LineItemType)
+  primary key l_orderkey, l_linenumber;
+  
+write output to nc1:"/tmp/push_limit.adm";
+
+for $l in dataset('LineItem')
+limit 2
+return {
+"l_returnflag": $l.l_returnflag,
+"l_linestatus": $l.l_linestatus,
+"l_shipmode": $l.l_shipmode
+}
+
diff --git a/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan b/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan
new file mode 100644
index 0000000..829a245
--- /dev/null
+++ b/asterix-app/src/test/resources/optimizerts/results/limit-issue353.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- STREAM_LIMIT  |UNPARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$6(ASC), $$7(ASC) ]  |PARTITIONED|
+              -- STREAM_LIMIT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
index e69de29..e6a3879 100644
--- a/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
+++ b/asterix-app/src/test/resources/runtimets/queries/misc/ifthenelse_01/ifthenelse_01.1.ddl.aql
@@ -0,0 +1,2 @@
+drop dataverse test if exists;
+create dataverse test;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql
new file mode 100644
index 0000000..59e814a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.1.ddl.aql
@@ -0,0 +1,22 @@
+/*
+ * Description  : This test case is to verify the fix for issue196
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date         : 5th May 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type testtype1 as open {
+id : int32
+}
+
+create type testtype2 as open {
+id : int32
+}
+
+create dataset t1(testtype1) primary key id;
+create dataset t2(testtype2) primary key id;
+
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql
new file mode 100644
index 0000000..18756f9
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.2.update.aql
@@ -0,0 +1,20 @@
+/*
+ * Description  : This test case is to verify the fix for issue196
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date         : 5th May 2013
+ */
+
+use dataverse test;
+
+insert into dataset t1({"id":24});
+insert into dataset t1({"id":23});
+insert into dataset t1({"id":21});
+insert into dataset t1({"id":44});
+insert into dataset t1({"id":64});
+
+insert into dataset t2({"id":24});
+insert into dataset t2({"id":23});
+insert into dataset t2({"id":21});
+insert into dataset t2({"id":44});
+insert into dataset t2({"id":64});
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql
new file mode 100644
index 0000000..43d1980
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue196/query-issue196.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue196
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=196
+ * Expected Res : Success
+ * Date         : 5th May 2013
+ */
+
+use dataverse test;
+
+let $a := (for $l in dataset('t1') order by $l.id return $l)
+let $b := (for $m in dataset('t2') order by $m.id return $m)
+return {"a":$a,"b":$b}
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql
new file mode 100644
index 0000000..a49658e
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.1.ddl.aql
@@ -0,0 +1,47 @@
+/*
+ * Description  : This test case is to verify the fix for issue343.  It is a more general case.
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32,
+  street: string,
+  city: string
+}
+
+create type AllType as open {
+  id: int32,
+  name: string,
+  age: float,
+  salary: double,
+  married: boolean,
+  interests: {{string}},
+  children: [string],
+  address: AddressType,
+  dob: date,
+  time: time,
+  datetime: datetime,
+  duration: duration,
+  location2d: point,
+  location3d: point3d,
+  line: line,
+  polygon: polygon,
+  circle: circle
+}
+
+create type MyListType as open{
+	id: int32,
+	mylist: [string]
+}
+
+create dataset All(AllType)
+  primary key id;
+  
+create dataset MyList(MyListType)
+  primary key id;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql
new file mode 100644
index 0000000..39b85f3
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.2.update.aql
@@ -0,0 +1,21 @@
+/*
+ * Description  : This test case is to verify the fix for issue343.  It is a more general case.
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+insert into dataset MyList (
+ {
+   "id" : 1,
+   "mylist": ["blah"]
+ }
+);
+
+insert into dataset All (
+for $m in dataset MyList
+let $record:= { "id": 13, "name": string("Nancy"), "age": 32.5f, "salary": 12.000 ,"married": boolean("true"), "interests": {{"reading", "writing"}}, "children": ["Brad", "Scott"],  "address": {  "number": 8389,  "street": "Hill St.",  "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30Z"), "datetime": datetime("-1951-12-27T12:20:30"),  "duration": duration("P10Y11M12DT10H50M30S"),  "location2d": point("41.00,44.00"),  "location3d": point3d("44.00,13.00,41.00"), "line" : line("10.1,11.1 10.2,11.2"), "polygon" : polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle" : circle("10.1,11.1 10.2"), "mylist" : $m.mylist }
+return $record
+);
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql
new file mode 100644
index 0000000..e17a6e0
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343-2/query-issue343-2.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue343.  It is a more general case.
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+for $x in dataset All
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql
new file mode 100644
index 0000000..bd2ab1a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.1.ddl.aql
@@ -0,0 +1,41 @@
+/*
+ * Description  : This test case is to verify the fix for issue343
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type AddressType as open {
+  number: int32,
+  street: string,
+  city: string
+}
+
+create type AllType as open {
+  id: int32,
+  name: string,
+  age: float,
+  salary: double,
+  married: boolean,
+  interests: {{string}},
+  children: [string],
+  address: AddressType,
+  dob: date,
+  time: time,
+  datetime: datetime,
+  duration: duration,
+  location2d: point,
+  location3d: point3d,
+  line: line,
+  polygon: polygon,
+  circle: circle
+}
+
+create dataset All(AllType)
+  primary key id;
+  
+ 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql
new file mode 100644
index 0000000..eaafddd
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.2.update.aql
@@ -0,0 +1,14 @@
+/*
+ * Description  : This test case is to verify the fix for issue343
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+insert into dataset All (
+let $addedList := ["blah"]
+let $record:= { "id": 13, "name": string("Nancy"), "age": 32.5f, "salary": 12.000 ,"married": boolean("true"), "interests": {{"reading", "writing"}}, "children": ["Brad", "Scott"],  "address": {  "number": 8389,  "street": "Hill St.",  "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30Z"), "datetime": datetime("-1951-12-27T12:20:30"),  "duration": duration("P10Y11M12DT10H50M30S"),  "location2d": point("41.00,44.00"),  "location3d": point3d("44.00,13.00,41.00"), "line" : line("10.1,11.1 10.2,11.2"), "polygon" : polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle" : circle("10.1,11.1 10.2"), "mylist" : $addedList }
+return $record
+)
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql
new file mode 100644
index 0000000..9dfde51
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue343/query-issue343.3.query.aql
@@ -0,0 +1,11 @@
+/*
+ * Description  : This test case is to verify the fix for issue343
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=343
+ * Expected Res : Success
+ * Date         : 30th April 2013
+ */
+
+use dataverse test;
+
+for $x in dataset All
+return $x
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql
new file mode 100644
index 0000000..85f2160
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TweetMessageType as open {
+tweetid: string
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid; 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql
new file mode 100644
index 0000000..372d7ca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.2.update.aql
@@ -0,0 +1,50 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+load dataset TweetMessages
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/twitter/tw_messages.adm"),("format"="adm"));
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"13",
+    "user":
+        {"screen-name":"NathanGiesen@211",
+         "lang":"en",
+         "friends_count":39345,
+         "statuses_count":479,
+         "name":"Nathan Giesen",
+         "followers_count":49420,
+         "hobbies":["basket weaving","mud wrestling"]
+        },
+    "sender-location":point("47.44,80.65"),
+    "send-time":datetime("2008-04-26T10:10:35"),
+    "referred-topics":{{"tweeting"}},
+    "message-text":"tweety tweet, my fellow tweeters!"
+   }
+);
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"15",
+    "user":
+        {"screen-name":"Jason17",
+         "lang":"en",
+         "friends_count":393,
+         "statuses_count":47,
+         "name":"Nathan Giesen",
+         "followers_count":420,
+         "hobbies":["swimming"]
+        },
+    "sender-location":point("49.44,80.65"),
+    "send-time":datetime("2009-04-26T10:10:35"),
+    "referred-topics":{{"nothing"}},
+    "message-text":"Nothing to say!"
+   }
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql
new file mode 100644
index 0000000..298940a
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350-2/query-issue350-2.3.query.aql
@@ -0,0 +1,13 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+for $tm  in dataset TweetMessages
+where (every $h in $tm.user.hobbies satisfies $h = "basket weaving")
+order by $tm.tweetid
+return $tm;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql
new file mode 100644
index 0000000..85f2160
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.1.ddl.aql
@@ -0,0 +1,17 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+drop dataverse TinySocial if exists;
+create dataverse TinySocial;
+use dataverse TinySocial;
+
+create type TweetMessageType as open {
+tweetid: string
+};
+
+create dataset TweetMessages(TweetMessageType)
+primary key tweetid; 
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql
new file mode 100644
index 0000000..372d7ca
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.2.update.aql
@@ -0,0 +1,50 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+load dataset TweetMessages
+using "edu.uci.ics.asterix.external.dataset.adapter.NCFileSystemAdapter"
+(("path"="nc1://data/twitter/tw_messages.adm"),("format"="adm"));
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"13",
+    "user":
+        {"screen-name":"NathanGiesen@211",
+         "lang":"en",
+         "friends_count":39345,
+         "statuses_count":479,
+         "name":"Nathan Giesen",
+         "followers_count":49420,
+         "hobbies":["basket weaving","mud wrestling"]
+        },
+    "sender-location":point("47.44,80.65"),
+    "send-time":datetime("2008-04-26T10:10:35"),
+    "referred-topics":{{"tweeting"}},
+    "message-text":"tweety tweet, my fellow tweeters!"
+   }
+);
+
+insert into dataset TweetMessages
+(
+   {"tweetid":"15",
+    "user":
+        {"screen-name":"Jason17",
+         "lang":"en",
+         "friends_count":393,
+         "statuses_count":47,
+         "name":"Nathan Giesen",
+         "followers_count":420,
+         "hobbies":["swimming"]
+        },
+    "sender-location":point("49.44,80.65"),
+    "send-time":datetime("2009-04-26T10:10:35"),
+    "referred-topics":{{"nothing"}},
+    "message-text":"Nothing to say!"
+   }
+);
diff --git a/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql
new file mode 100644
index 0000000..79a620b
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue350/query-issue350.3.query.aql
@@ -0,0 +1,12 @@
+/*
+ * Description  : This test case is to verify the fix for issue350
+ 				: https://code.google.com/p/asterixdb/issues/detail?id=350
+ * Expected Res : Success
+ * Date         : 28th April 2013
+ */
+
+use dataverse TinySocial;
+
+for $tm  in dataset TweetMessages
+where (some $h in $tm.user.hobbies satisfies $h = "basket weaving")
+return $tm;
\ No newline at end of file
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm
new file mode 100644
index 0000000..f93766f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue196/query-issue196.1.adm
@@ -0,0 +1 @@
+{ "a": [ { "id": 21 }, { "id": 23 }, { "id": 24 }, { "id": 44 }, { "id": 64 } ], "b": [ { "id": 21 }, { "id": 23 }, { "id": 24 }, { "id": 44 }, { "id": 64 } ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
new file mode 100644
index 0000000..5196a0f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343-2/query-issue343-2.1.adm
@@ -0,0 +1 @@
+{ "id": 13, "name": "Nancy", "age": 32.5f, "salary": 12.0d, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
new file mode 100644
index 0000000..5196a0f
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue343/query-issue343.1.adm
@@ -0,0 +1 @@
+{ "id": 13, "name": "Nancy", "age": 32.5f, "salary": 12.0d, "married": true, "interests": {{ "reading", "writing" }}, "children": [ "Brad", "Scott" ], "address": { "number": 8389, "street": "Hill St.", "city": "Mountain View" }, "dob": date("-2011-01-27"), "time": time("12:20:30.000Z"), "datetime": datetime("-1951-12-27T12:20:30.000Z"), "duration": duration("P10Y11M12DT10H50M30S"), "location2d": point("41.0,44.0"), "location3d": point3d("44.0,13.0,41.0"), "line": line("10.1,11.1 10.2,11.2"), "polygon": polygon("1.2,1.3 2.1,2.5 3.5,3.6 4.6,4.8"), "circle": circle("10.1,11.1 10.2"), "mylist": [ "blah" ] }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
new file mode 100644
index 0000000..6466feb
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350-2/query-issue350-2.1.adm
@@ -0,0 +1,10 @@
+{ "tweetid": "1", "tweetid-copy": "1", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstein", "followers_count": 3311368 }, "sender-location": point("42.13,80.43"), "send-time": datetime("2005-12-05T21:06:41.000Z"), "send-time-copy": datetime("2005-12-05T21:06:41.000Z"), "referred-topics": {{ "samsung", "plan" }}, "message-text": " love samsung the plan is amazing" }
+{ "tweetid": "10", "tweetid-copy": "10", "user": { "screen-name": "Rolldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstful", "followers_count": 3311368 }, "sender-location": point("46.94,93.98"), "send-time": datetime("2011-04-07T14:08:46.000Z"), "send-time-copy": datetime("2011-04-07T14:08:46.000Z"), "referred-topics": {{ "t-mobile", "signal" }}, "message-text": " like t-mobile the signal is good" }
+{ "tweetid": "2", "tweetid-copy": "2", "user": { "screen-name": "RollandEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "David Eckhardstein", "followers_count": 3311368 }, "sender-location": point("28.86,70.44"), "send-time": datetime("2007-08-15T06:44:17.000Z"), "send-time-copy": datetime("2007-08-15T06:44:17.000Z"), "referred-topics": {{ "sprint", "voice-clarity" }}, "message-text": " like sprint its voice-clarity is mind-blowing" }
+{ "tweetid": "3", "tweetid-copy": "3", "user": { "screen-name": "RollandEckhard#500", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Hetfield", "followers_count": 3311368 }, "sender-location": point("39.84,86.48"), "send-time": datetime("2008-12-24T00:07:04.000Z"), "send-time-copy": datetime("2008-12-24T00:07:04.000Z"), "referred-topics": {{ "verizon", "voice-command" }}, "message-text": " can't stand verizon its voice-command is terrible:(" }
+{ "tweetid": "4", "tweetid-copy": "4", "user": { "screen-name": "RollandEckhardstein#221", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardstinz", "followers_count": 3311368 }, "sender-location": point("27.67,87.32"), "send-time": datetime("2007-02-05T16:39:13.000Z"), "send-time-copy": datetime("2007-02-05T16:39:13.000Z"), "referred-topics": {{ "t-mobile", "customer-service" }}, "message-text": " love t-mobile its customer-service is mind-blowing" }
+{ "tweetid": "5", "tweetid-copy": "5", "user": { "screen-name": "RollandEcstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckhardst", "followers_count": 3311368 }, "sender-location": point("27.3,92.77"), "send-time": datetime("2010-09-12T06:15:28.000Z"), "send-time-copy": datetime("2010-09-12T06:15:28.000Z"), "referred-topics": {{ "t-mobile", "customization" }}, "message-text": " like t-mobile the customization is amazing:)" }
+{ "tweetid": "6", "tweetid-copy": "6", "user": { "screen-name": "Rollkhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Kirk Hammette ", "followers_count": 3311368 }, "sender-location": point("45.62,84.78"), "send-time": datetime("2012-01-23T06:23:13.000Z"), "send-time-copy": datetime("2012-01-23T06:23:13.000Z"), "referred-topics": {{ "iphone", "network" }}, "message-text": " like iphone its network is awesome:)" }
+{ "tweetid": "7", "tweetid-copy": "7", "user": { "screen-name": "andEckhardstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland khardstein", "followers_count": 3311368 }, "sender-location": point("44.12,81.46"), "send-time": datetime("2012-02-17T17:30:26.000Z"), "send-time-copy": datetime("2012-02-17T17:30:26.000Z"), "referred-topics": {{ "t-mobile", "network" }}, "message-text": " hate t-mobile the network is bad" }
+{ "tweetid": "8", "tweetid-copy": "8", "user": { "screen-name": "Rolltein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Ron Eckhardstein", "followers_count": 3311368 }, "sender-location": point("36.86,90.71"), "send-time": datetime("2009-03-12T13:18:04.000Z"), "send-time-copy": datetime("2009-03-12T13:18:04.000Z"), "referred-topics": {{ "at&t", "touch-screen" }}, "message-text": " dislike at&t its touch-screen is OMG" }
+{ "tweetid": "9", "tweetid-copy": "9", "user": { "screen-name": "Roldstein#211", "lang": "en", "friends_count": 3657079, "statuses_count": 268, "name": "Rolland Eckdstein", "followers_count": 3311368 }, "sender-location": point("29.07,97.05"), "send-time": datetime("2012-08-15T20:19:46.000Z"), "send-time-copy": datetime("2012-08-15T20:19:46.000Z"), "referred-topics": {{ "verizon", "speed" }}, "message-text": " hate verizon its speed is bad" }
diff --git a/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
new file mode 100644
index 0000000..e5ac8ab
--- /dev/null
+++ b/asterix-app/src/test/resources/runtimets/results/open-closed/query-issue350/query-issue350.1.adm
@@ -0,0 +1 @@
+{ "tweetid": "13", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39345, "statuses_count": 479, "name": "Nathan Giesen", "followers_count": 49420, "hobbies": [ "basket weaving", "mud wrestling" ] }, "sender-location": point("47.44,80.65"), "send-time": datetime("2008-04-26T10:10:35.000Z"), "referred-topics": {{ "tweeting" }}, "message-text": "tweety tweet, my fellow tweeters!" }
diff --git a/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterix-app/src/test/resources/runtimets/testsuite.xml
index 2c9e65a..730c334 100644
--- a/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -2728,6 +2728,31 @@
         <output-dir compare="Text">query-proposal</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue350">
+        <output-dir compare="Text">query-issue350</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue350-2">
+        <output-dir compare="Text">query-issue350-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue343">
+        <output-dir compare="Text">query-issue343</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue343-2">
+        <output-dir compare="Text">query-issue343-2</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="open-closed">
+      <compilation-unit name="query-issue196">
+        <output-dir compare="Text">query-issue196</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="quantifiers">
     <test-case FilePath="quantifiers">
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 ae44a10..bea2294 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
@@ -27,6 +27,7 @@
 import edu.uci.ics.asterix.om.typecomputer.impl.BinaryBooleanOrNullFunctionTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringBoolOrNullTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.BinaryStringStringOrNullTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.impl.CastListResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.CastRecordResultTypeComputer;
 import edu.uci.ics.asterix.om.typecomputer.impl.ClosedRecordConstructorResultType;
 import edu.uci.ics.asterix.om.typecomputer.impl.FieldAccessByIndexResultType;
@@ -472,6 +473,8 @@
             "inject-failure", 2);
     public final static FunctionIdentifier CAST_RECORD = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
             "cast-record", 1);
+    public final static FunctionIdentifier CAST_LIST = new FunctionIdentifier(FunctionConstants.ASTERIX_NS,
+            "cast-list", 1);
 
     // Spatial and temporal type accessors
     public static final FunctionIdentifier ACCESSOR_TEMPORAL_YEAR = new FunctionIdentifier(
@@ -787,6 +790,7 @@
         add(REG_EXP, ABooleanTypeComputer.INSTANCE);
         add(INJECT_FAILURE, InjectFailureTypeComputer.INSTANCE);
         add(CAST_RECORD, CastRecordResultTypeComputer.INSTANCE);
+        add(CAST_LIST, CastListResultTypeComputer.INSTANCE);
 
         add(TID, AInt32TypeComputer.INSTANCE);
         add(TIME_CONSTRUCTOR, OptionalATimeTypeComputer.INSTANCE);
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
index ab32b6b..84c9e6c 100644
--- a/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/pointables/AListPointable.java
@@ -93,6 +93,7 @@
     @Override
     public void set(byte[] b, int s, int len) {
         reset();
+        super.set(b, s, len);
 
         int numberOfitems = AInt32SerializerDeserializer.getInt(b, s + 6);
         int itemOffset;
diff --git a/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java
new file mode 100644
index 0000000..4c3c56b
--- /dev/null
+++ b/asterix-om/src/main/java/edu/uci/ics/asterix/om/typecomputer/impl/CastListResultTypeComputer.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.om.typecomputer.impl;
+
+import edu.uci.ics.asterix.om.typecomputer.base.IResultTypeComputer;
+import edu.uci.ics.asterix.om.typecomputer.base.TypeComputerUtilities;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import edu.uci.ics.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import edu.uci.ics.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+/**
+ * The type computer for the cast-list function
+ * 
+ * @author yingyib
+ */
+public class CastListResultTypeComputer implements IResultTypeComputer {
+
+    public static final CastListResultTypeComputer INSTANCE = new CastListResultTypeComputer();
+
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) expression;
+        return TypeComputerUtilities.getRequiredType(funcExpr);
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java
new file mode 100644
index 0000000..c1817e5
--- /dev/null
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/evaluators/functions/CastListDescriptor.java
@@ -0,0 +1,105 @@
+/*
+ * Copyright 2009-2013 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package edu.uci.ics.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
+import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.pointables.PointableAllocator;
+import edu.uci.ics.asterix.om.pointables.base.IVisitablePointable;
+import edu.uci.ics.asterix.om.pointables.cast.ACastVisitor;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
+import edu.uci.ics.asterix.om.types.IAType;
+import edu.uci.ics.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
+import edu.uci.ics.hyracks.algebricks.common.utils.Triple;
+import edu.uci.ics.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluator;
+import edu.uci.ics.hyracks.algebricks.runtime.base.ICopyEvaluatorFactory;
+import edu.uci.ics.hyracks.data.std.api.IDataOutputProvider;
+import edu.uci.ics.hyracks.data.std.util.ArrayBackedValueStorage;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+/**
+ * The runtime function for casting a list(unordered list or ordered list)
+ * 
+ * @author yingyib
+ *
+ */
+public class CastListDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new CastListDescriptor();
+        }
+    };
+
+    private static final long serialVersionUID = 1L;
+    private AbstractCollectionType reqType;
+    private AbstractCollectionType inputType;
+
+    public void reset(AbstractCollectionType reqType, AbstractCollectionType inputType) {
+        this.reqType = reqType;
+        this.inputType = inputType;
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return AsterixBuiltinFunctions.CAST_LIST;
+    }
+
+    @Override
+    public ICopyEvaluatorFactory createEvaluatorFactory(final ICopyEvaluatorFactory[] args) {
+        final ICopyEvaluatorFactory recordEvalFactory = args[0];
+
+        return new ICopyEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public ICopyEvaluator createEvaluator(final IDataOutputProvider output) throws AlgebricksException {
+                final DataOutput out = output.getDataOutput();
+                final ArrayBackedValueStorage recordBuffer = new ArrayBackedValueStorage();
+                final ICopyEvaluator recEvaluator = recordEvalFactory.createEvaluator(recordBuffer);
+
+                return new ICopyEvaluator() {
+                    // pointable allocator
+                    private PointableAllocator allocator = new PointableAllocator();
+                    final IVisitablePointable recAccessor = allocator.allocateListValue(inputType);
+                    final IVisitablePointable resultAccessor = allocator.allocateListValue(reqType);
+                    final ACastVisitor castVisitor = new ACastVisitor();
+                    final Triple<IVisitablePointable, IAType, Boolean> arg = new Triple<IVisitablePointable, IAType, Boolean>(
+                            resultAccessor, reqType, Boolean.FALSE);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple) throws AlgebricksException {
+                        try {
+                            recordBuffer.reset();
+                            recEvaluator.evaluate(tuple);
+                            recAccessor.set(recordBuffer);
+                            recAccessor.accept(castVisitor, arg);
+                            out.write(resultAccessor.getByteArray(), resultAccessor.getStartOffset(),
+                                    resultAccessor.getLength());
+                        } catch (Exception ioe) {
+                            throw new AlgebricksException(ioe);
+                        }
+                    }
+                };
+            }
+        };
+    }
+}
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
index 1b6d357..618a3ee 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/formats/NonTaggedDataFormat.java
@@ -41,6 +41,7 @@
 import edu.uci.ics.asterix.om.types.ATypeTag;
 import edu.uci.ics.asterix.om.types.AUnionType;
 import edu.uci.ics.asterix.om.types.AUnorderedListType;
+import edu.uci.ics.asterix.om.types.AbstractCollectionType;
 import edu.uci.ics.asterix.om.types.BuiltinType;
 import edu.uci.ics.asterix.om.types.IAType;
 import edu.uci.ics.asterix.runtime.aggregates.collections.ListifyAggregateDescriptor;
@@ -67,6 +68,11 @@
 import edu.uci.ics.asterix.runtime.aggregates.std.SumAggregateDescriptor;
 import edu.uci.ics.asterix.runtime.aggregates.stream.EmptyStreamAggregateDescriptor;
 import edu.uci.ics.asterix.runtime.aggregates.stream.NonEmptyStreamAggregateDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
+import edu.uci.ics.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalDayAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalHourAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalIntervalEndAccessor;
@@ -76,11 +82,6 @@
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalMonthAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalSecondAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.accessors.TemporalYearAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleCenterAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.CircleRadiusAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.LineRectanglePolygonAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.PointXCoordinateAccessor;
-import edu.uci.ics.asterix.runtime.evaluators.accessors.PointYCoordinateAccessor;
 import edu.uci.ics.asterix.runtime.evaluators.common.CreateMBREvalFactory;
 import edu.uci.ics.asterix.runtime.evaluators.common.FieldAccessByIndexEvalFactory;
 import edu.uci.ics.asterix.runtime.evaluators.common.FunctionManagerImpl;
@@ -113,6 +114,7 @@
 import edu.uci.ics.asterix.runtime.evaluators.constructors.AYearMonthDurationConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.AndDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.AnyCollectionMemberDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.CastListDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.CastRecordDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.ClosedRecordConstructorDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.CodePointToStringDescriptor;
@@ -172,10 +174,6 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StartsWithDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
-import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StringConcatDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StringEndWithDescrtiptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.StringEqualDescriptor;
@@ -191,6 +189,10 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.Substring2Descriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringAfterDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringBeforeDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SubstringDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.SwitchCaseDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.UnorderedListConstructorDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.WordTokensDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDateDurationDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddDatetimeDurationDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.AddTimeDurationDescriptor;
@@ -226,6 +228,9 @@
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalOverlapsDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartedByDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.IntervalStartsDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MillisecondsOfDayTimeDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.MonthsOfYearMonthDurationDescriptor;
+import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.OverlapDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDateDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractDatetimeDescriptor;
 import edu.uci.ics.asterix.runtime.evaluators.functions.temporal.SubtractTimeDescriptor;
@@ -468,6 +473,7 @@
         temp.add(SwitchCaseDescriptor.FACTORY);
         temp.add(RegExpDescriptor.FACTORY);
         temp.add(InjectFailureDescriptor.FACTORY);
+        temp.add(CastListDescriptor.FACTORY);
         temp.add(CastRecordDescriptor.FACTORY);
         temp.add(NotNullDescriptor.FACTORY);
 
@@ -697,10 +703,17 @@
             }
         }
         if (fd.getIdentifier().equals(AsterixBuiltinFunctions.CAST_RECORD)) {
-            ARecordType rt = (ARecordType) TypeComputerUtilities.getRequiredType((AbstractFunctionCallExpression) expr);
-            ARecordType it = (ARecordType) TypeComputerUtilities.getInputType((AbstractFunctionCallExpression) expr);
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            ARecordType rt = (ARecordType) TypeComputerUtilities.getRequiredType(funcExpr);
+            ARecordType it = (ARecordType) context.getType(funcExpr.getArguments().get(0).getValue());
             ((CastRecordDescriptor) fd).reset(rt, it);
         }
+        if (fd.getIdentifier().equals(AsterixBuiltinFunctions.CAST_LIST)) {
+            AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+            AbstractCollectionType rt = (AbstractCollectionType) TypeComputerUtilities.getRequiredType(funcExpr);
+            AbstractCollectionType it = (AbstractCollectionType) context.getType(funcExpr.getArguments().get(0).getValue());
+            ((CastListDescriptor) fd).reset(rt, it);
+        }
         if (fd.getIdentifier().equals(AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)) {
             ARecordType rt = (ARecordType) context.getType(expr);
             ((OpenRecordConstructorDescriptor) fd).reset(rt,
diff --git a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
index c70cdf8..f29d6be 100644
--- a/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
+++ b/asterix-runtime/src/main/java/edu/uci/ics/asterix/runtime/unnestingfunctions/std/ScanCollectionDescriptor.java
@@ -22,6 +22,8 @@
 import edu.uci.ics.asterix.om.functions.AsterixBuiltinFunctions;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptor;
 import edu.uci.ics.asterix.om.functions.IFunctionDescriptorFactory;
+import edu.uci.ics.asterix.om.types.ATypeTag;
+import edu.uci.ics.asterix.om.types.EnumDeserializer;
 import edu.uci.ics.asterix.runtime.evaluators.common.AsterixListAccessor;
 import edu.uci.ics.asterix.runtime.unnestingfunctions.base.AbstractUnnestingFunctionDynamicDescriptor;
 import edu.uci.ics.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -73,12 +75,19 @@
                 private ArrayBackedValueStorage inputVal = new ArrayBackedValueStorage();
                 private ICopyEvaluator argEval = listEvalFactory.createEvaluator(inputVal);
                 private int itemIndex;
+                private boolean metNull = false;
 
                 @Override
                 public void init(IFrameTupleReference tuple) throws AlgebricksException {
                     try {
                         inputVal.reset();
                         argEval.evaluate(tuple);
+                        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER
+                                .deserialize(inputVal.getByteArray()[0]);
+                        if (typeTag == ATypeTag.NULL) {
+                            metNull = true;
+                            return;
+                        }
                         listAccessor.reset(inputVal.getByteArray(), 0);
                         itemIndex = 0;
                     } catch (AsterixException e) {
@@ -89,10 +98,12 @@
                 @Override
                 public boolean step() throws AlgebricksException {
                     try {
-                        if (itemIndex < listAccessor.size()) {
-                            listAccessor.writeItem(itemIndex, out);
-                            ++itemIndex;
-                            return true;
+                        if (!metNull) {
+                            if (itemIndex < listAccessor.size()) {
+                                listAccessor.writeItem(itemIndex, out);
+                                ++itemIndex;
+                                return true;
+                            }
                         }
                     } catch (IOException e) {
                         throw new AlgebricksException(e);