You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by al...@apache.org on 2019/07/31 18:04:06 UTC

[asterixdb] branch master updated: [ASTERIXDB-2615][*DB] Enable constant folding rule to determine when to fold

This is an automated email from the ASF dual-hosted git repository.

alsuliman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 8c4dc24  [ASTERIXDB-2615][*DB] Enable constant folding rule to determine when to fold
8c4dc24 is described below

commit 8c4dc247c66593b0377e9468e0782d3991239e4f
Author: Ali Alsuliman <al...@gmail.com>
AuthorDate: Tue Jul 30 01:04:36 2019 -0700

    [ASTERIXDB-2615][*DB] Enable constant folding rule to determine when to fold
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    Details:
    Currently, some functions are excluded from being evaluated at compile time in
    constant folding rule. Those functions are maintained in a set. The reason for
    excluding them is because they can produce records/lists in their "open" format
    and constant folding them will make them "closed" which would lead to incorrect
    results. Any time a new function is implemented, the developer would have to pay
    attention and add their functions if they should be excluded, too.
    
    The exclusion set should be removed. The constant folding rule should determine
    from the output type of the function whether it can/should be folded or not.
    
    - fixed an issue with StaticTypeCastUtil where it would not open up a field
      when that field's value is a function call.
    - modified casting items of lists to avoid casting open the item when it is
      already in the opened up format.
    - modified the type computer of object_remove_fields to handle constant
      arguments (the 2nd argument which is the name of the fields) since the
      argument can be constant folded now.
    - modified field-access-by-index to handle evaluating it at compile time
      (by constant folding rule) in the case where the object accessed is missing
      (or non-object)
    
    Change-Id: I3964aa8accaaae3b4c5c7ddf928e6cbd73d517f6
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/3509
    Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Ali Alsuliman <al...@gmail.com>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
 .../optimizer/rules/ConstantFoldingRule.java       |  99 +++++++++---------
 .../rules/typecast/StaticTypeCastUtil.java         | 113 +++++++++++++--------
 .../array_append/array_append.3.query.sqlpp        |  28 ++---
 .../array_concat/array_concat.3.query.sqlpp        |  28 ++---
 .../array_distinct/array_distinct.3.query.sqlpp    |  30 +++---
 .../array_distinct/array_distinct.4.query.sqlpp    |  16 +--
 .../array_flatten/array_flatten.3.query.sqlpp      |  42 ++++----
 .../array_ifnull/array_ifnull.3.query.sqlpp        |  34 ++++---
 .../array_insert/array_insert.3.query.sqlpp        |  56 +++++-----
 .../array_prepend/array_prepend.3.query.sqlpp      |  28 ++---
 .../array_fun/array_put/array_put.3.query.sqlpp    |  30 +++---
 .../array_fun/array_put/array_put.4.query.sqlpp    |  18 ++--
 .../array_range/array_range.3.query.sqlpp          |  48 ++++-----
 .../array_remove/array_remove.3.query.sqlpp        |  28 ++---
 .../array_remove/array_remove.4.query.sqlpp        |  18 ++--
 .../array_remove.5.query.sqlpp}                    |  17 ++--
 ...remove.5.ddl.sqlpp => array_remove.6.ddl.sqlpp} |   0
 .../array_replace/array_replace.3.query.sqlpp      |  58 ++++++-----
 .../array_replace/array_replace.4.query.sqlpp      |  18 ++--
 .../array_fun/array_sort/array_sort.3.query.sqlpp  |  30 +++---
 .../array_fun/array_sort/array_sort.4.query.sqlpp  |  16 +--
 .../object_unwrap/object_unwarp.3.query.sqlpp      |  11 +-
 ...arp.4.ddl.sqlpp => object_unwarp.4.query.sqlpp} |   9 +-
 ...nwarp.4.ddl.sqlpp => object_unwarp.5.ddl.sqlpp} |   0
 .../array_fun/array_append/array_append.3.adm      |  11 +-
 .../array_fun/array_concat/array_concat.3.adm      |  11 +-
 .../array_fun/array_distinct/array_distinct.3.adm  |  12 ++-
 .../array_fun/array_distinct/array_distinct.4.adm  |   5 +-
 .../array_fun/array_flatten/array_flatten.3.adm    |  18 +++-
 .../array_fun/array_ifnull/array_ifnull.3.adm      |  14 ++-
 .../array_fun/array_insert/array_insert.3.adm      |  25 ++++-
 .../array_fun/array_prepend/array_prepend.3.adm    |  11 +-
 .../results/array_fun/array_put/array_put.3.adm    |  12 ++-
 .../results/array_fun/array_put/array_put.4.adm    |   6 +-
 .../array_fun/array_range/array_range.3.adm        |  21 +++-
 .../array_fun/array_remove/array_remove.3.adm      |  11 +-
 .../array_fun/array_remove/array_remove.4.adm      |   6 +-
 .../array_fun/array_remove/array_remove.5.adm      |   8 ++
 .../array_fun/array_replace/array_replace.3.adm    |  26 ++++-
 .../array_fun/array_replace/array_replace.4.adm    |   6 +-
 .../results/array_fun/array_sort/array_sort.3.adm  |  12 ++-
 .../results/array_fun/array_sort/array_sort.4.adm  |   5 +-
 .../objects/object_unwrap/object_unwrap.3.adm      |   2 +-
 .../objects/object_unwrap/object_unwrap.4.adm      |   1 +
 .../results/types/to_number_01/to_number_01.1.adm  |   2 +-
 .../impl/RecordRemoveFieldsTypeComputer.java       |  94 +++++++++--------
 .../records/FieldAccessByIndexEvalFactory.java     |  36 +++----
 .../runtime/functions/FunctionTypeInferers.java    |   8 +-
 48 files changed, 691 insertions(+), 447 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
index 4c6f338..7a63ea3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/ConstantFoldingRule.java
@@ -21,10 +21,8 @@ package org.apache.asterix.optimizer.rules;
 
 import java.io.DataInputStream;
 import java.nio.ByteBuffer;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -45,10 +43,10 @@ import org.apache.asterix.om.base.ADouble;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
-import org.apache.asterix.om.typecomputer.base.TypeCastUtils;
 import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.types.TypeTagUtil;
@@ -90,31 +88,17 @@ import org.apache.hyracks.data.std.primitive.VoidPointable;
 import org.apache.hyracks.dataflow.common.comm.util.ByteBufferInputStream;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
 
 public class ConstantFoldingRule implements IAlgebraicRewriteRule {
 
     private final ConstantFoldingVisitor cfv = new ConstantFoldingVisitor();
     private final JobGenContext jobGenCtx;
 
-    // Function Identifier sets that the ConstantFolding rule should skip to apply.
-    // Most of them are record-related functions.
-
-    private static final Set<FunctionIdentifier> FUNC_ID_SET_THAT_SHOULD_NOT_BE_APPLIED =
-            new HashSet<>(ImmutableSet.of(BuiltinFunctions.RECORD_MERGE, BuiltinFunctions.ADD_FIELDS,
-                    BuiltinFunctions.REMOVE_FIELDS, BuiltinFunctions.GET_RECORD_FIELDS,
-                    BuiltinFunctions.GET_RECORD_FIELD_VALUE, BuiltinFunctions.FIELD_ACCESS_NESTED,
-                    BuiltinFunctions.GET_ITEM, BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR,
-                    BuiltinFunctions.FIELD_ACCESS_BY_INDEX, BuiltinFunctions.CAST_TYPE, BuiltinFunctions.META,
-                    BuiltinFunctions.META_KEY, BuiltinFunctions.RECORD_CONCAT, BuiltinFunctions.RECORD_CONCAT_STRICT,
-                    BuiltinFunctions.RECORD_PAIRS, BuiltinFunctions.PAIRS, BuiltinFunctions.TO_ATOMIC,
-                    BuiltinFunctions.TO_ARRAY)); //Initialize with BUILTIN FUNC ID SET THAT SHOULD NOT BE APPLIED
-
     private static final Map<FunctionIdentifier, IAObject> FUNC_ID_TO_CONSTANT = ImmutableMap
             .of(BuiltinFunctions.NUMERIC_E, new ADouble(Math.E), BuiltinFunctions.NUMERIC_PI, new ADouble(Math.PI));
 
     /**
-     * Throws exceptions in substituiteProducedVariable, setVarType, and one getVarType method.
+     * Throws exceptions in substituteProducedVariable, setVarType, and one getVarType method.
      */
     private static final IVariableTypeEnvironment _emptyTypeEnv = new IVariableTypeEnvironment() {
 
@@ -157,10 +141,6 @@ public class ConstantFoldingRule implements IAlgebraicRewriteRule {
                 ExpressionTypeComputer.INSTANCE, null, null, null, null, GlobalConfig.DEFAULT_FRAME_SIZE, null);
     }
 
-    public static void addNonFoldableFunction(FunctionIdentifier fid) {
-        FUNC_ID_SET_THAT_SHOULD_NOT_BE_APPLIED.add(fid);
-    }
-
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
@@ -195,11 +175,11 @@ public class ConstantFoldingRule implements IAlgebraicRewriteRule {
         @Override
         public boolean transform(Mutable<ILogicalExpression> exprRef) throws AlgebricksException {
             AbstractLogicalExpression expr = (AbstractLogicalExpression) exprRef.getValue();
-            Pair<Boolean, ILogicalExpression> p = expr.accept(this, null);
-            if (p.first) {
-                exprRef.setValue(p.second);
+            Pair<Boolean, ILogicalExpression> newExpression = expr.accept(this, null);
+            if (newExpression.first) {
+                exprRef.setValue(newExpression.second);
             }
-            return p.first;
+            return newExpression.first;
         }
 
         @Override
@@ -216,30 +196,12 @@ public class ConstantFoldingRule implements IAlgebraicRewriteRule {
         @Override
         public Pair<Boolean, ILogicalExpression> visitScalarFunctionCallExpression(ScalarFunctionCallExpression expr,
                 Void arg) throws AlgebricksException {
-            boolean changed = changeRec(expr, arg);
-            if (!checkArgs(expr) || !expr.isFunctional()) {
+            boolean changed = constantFoldArgs(expr, arg);
+            if (!allArgsConstant(expr) || !expr.isFunctional() || !canConstantFold(expr)) {
                 return new Pair<>(changed, expr);
             }
 
-            // Skip Constant Folding for the record-related functions.
-            if (FUNC_ID_SET_THAT_SHOULD_NOT_BE_APPLIED.contains(expr.getFunctionIdentifier())) {
-                return new Pair<>(false, null);
-            }
-
             try {
-                // Current List SerDe assumes a strongly typed list, so we do not constant fold the list constructors
-                // if they are not strongly typed
-                if (expr.getFunctionIdentifier().equals(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR)
-                        || expr.getFunctionIdentifier().equals(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)) {
-                    AbstractCollectionType listType = (AbstractCollectionType) TypeCastUtils.getRequiredType(expr);
-                    if (listType != null && (listType.getItemType().getTypeTag() == ATypeTag.ANY
-                            || listType.getItemType() instanceof AbstractCollectionType)) {
-                        //case1: listType == null,  could be a nested list inside a list<ANY>
-                        //case2: itemType = ANY
-                        //case3: itemType = a nested list
-                        return new Pair<>(false, null);
-                    }
-                }
                 if (expr.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
                     ARecordType rt = (ARecordType) _emptyTypeEnv.getType(expr.getArguments().get(0).getValue());
                     String str = ConstantExpressionUtil.getStringConstant(expr.getArguments().get(1).getValue());
@@ -285,25 +247,25 @@ public class ConstantFoldingRule implements IAlgebraicRewriteRule {
         @Override
         public Pair<Boolean, ILogicalExpression> visitAggregateFunctionCallExpression(
                 AggregateFunctionCallExpression expr, Void arg) throws AlgebricksException {
-            boolean changed = changeRec(expr, arg);
+            boolean changed = constantFoldArgs(expr, arg);
             return new Pair<>(changed, expr);
         }
 
         @Override
         public Pair<Boolean, ILogicalExpression> visitStatefulFunctionCallExpression(
                 StatefulFunctionCallExpression expr, Void arg) throws AlgebricksException {
-            boolean changed = changeRec(expr, arg);
+            boolean changed = constantFoldArgs(expr, arg);
             return new Pair<>(changed, expr);
         }
 
         @Override
         public Pair<Boolean, ILogicalExpression> visitUnnestingFunctionCallExpression(
                 UnnestingFunctionCallExpression expr, Void arg) throws AlgebricksException {
-            boolean changed = changeRec(expr, arg);
+            boolean changed = constantFoldArgs(expr, arg);
             return new Pair<>(changed, expr);
         }
 
-        private boolean changeRec(AbstractFunctionCallExpression expr, Void arg) throws AlgebricksException {
+        private boolean constantFoldArgs(AbstractFunctionCallExpression expr, Void arg) throws AlgebricksException {
             boolean changed = false;
             for (Mutable<ILogicalExpression> r : expr.getArguments()) {
                 Pair<Boolean, ILogicalExpression> p2 = r.getValue().accept(this, arg);
@@ -315,7 +277,7 @@ public class ConstantFoldingRule implements IAlgebraicRewriteRule {
             return changed;
         }
 
-        private boolean checkArgs(AbstractFunctionCallExpression expr) {
+        private boolean allArgsConstant(AbstractFunctionCallExpression expr) {
             for (Mutable<ILogicalExpression> r : expr.getArguments()) {
                 if (r.getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
                     return false;
@@ -323,5 +285,40 @@ public class ConstantFoldingRule implements IAlgebraicRewriteRule {
             }
             return true;
         }
+
+        private boolean canConstantFold(ScalarFunctionCallExpression function) throws AlgebricksException {
+            // skip all functions that would produce records/arrays/multisets (derived types) in their open format
+            // this is because constant folding them will make them closed (currently)
+            if (function.getFunctionIdentifier().equals(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)) {
+                return false;
+            }
+            IAType returnType = (IAType) _emptyTypeEnv.getType(function);
+            return canConstantFoldType(returnType);
+        }
+
+        private boolean canConstantFoldType(IAType returnType) {
+            ATypeTag tag = returnType.getTypeTag();
+            if (tag == ATypeTag.ANY) {
+                // if the function is to return a record (or derived data), that record would (should) be an open record
+                return false;
+            } else if (tag == ATypeTag.OBJECT) {
+                ARecordType recordType = (ARecordType) returnType;
+                if (recordType.isOpen()) {
+                    return false;
+                }
+                IAType[] fieldTypes = recordType.getFieldTypes();
+                for (int i = 0; i < fieldTypes.length; i++) {
+                    if (!canConstantFoldType(fieldTypes[i])) {
+                        return false;
+                    }
+                }
+            } else if (tag.isListType()) {
+                AbstractCollectionType listType = (AbstractCollectionType) returnType;
+                return canConstantFoldType(listType.getItemType());
+            } else if (tag == ATypeTag.UNION) {
+                return canConstantFoldType(((AUnionType) returnType).getActualType());
+            }
+            return true;
+        }
     }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
index 19cc680..ac2460e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/typecast/StaticTypeCastUtil.java
@@ -42,6 +42,7 @@ import org.apache.asterix.om.types.AUnionType;
 import org.apache.asterix.om.types.AbstractCollectionType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeHelper;
 import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.asterix.om.utils.NonTaggedFormatUtil;
 import org.apache.commons.lang3.mutable.Mutable;
@@ -54,6 +55,7 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCa
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 
 /**
@@ -62,17 +64,22 @@ import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
  * 1. public static boolean rewriteListExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
  * IVariableTypeEnvironment env) throws AlgebricksException, which only enforces the list type recursively.
  * 2. public static boolean rewriteFuncExpr(AbstractFunctionCallExpression funcExpr, IAType reqType, IAType inputType,
- * IVariableTypeEnvironment env) throws AlgebricksException, which enforces the list type and the record type recursively.
+ * IVariableTypeEnvironment env) throws AlgebricksException, which enforces the list type and the record type
+ * recursively.
  *
  * @author yingyib
  */
 public class StaticTypeCastUtil {
 
+    private StaticTypeCastUtil() {
+    }
+
     /**
      * This method is only called when funcExpr contains list constructor function calls.
      * The List constructor is very special because a nested list is of type List<ANY>.
-     * However, the bottom-up type inference (InferTypeRule in algebricks) did not infer that so we need this method to enforce the type.
-     * We do not want to break the generality of algebricks so this method is called in an ASTERIX rule: @ IntroduceEnforcedListTypeRule} .
+     * However, the bottom-up type inference (InferTypeRule in algebricks) did not infer that so we need this method to
+     * enforce the type. We do not want to break the generality of algebricks so this method is called in an ASTERIX
+     * rule: @ IntroduceEnforcedListTypeRule.
      *
      * @param funcExpr
      *            record constructor function expression
@@ -120,8 +127,9 @@ public class StaticTypeCastUtil {
      * The open record constructor is very special because
      * 1. a nested list in the open part is of type List<ANY>;
      * 2. a nested record in the open part is of type Open_Record{}.
-     * However, the bottom-up type inference (InferTypeRule in algebricks) did not infer that so we need this method to enforce the type.
-     * We do not want to break the generality of algebricks so this method is called in an ASTERIX rule: @ IntroduceStaticTypeCastRule} .
+     * However, the bottom-up type inference (InferTypeRule in algebricks) did not infer that so we need this method
+     * to enforce the type. We do not want to break the generality of algebricks so this method is called in an
+     * ASTERIX rule: @ IntroduceStaticTypeCastRule
      *
      * @param funcExpr
      *            the function expression whose type needs to be top-down enforced
@@ -137,8 +145,8 @@ public class StaticTypeCastUtil {
     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".
+         * 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() == BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR) {
@@ -166,7 +174,7 @@ public class StaticTypeCastUtil {
                 if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                     AbstractFunctionCallExpression argFuncExpr = (AbstractFunctionCallExpression) argExpr;
                     IAType exprType = (IAType) env.getType(argFuncExpr);
-                    changed = changed || rewriteFuncExpr(argFuncExpr, exprType, exprType, env);
+                    changed = rewriteFuncExpr(argFuncExpr, exprType, exprType, env) || changed;
                 }
             }
             if (!compatible(reqType, inputType)) {
@@ -225,40 +233,44 @@ public class StaticTypeCastUtil {
 
         TypeCastUtils.setRequiredAndInputTypes(funcExpr, requiredListType, inputListType);
         List<Mutable<ILogicalExpression>> args = funcExpr.getArguments();
-
+        // TODO: if required type = [bigint], input type = [small_int], how is this method enforcing that?
+        // TODO: it seems it's only concerned with dealing with complex types items (records,lists)
         IAType requiredItemType = requiredListType.getItemType();
         IAType inputItemType = inputListType.getItemType();
         boolean changed = false;
         for (int j = 0; j < args.size(); j++) {
-            ILogicalExpression arg = args.get(j).getValue();
+            Mutable<ILogicalExpression> argRef = args.get(j);
+            ILogicalExpression arg = argRef.getValue();
             IAType currentItemType = (inputItemType == null || inputItemType == BuiltinType.ANY)
                     ? (IAType) env.getType(arg) : inputItemType;
             switch (arg.getExpressionTag()) {
                 case FUNCTION_CALL:
                     ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) arg;
                     changed |= rewriteFuncExpr(argFunc, requiredItemType, currentItemType, env);
-                    changed |= castItem(requiredItemType, currentItemType, argFunc, args.get(j));
+                    changed |= castItem(argRef, argFunc, requiredItemType, env);
                     break;
                 case VARIABLE:
-                    changed |= injectCastToRelaxType(args.get(j), currentItemType, env);
+                    // TODO(ali): why are we always casting to an open type without considering "requiredItemType"?
+                    changed |= injectCastToRelaxType(argRef, currentItemType, env);
                     break;
             }
         }
         return changed;
     }
 
-    private static boolean castItem(IAType requiredItemType, IAType currentItemType,
-            ScalarFunctionCallExpression itemExpr, Mutable<ILogicalExpression> itemExprRef) throws AlgebricksException {
-        if (TypeResolverUtil.needsCast(requiredItemType, currentItemType) && shouldCast(itemExpr)) {
-            injectCastFunction(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE), requiredItemType,
-                    currentItemType, itemExprRef, itemExpr);
+    private static boolean castItem(Mutable<ILogicalExpression> itemExprRef, ScalarFunctionCallExpression itemExpr,
+            IAType requiredItemType, IVariableTypeEnvironment env) throws AlgebricksException {
+        IAType itemType = (IAType) env.getType(itemExpr);
+        if (TypeResolverUtil.needsCast(requiredItemType, itemType) && !satisfied(requiredItemType, itemType)) {
+            injectCastFunction(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE), requiredItemType, itemType,
+                    itemExprRef, itemExpr);
             return true;
         }
         return false;
     }
 
-    private static boolean shouldCast(ScalarFunctionCallExpression itemExpr) {
-        return TypeCastUtils.getRequiredType(itemExpr) == null;
+    private static boolean satisfied(IAType required, IAType actual) {
+        return required.getTypeTag() == ATypeTag.ANY && TypeHelper.isFullyOpen(actual);
     }
 
     /**
@@ -481,58 +493,68 @@ public class StaticTypeCastUtil {
         return -1;
     }
 
-    private static boolean injectCastToRelaxType(Mutable<ILogicalExpression> expRef, IAType inputFieldType,
+    // casts exprRef (which is either a function call or a variable) to fully open if it is not already fully open
+    private static boolean injectCastToRelaxType(Mutable<ILogicalExpression> expRef, IAType expType,
             IVariableTypeEnvironment env) throws AlgebricksException {
         ILogicalExpression argExpr = expRef.getValue();
-        List<LogicalVariable> parameterVars = new ArrayList<LogicalVariable>();
+        List<LogicalVariable> parameterVars = new ArrayList<>();
         argExpr.getUsedVariables(parameterVars);
-        // we need to handle open fields recursively by their default
-        // types
-        // for list, their item type is any
-        // for record, their
         boolean castInjected = false;
         if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
                 || argExpr.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
-            IAType reqFieldType = inputFieldType;
+            IAType exprActualType = expType;
+            if (expType.getTypeTag() == ATypeTag.UNION) {
+                exprActualType = ((AUnionType) expType).getActualType();
+            }
+            IAType requiredType = exprActualType;
             // do not enforce nested type in the case of no-used variables
-            switch (inputFieldType.getTypeTag()) {
+            switch (exprActualType.getTypeTag()) {
                 case OBJECT:
-                    reqFieldType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+                    requiredType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
                     break;
                 case ARRAY:
-                    reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
+                    requiredType = DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE;
                     break;
                 case MULTISET:
-                    reqFieldType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
+                    requiredType = DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE;
                     break;
                 default:
                     break;
             }
-            // do not enforce nested type in the case of no-used variables
-            if (!inputFieldType.equals(reqFieldType) && !parameterVars.isEmpty()) {
-                //inject dynamic type casting
-                injectCastFunction(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE), reqFieldType,
-                        inputFieldType, expRef, argExpr);
+            // add cast(expr) if the expr is a variable or using a variable or non constructor function call expr.
+            // skip if expr is a constructor with values where you can traverse and cast fields/items individually
+            if (!exprActualType.equals(requiredType) && (!parameterVars.isEmpty() || !isComplexConstructor(argExpr))) {
+                injectCastFunction(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE), requiredType, expType,
+                        expRef, argExpr);
                 castInjected = true;
             }
             //recursively rewrite function arguments
             if (argExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
-                    && TypeCastUtils.getRequiredType((AbstractFunctionCallExpression) argExpr) == null
-                    && reqFieldType != null) {
+                    && TypeCastUtils.getRequiredType((AbstractFunctionCallExpression) argExpr) == null) {
+                AbstractFunctionCallExpression argFunc = (AbstractFunctionCallExpression) argExpr;
                 if (castInjected) {
                     //rewrite the arg expression inside the dynamic cast
-                    ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
-                    rewriteFuncExpr(argFunc, inputFieldType, inputFieldType, env);
+                    rewriteFuncExpr(argFunc, exprActualType, exprActualType, env);
                 } else {
                     //rewrite arg
-                    ScalarFunctionCallExpression argFunc = (ScalarFunctionCallExpression) argExpr;
-                    rewriteFuncExpr(argFunc, reqFieldType, inputFieldType, env);
+                    rewriteFuncExpr(argFunc, requiredType, exprActualType, env);
                 }
             }
         }
         return castInjected;
     }
 
+    private static boolean isComplexConstructor(ILogicalExpression expression) {
+        if (expression.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+            FunctionIdentifier funIdentifier = ((AbstractFunctionCallExpression) expression).getFunctionIdentifier();
+            return funIdentifier.equals(BuiltinFunctions.UNORDERED_LIST_CONSTRUCTOR)
+                    || funIdentifier.equals(BuiltinFunctions.ORDERED_LIST_CONSTRUCTOR)
+                    || funIdentifier.equals(BuiltinFunctions.OPEN_RECORD_CONSTRUCTOR)
+                    || funIdentifier.equals(BuiltinFunctions.CLOSED_RECORD_CONSTRUCTOR);
+        }
+        return false;
+    }
+
     /**
      * Inject a dynamic cast function wrapping an existing expression
      *
@@ -546,12 +568,13 @@ public class StaticTypeCastUtil {
      *            the expression reference
      * @param argExpr
      *            the original expression
-     * @throws AlgebricksException
+     * @throws AlgebricksException if types are incompatible (tag-wise)
      */
     private static void injectCastFunction(IFunctionInfo funcInfo, IAType reqType, IAType inputType,
             Mutable<ILogicalExpression> exprRef, ILogicalExpression argExpr) throws AlgebricksException {
         ScalarFunctionCallExpression cast = new ScalarFunctionCallExpression(funcInfo);
-        cast.getArguments().add(new MutableObject<ILogicalExpression>(argExpr));
+        cast.getArguments().add(new MutableObject<>(argExpr));
+        cast.setSourceLocation(argExpr.getSourceLocation());
         exprRef.setValue(cast);
         TypeCastUtils.setRequiredAndInputTypes(cast, reqType, inputType);
     }
@@ -576,8 +599,8 @@ public class StaticTypeCastUtil {
                 return false;
             }
         }
-        Set<IAType> reqTypePossible = new HashSet<IAType>();
-        Set<IAType> inputTypePossible = new HashSet<IAType>();
+        Set<IAType> reqTypePossible = new HashSet<>();
+        Set<IAType> inputTypePossible = new HashSet<>();
         if (reqType.getTypeTag() == ATypeTag.UNION) {
             AUnionType unionType = (AUnionType) reqType;
             reqTypePossible.addAll(unionType.getUnionList());
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.3.query.sqlpp
index c8673f5..cba606f 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_append/array_append.3.query.sqlpp
@@ -17,17 +17,19 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_append(t.`referred-topics`, "sth", 5) from TweetMessages t order by t.tweetid),
-  "t2": (select array_append([3, "John"], (select value v.compType from d1 v))),
-  "t3": (array_append([3], 7, null, missing)),    // missing
-  "t4": (array_append("non_array", 5)),           // null
-  "t5": (array_append(null, 3, 9)),               // null
-  "t6": (array_append("non_array", 5, missing)),  // missing
-  "t7": (array_append([], 5, 10, 12.0, "sth")),   // OK
-  "t8": (array_append(missing, 3, 9)),            // missing
-  "t9": (array_append([3], 3, [9], null, "sth")), // OK to add nulls
-  "t10": (select array_append(d.followers, "sth1", "sth2") from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_append(t.`referred-topics`, "sth", 5) from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (select array_append([3, "John"], (select value v.compType from d1 v)))},
+  {"id": 3, "t3": (array_append([3], 7, null, missing))},    // missing
+  {"id": 4, "t4": (array_append("non_array", 5))},           // null
+  {"id": 5, "t5": (array_append(null, 3, 9))},               // null
+  {"id": 6, "t6": (array_append("non_array", 5, missing))},  // missing
+  {"id": 7, "t7": (array_append([], 5, 10, 12.0, "sth"))},   // OK
+  {"id": 8, "t8": (array_append(missing, 3, 9))},            // missing
+  {"id": 9, "t9": (array_append([3], 3, [9], null, "sth"))}, // OK to add nulls
+  {"id": 10, "t10": (select array_append(d.followers, "sth1", "sth2") from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.3.query.sqlpp
index 35d959c..f93dd41 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_concat/array_concat.3.query.sqlpp
@@ -17,17 +17,19 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_concat(t.`referred-topics`, {{"sth", 5}}, {{3,2}}) from TweetMessages t order by t.tweetid),
-  "t2": (select array_concat([3, "John"], (select value v.compType from d1 v))),
-  "t3": (array_concat([3,5,1], null, missing)),   // missing
-  "t4": (array_concat([3,1,6], "non_array")),     // null
-  "t5": (array_concat(null, [3, 9])),             // null
-  "t6": (array_concat("non_array", [5], missing)),// missing
-  "t7": (array_concat([], [5, 10], [12.0], [])),  // OK
-  "t8": (array_concat(missing, 3, 9)),            // missing
-  "t9": (array_concat([3, missing], [3, [9], null, missing, null])),  // OK with nulls
-  "t10": (select array_concat(d.followers, ["sth", 5], [3,2]) from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_concat(t.`referred-topics`, {{"sth", 5}}, {{3,2}}) from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (select array_concat([3, "John"], (select value v.compType from d1 v)))},
+  {"id": 3, "t3": (array_concat([3,5,1], null, missing))},   // missing
+  {"id": 4, "t4": (array_concat([3,1,6], "non_array"))},     // null
+  {"id": 5, "t5": (array_concat(null, [3, 9]))},             // null
+  {"id": 6, "t6": (array_concat("non_array", [5], missing))},// missing
+  {"id": 7, "t7": (array_concat([], [5, 10], [12.0], []))},  // OK
+  {"id": 8, "t8": (array_concat(missing, 3, 9))},            // missing
+  {"id": 9, "t9": (array_concat([3, missing], [3, [9], null, missing, null]))},  // OK with nulls
+  {"id": 10, "t10": (select array_concat(d.followers, ["sth", 5], [3,2]) from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.3.query.sqlpp
index dd01484..385cded 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.3.query.sqlpp
@@ -17,18 +17,20 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_distinct(array_append(t.`referred-topics`, "verizon", "platform")) from TweetMessages t order by t.tweetid),
-  "t2": (array_distinct([19, 5, 7, 7, 5, 2])),
-  "t3": (array_distinct([19, 5, 7, 7, 5.1, 5.0, 2, 7])),
-  "t4": (array_distinct([19, 5, "a", 7.5, "A", "a", "John", "a"])),
-  "t5": (array_distinct([19, missing, 7, null, 5, null])),
-  "t6": (array_distinct([3])),
-  "t7": (array_distinct("non_array")),
-  "t8": (array_distinct([])),
-  "t9": (array_distinct(missing)),
-  "t10": (array_distinct(null)),
-  "t11": (select array_distinct(d.followers) from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_distinct(array_append(t.`referred-topics`, "verizon", "platform")) from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (array_distinct([19, 5, 7, 7, 5, 2]))},
+  {"id": 3, "t3": (array_distinct([19, 5, 7, 7, 5.1, 5.0, 2, 7]))},
+  {"id": 4, "t4": (array_distinct([19, 5, "a", 7.5, "A", "a", "John", "a"]))},
+  {"id": 5, "t5": (array_distinct([19, missing, 7, null, 5, null]))},
+  {"id": 6, "t6": (array_distinct([3]))},
+  {"id": 7, "t7": (array_distinct("non_array"))},
+  {"id": 8, "t8": (array_distinct([]))},
+  {"id": 9, "t9": (array_distinct(missing))},
+  {"id": 10, "t10": (array_distinct(null))},
+  {"id": 10, "t11": (select array_distinct(d.followers) from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp
index ad6c9df..cbf50c0 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp
@@ -22,11 +22,13 @@
 * Expected Res : Success
 */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_distinct([ [5,1], [6,2], [9,7], [6,2], [10,11], [6,2], [0,0] ])),
-  "t2": (array_distinct([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 2, "age": 29}, {"id": 1, "age": 34}])),
-  "t3": (from openDs select array_distinct(list_f) order by id),
-  "t4": (from closedDs select array_distinct(list_f) order by id)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_distinct([ [5,1], [6,2], [9,7], [6,2], [10,11], [6,2], [0,0] ]))},
+  {"id": 2, "t2": (array_distinct([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 2, "age": 29}, {"id": 1, "age": 34}]))},
+  {"id": 3, "t3": (from openDs select array_distinct(list_f) order by id)},
+  {"id": 4, "t4": (from closedDs select array_distinct(list_f) order by id)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.3.query.sqlpp
index 323d018..681d22f 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_flatten/array_flatten.3.query.sqlpp
@@ -17,24 +17,26 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_flatten([2, 3, [7,1,2], [8, [12, 13, 14]]], 1)),
-  "t2": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 1)),
-  "t3": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2)),
-  "t4": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 4)),
-  "t5": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 0)),
-  "t6": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], -1)),
-  "t7": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2.0)),  // OK
-  "t8": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2.1)),  // null
-  "t9": (array_flatten([2, 3, null, [7,1, missing, "a"], [8, [12, null, "b", 14, missing]]], 2)),
-  "t10": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], missing)),
-  "t11": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], null)),
-  "t12": (array_flatten(missing, 2)),
-  "t13": (array_flatten(null, 2)),
-  "t14": (array_flatten(null, missing)),
-  "t15": (array_flatten("non_array", 2)),
-  "t16": (array_flatten([1,2,3, [5,6,7]], "non-numeric")),
-  "t17": (select array_flatten(d.followers, 1) from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_flatten([2, 3, [7,1,2], [8, [12, 13, 14]]], 1))},
+  {"id": 2, "t2": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 1))},
+  {"id": 3, "t3": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2))},
+  {"id": 4, "t4": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 4))},
+  {"id": 5, "t5": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 0))},
+  {"id": 6, "t6": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], -1))},
+  {"id": 7, "t7": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2.0))},  // OK
+  {"id": 8, "t8": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], 2.1))},  // null
+  {"id": 9, "t9": (array_flatten([2, 3, null, [7,1, missing, "a"], [8, [12, null, "b", 14, missing]]], 2))},
+  {"id": 10, "t10": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], missing))},
+  {"id": 11, "t11": (array_flatten([2, 3, [7,1,"a"], [8, [12, "b", 14]]], null))},
+  {"id": 12, "t12": (array_flatten(missing, 2))},
+  {"id": 13, "t13": (array_flatten(null, 2))},
+  {"id": 14, "t14": (array_flatten(null, missing))},
+  {"id": 15, "t15": (array_flatten("non_array", 2))},
+  {"id": 16, "t16": (array_flatten([1,2,3, [5,6,7]], "non-numeric"))},
+  {"id": 17, "t17": (select array_flatten(d.followers, 1) from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.3.query.sqlpp
index e2d834e..3baf452 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_ifnull/array_ifnull.3.query.sqlpp
@@ -17,20 +17,22 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_ifnull(t.`referred-topics`) from TweetMessages t order by t.tweetid),
-  "t2": (select array_ifnull((select value v.compType from d1 v))),
-  "t3": (array_ifnull([null, null, 3, 4.9, null])),
-  "t4": (array_ifnull([missing, 2, "a"])),
-  "t5": (array_ifnull([4, 2, "a"])),
-  "t6": (array_ifnull([4, 2, null, missing])),
-  "t7": (array_ifnull([null, null, null])),           // null
-  "t8": (array_ifnull([missing, missing, missing])),  // null
-  "t9": (array_ifnull([missing, null, missing])),   // null
-  "t10": (array_ifnull("non_array")),               // null
-  "t11": (array_ifnull([])),                        // null
-  "t12": (array_ifnull(missing)),                   // missing
-  "t13": (array_ifnull(null))                       // null
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_ifnull(t.`referred-topics`) from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (select array_ifnull((select value v.compType from d1 v)))},
+  {"id": 3, "t3": (array_ifnull([null, null, 3, 4.9, null]))},
+  {"id": 4, "t4": (array_ifnull([missing, 2, "a"]))},
+  {"id": 5, "t5": (array_ifnull([4, 2, "a"]))},
+  {"id": 6, "t6": (array_ifnull([4, 2, null, missing]))},
+  {"id": 7, "t7": (array_ifnull([null, null, null]))},           // null
+  {"id": 8, "t8": (array_ifnull([missing, missing, missing]))},  // null
+  {"id": 9, "t9": (array_ifnull([missing, null, missing]))},   // null
+  {"id": 10, "t10": (array_ifnull("non_array"))},               // null
+  {"id": 11, "t11": (array_ifnull([]))},                        // null
+  {"id": 12, "t12": (array_ifnull(missing))},                   // missing
+  {"id": 13, "t13": (array_ifnull(null))}                       // null
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.3.query.sqlpp
index c0b65e7..6b93b2b 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_insert/array_insert.3.query.sqlpp
@@ -17,31 +17,33 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_insert([1,2,3], 0, "a", "b")),          // OK
-  "t2": (array_insert([1,2,3], 3, "a", "b")),          // OK
-  "t3": (array_insert([1,1,2,4], 3, "a", "b")),        // OK
-  "t4": (array_insert([1,1,2,4], 3, 7, "a", 7, "one more")),  // OK
-  "t5": (array_insert([1,2,3], 4, "a")),          // null, out of bound
-  "t6": (array_insert([1,2,3], -1, "a", "b")),    // OK
-  "t7": (array_insert([1,2,3], -4, "a", "b")),    // null, out of bound
-  "t8": (array_insert("non_array", 5, "val")),    // null
-  "t9": (array_insert("non_array", 5, missing)),  // missing
-  "t10": (array_insert([], 5, 10, 12.0, "sth")),  // null, out of bound
-  "t11": (array_insert([], 0, 10, 12.0, "sth")),  // OK
-  "t12": (array_insert([6], "a", 9)),             // null, position non-numeric
-  "t13": (array_insert([6], 1.0, 9)),             // OK
-  "t14": (array_insert([6], 1.5, 9)),             // null, position with decimals
-  "t15": (array_insert(null, 3, 9)),              // null
-  "t16": (array_insert(missing, 3, 9)),           // missing
-  "t17": (array_insert([6], 1, null, 9, null)),   // OK to insert nulls
-  "t18": (array_insert([6], null, 5, 9, null)),   // null
-  "t19": (array_insert([6], 3, null, missing, 9, null)),  // missing
-  "t20": (select array_insert(t.`referred-topics`, 0, 5) from TweetMessages t order by t.tweetid),
-  "t21": (select array_insert(d.followers, 0, 5, 3) from d1 d),
-  "t22": (array_insert([1,2,3], float("NaN"), "a", "b")),
-  "t23": (array_insert([1,2,3], float("INF"), "a", "b")),
-  "t24": (array_insert([1,2,3], float("-INF"), "a", "b"))
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_insert([1,2,3], 0, "a", "b"))},          // OK
+  {"id": 2, "t2": (array_insert([1,2,3], 3, "a", "b"))},          // OK
+  {"id": 3, "t3": (array_insert([1,1,2,4], 3, "a", "b"))},        // OK
+  {"id": 4, "t4": (array_insert([1,1,2,4], 3, 7, "a", 7, "one more"))},  // OK
+  {"id": 5, "t5": (array_insert([1,2,3], 4, "a"))},          // null, out of bound
+  {"id": 6, "t6": (array_insert([1,2,3], -1, "a", "b"))},    // OK
+  {"id": 7, "t7": (array_insert([1,2,3], -4, "a", "b"))},    // null, out of bound
+  {"id": 8, "t8": (array_insert("non_array", 5, "val"))},    // null
+  {"id": 9, "t9": (array_insert("non_array", 5, missing))},  // missing
+  {"id": 10, "t10": (array_insert([], 5, 10, 12.0, "sth"))},  // null, out of bound
+  {"id": 11, "t11": (array_insert([], 0, 10, 12.0, "sth"))},  // OK
+  {"id": 12, "t12": (array_insert([6], "a", 9))},             // null, position non-numeric
+  {"id": 13, "t13": (array_insert([6], 1.0, 9))},             // OK
+  {"id": 14, "t14": (array_insert([6], 1.5, 9))},             // null, position with decimals
+  {"id": 15, "t15": (array_insert(null, 3, 9))},              // null
+  {"id": 16, "t16": (array_insert(missing, 3, 9))},           // missing
+  {"id": 17, "t17": (array_insert([6], 1, null, 9, null))},   // OK to insert nulls
+  {"id": 18, "t18": (array_insert([6], null, 5, 9, null))},   // null
+  {"id": 19, "t19": (array_insert([6], 3, null, missing, 9, null))},  // missing
+  {"id": 20, "t20": (select array_insert(t.`referred-topics`, 0, 5) from TweetMessages t order by t.tweetid)},
+  {"id": 21, "t21": (select array_insert(d.followers, 0, 5, 3) from d1 d)},
+  {"id": 22, "t22": (array_insert([1,2,3], float("NaN"), "a", "b"))},
+  {"id": 23, "t23": (array_insert([1,2,3], float("INF"), "a", "b"))},
+  {"id": 24, "t24": (array_insert([1,2,3], float("-INF"), "a", "b"))}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.3.query.sqlpp
index 1603570e..076293b 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_prepend/array_prepend.3.query.sqlpp
@@ -17,17 +17,19 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_prepend("sth", 5, t.`referred-topics`) from TweetMessages t order by t.tweetid),
-  "t2": (select array_prepend((select value v.compType from d1 v), [3, "John"])),
-  "t3": (array_prepend(7, null, missing, [3])),     // missing
-  "t4": (array_prepend(5, "non_array")),            // null
-  "t5": (array_prepend(3, 9, null)),                // null
-  "t6": (array_prepend(5, missing, "non_array")),   // missing
-  "t7": (array_prepend(5, 10, 12.0, "sth",[77, "val"])),   // OK
-  "t8": (array_prepend(5, 10, null, "sth",[77, "val"])),   // OK to insert nulls
-  "t9": (array_prepend(3, 9, missing)),              // missing
-  "t10": (select array_prepend("sth1", "sth2", d.followers) from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_prepend("sth", 5, t.`referred-topics`) from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (select array_prepend((select value v.compType from d1 v), [3, "John"]))},
+  {"id": 3, "t3": (array_prepend(7, null, missing, [3]))},     // missing
+  {"id": 4, "t4": (array_prepend(5, "non_array"))},            // null
+  {"id": 5, "t5": (array_prepend(3, 9, null))},                // null
+  {"id": 6, "t6": (array_prepend(5, missing, "non_array"))},   // missing
+  {"id": 7, "t7": (array_prepend(5, 10, 12.0, "sth",[77, "val"]))},   // OK
+  {"id": 8, "t8": (array_prepend(5, 10, null, "sth",[77, "val"]))},   // OK to insert nulls
+  {"id": 9, "t9": (array_prepend(3, 9, missing))},              // missing
+  {"id": 10, "t10": (select array_prepend("sth1", "sth2", d.followers) from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.3.query.sqlpp
index 44ff8f2..7579b33 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.3.query.sqlpp
@@ -17,18 +17,20 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_put(t.`referred-topics`, "t-mobile", 5) from TweetMessages t order by t.tweetid),
-  "t2": (array_put([3], 7, null, missing)),    // missing
-  "t3": (array_put("non_array", 5)),           // null
-  "t4": (array_put("non_array", 5, missing)),  // missing
-  "t5": (array_put([], 5, 10, 12.0, "sth")),   // OK
-  "t6": (array_put(missing, 3, 9)),            // missing
-  "t7": (array_put(null, 3, 9)),               // null
-  "t8": (array_put([3,2,"sth"], 3, 9, 9, 3, "sth")), // OK
-  "t9": (array_put([3,2,"sth"], 1, 5)),     // OK
-  "t10": (array_put([3,2,"sth"], null, 5)),   // null
-  "t11": (select array_put(d.followers, "sth1", "John Green", "sth2") from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_put(t.`referred-topics`, "t-mobile", 5) from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (array_put([3], 7, null, missing))},    // missing
+  {"id": 3, "t3": (array_put("non_array", 5))},           // null
+  {"id": 4, "t4": (array_put("non_array", 5, missing))},  // missing
+  {"id": 5, "t5": (array_put([], 5, 10, 12.0, "sth"))},   // OK
+  {"id": 6, "t6": (array_put(missing, 3, 9))},            // missing
+  {"id": 7, "t7": (array_put(null, 3, 9))},               // null
+  {"id": 8, "t8": (array_put([3,2,"sth"], 3, 9, 9, 3, "sth"))}, // OK
+  {"id": 9, "t9": (array_put([3,2,"sth"], 1, 5))},     // OK
+  {"id": 10, "t10": (array_put([3,2,"sth"], null, 5))},   // null
+  {"id": 11, "t11": (select array_put(d.followers, "sth1", "John Green", "sth2") from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.4.query.sqlpp
index 61e41cf..3b2d4f7 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_put/array_put.4.query.sqlpp
@@ -22,12 +22,14 @@
 * Expected Res : Success
 */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_put([1,2,3, [9,8]], 5, ["adding a list", "yes"])),
-  "t2": (array_put([ [5,1,2] , [3,2] , [90,100] ], [3,2], [80,100])),
-  "t3": (array_put([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}], {"id": 4, "age": 90}, {"id": 2, "age": 29})),
-  "t4": (from openDs select array_put(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id),
-  "t5": (from closedDs select array_put(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_put([1,2,3, [9,8]], 5, ["adding a list", "yes"]))},
+  {"id": 2, "t2": (array_put([ [5,1,2] , [3,2] , [90,100] ], [3,2], [80,100]))},
+  {"id": 3, "t3": (array_put([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}], {"id": 4, "age": 90}, {"id": 2, "age": 29}))},
+  {"id": 4, "t4": (from openDs select array_put(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id)},
+  {"id": 5, "t5": (from closedDs select array_put(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.3.query.sqlpp
index d36a9f1..e2ecacb 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_range/array_range.3.query.sqlpp
@@ -17,27 +17,29 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_range(1,6)),
-  "t2": (array_range(0,25,5)),
-  "t3": (array_range(0.1,2)),
-  "t4": (array_range(10, 3, -3)),
-  "t5": (array_range(-1, -8, -3)),
-  "t6": (array_range(1, 8, -3)),
-  "t7": (array_range(10, 2, 3)),
-  "t8": (array_range(1, 13, 0)),
-  "t9": (array_range(2, 2, 1)),
-  "t10": (array_range(2, 3, 1)),
-  "t11": (array_range(2, 9, 1.1)),
-  "t12": (array_range(2, 9, null)),
-  "t13": (array_range(2, missing, 1)),
-  "t14": (array_range(2, missing)),
-  "t15": (array_range(2, 7, "a")),
-  "t16": (array_range(2, "a", 1)),
-  "t17": (array_range("a", 7, 1)),
-  "t18": (array_range(5, 10, float("NaN"))),
-  "t19": (array_range(5, float("INF"), 1)),
-  "t20": (array_range(5, float("-INF"), -2))
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_range(1,6))},
+  {"id": 2, "t2": (array_range(0,25,5))},
+  {"id": 3, "t3": (array_range(0.1,2))},
+  {"id": 4, "t4": (array_range(10, 3, -3))},
+  {"id": 5, "t5": (array_range(-1, -8, -3))},
+  {"id": 6, "t6": (array_range(1, 8, -3))},
+  {"id": 7, "t7": (array_range(10, 2, 3))},
+  {"id": 8, "t8": (array_range(1, 13, 0))},
+  {"id": 9, "t9": (array_range(2, 2, 1))},
+  {"id": 10, "t10": (array_range(2, 3, 1))},
+  {"id": 11, "t11": (array_range(2, 9, 1.1))},
+  {"id": 12, "t12": (array_range(2, 9, null))},
+  {"id": 13, "t13": (array_range(2, missing, 1))},
+  {"id": 14, "t14": (array_range(2, missing))},
+  {"id": 15, "t15": (array_range(2, 7, "a"))},
+  {"id": 16, "t16": (array_range(2, "a", 1))},
+  {"id": 17, "t17": (array_range("a", 7, 1))},
+  {"id": 18, "t18": (array_range(5, 10, float("NaN")))},
+  {"id": 19, "t19": (array_range(5, float("INF"), 1))},
+  {"id": 20, "t20": (array_range(5, float("-INF"), -2))}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.3.query.sqlpp
index ca1d2af..05213bd 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.3.query.sqlpp
@@ -17,17 +17,19 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_remove(t.`referred-topics`, "t-mobile") from TweetMessages t order by t.tweetid),
-  "t2": (array_remove([3], 7, null, missing)),    // missing
-  "t3": (array_remove("non_array", 5)),           // null
-  "t4": (array_remove("non_array", 5, missing)),  // missing
-  "t5": (array_remove([], 5, 10, 12.0, "sth")),   // OK
-  "t6": (array_remove(missing, 3, 9)),            // missing
-  "t7": (array_remove(null, 3, 9)),               // null
-  "t8": (array_remove([3,2,"sth"], 3, 9, 9, 3, "sth")), // OK
-  "t9": (array_remove([3,2,"sth"], 1, null)),      // null
-  "t10": (select array_remove(d.followers, "sth1", "John Green") from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_remove(t.`referred-topics`, "t-mobile") from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (array_remove([3], 7, null, missing))},    // missing
+  {"id": 3, "t3": (array_remove("non_array", 5))},           // null
+  {"id": 4, "t4": (array_remove("non_array", 5, missing))},  // missing
+  {"id": 5, "t5": (array_remove([], 5, 10, 12.0, "sth"))},   // OK
+  {"id": 6, "t6": (array_remove(missing, 3, 9))},            // missing
+  {"id": 7, "t7": (array_remove(null, 3, 9))},               // null
+  {"id": 8, "t8": (array_remove([3,2,"sth"], 3, 9, 9, 3, "sth"))}, // OK
+  {"id": 9, "t9": (array_remove([3,2,"sth"], 1, null))},      // null
+  {"id": 10, "t10": (select array_remove(d.followers, "sth1", "John Green") from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.4.query.sqlpp
index efaaa85..88304bc 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.4.query.sqlpp
@@ -22,12 +22,14 @@
 * Expected Res : Success
 */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_remove([1,2,3, [9,8], ["str1", "str2"], [90,100]], [9,8], [90,100])),
-  "t2": (array_remove([ [5,1,2] , [3,2] , [90,100] ], [3,2], [80,100])),
-  "t3": (array_remove([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}], {"id": 4, "age": 90}, {"id": 2, "age": 29})),
-  "t4": (from openDs select array_remove(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id),
-  "t5": (from closedDs select array_remove(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_remove([1,2,3, [9,8], ["str1", "str2"], [90,100]], [9,8], [90,100]))},
+  {"id": 2, "t2": (array_remove([ [5,1,2] , [3,2] , [90,100] ], [3,2], [80,100]))},
+  {"id": 3, "t3": (array_remove([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}], {"id": 4, "age": 90}, {"id": 2, "age": 29}))},
+  {"id": 4, "t4": (from openDs select array_remove(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id)},
+  {"id": 5, "t5": (from closedDs select array_remove(list_f, [9999,3], {"state": "OH", "country": "US"}) order by id)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.query.sqlpp
similarity index 63%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.query.sqlpp
index ad6c9df..ce03884 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_distinct/array_distinct.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.query.sqlpp
@@ -18,15 +18,16 @@
  */
 
 /*
-* Description  : Testing an array function that needs to compare elements
+* Description  : Checking casting the records to open in the FROM list since it's a heterogeneous list
 * Expected Res : Success
 */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_distinct([ [5,1], [6,2], [9,7], [6,2], [10,11], [6,2], [0,0] ])),
-  "t2": (array_distinct([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 2, "age": 29}, {"id": 1, "age": 34}])),
-  "t3": (from openDs select array_distinct(list_f) order by id),
-  "t4": (from closedDs select array_distinct(list_f) order by id)
-};
\ No newline at end of file
+EXPLAIN
+FROM [
+  {"id": 1, "t1": (array_remove([1,2,3, [9,8], ["str1", "str2"], [90,100]], [9,8], [90,100]))},
+  {"id": 2, "t2": (array_remove([ [5,1,2] , [3,2] , [90,100] ], [3,2], [80,100]))},
+  {"id": 3, "t3": (array_remove([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}], {"id": 4, "age": 90}, {"id": 2, "age": 29}))}
+] AS d
+SELECT VALUE d;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.6.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.5.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_remove/array_remove.6.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.3.query.sqlpp
index 217d8a7..77512e0 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.3.query.sqlpp
@@ -17,32 +17,34 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_replace(t.`referred-topics`, "t-mobile", "coffee-mobile") from TweetMessages t order by t.tweetid),
-  "t2": (array_replace([3,-2,1,5,2,9,2,3,2,1,2], 2, -2)),
-  "t3": (array_replace([3,-2,1,5,2,9,2,3,2,1,2], 2.0, -4)),
-  "t4": (array_replace([3,-2,1,5,"2",9,2,3,2,1,2], 2.0, -4)),
-  "t5": (array_replace([3,-2,"a",5,"2",9,"a",3,"A",1,2], "a", -2)),
-  "t6": (array_replace([3,-2,"a",5,"2",9,[2,3,"A"],1,2], 2, -2)),
-  "t7": (array_replace([3,-2,"a",5,"2",9,[2,3,"A"],1,2], 2, [-2])),
-  "t8": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 3)),
-  "t9": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 0)),
-  "t10": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 4)),
-  "t11": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 5)),
-  "t12": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 10)),
-  "t13": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, -3)),
-  "t14": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 1.0)),
-  "t15": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 1.2)),
-  "t16": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, "4")),
-  "t17": (array_replace("non_array", 1, -5, 3)),
-  "t18": (array_replace([], 1, -5, 3)),
-  "t19": (array_replace([3,2], 1, -5, 3)),
-  "t20": (array_replace([3,2,3], 3, missing, 8)),
-  "t21": (array_replace([3,2,3], 3, null, 8)),
-  "t22": (array_replace([3,2], null, 3, 3)),
-  "t23": (array_replace([3,null,2], null, 3, 3)),
-  "t24": (array_replace(missing, null, 3, 3)),
-  "t25": (select array_replace(d.followers, "Emily Jones", "sth") from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_replace(t.`referred-topics`, "t-mobile", "coffee-mobile") from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (array_replace([3,-2,1,5,2,9,2,3,2,1,2], 2, -2))},
+  {"id": 3, "t3": (array_replace([3,-2,1,5,2,9,2,3,2,1,2], 2.0, -4))},
+  {"id": 4, "t4": (array_replace([3,-2,1,5,"2",9,2,3,2,1,2], 2.0, -4))},
+  {"id": 5, "t5": (array_replace([3,-2,"a",5,"2",9,"a",3,"A",1,2], "a", -2))},
+  {"id": 6, "t6": (array_replace([3,-2,"a",5,"2",9,[2,3,"A"],1,2], 2, -2))},
+  {"id": 7, "t7": (array_replace([3,-2,"a",5,"2",9,[2,3,"A"],1,2], 2, [-2]))},
+  {"id": 8, "t8": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 3))},
+  {"id": 9, "t9": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 0))},
+  {"id": 10, "t10": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 4))},
+  {"id": 11, "t11": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 5))},
+  {"id": 12, "t12": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 10))},
+  {"id": 13, "t13": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, -3))},
+  {"id": 14, "t14": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 1.0))},
+  {"id": 15, "t15": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, 1.2))},
+  {"id": 16, "t16": (array_replace([3,1,1,2,2,1,2,1,1], 1, -5, "4"))},
+  {"id": 17, "t17": (array_replace("non_array", 1, -5, 3))},
+  {"id": 18, "t18": (array_replace([], 1, -5, 3))},
+  {"id": 19, "t19": (array_replace([3,2], 1, -5, 3))},
+  {"id": 20, "t20": (array_replace([3,2,3], 3, missing, 8))},
+  {"id": 21, "t21": (array_replace([3,2,3], 3, null, 8))},
+  {"id": 22, "t22": (array_replace([3,2], null, 3, 3))},
+  {"id": 23, "t23": (array_replace([3,null,2], null, 3, 3))},
+  {"id": 24, "t24": (array_replace(missing, null, 3, 3))},
+  {"id": 25, "t25": (select array_replace(d.followers, "Emily Jones", "sth") from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.4.query.sqlpp
index 922c6a8..93017a4 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_replace/array_replace.4.query.sqlpp
@@ -22,12 +22,14 @@
 * Expected Res : Success
 */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_replace([ [5,1], [6,2], [9,7], [6,2], [10,11]], [6,2], [0,0])),
-  "t2": (array_replace([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 4, "age": 10}], {"id": 2, "age": 29}, {"id": 8, "age": 200})),
-  "t3": (from openDs select array_replace(list_f, [9999,3], "replace done") order by id),
-  "t4": (from openDs select array_replace(list_f, {"state": "OH", "country": "US"}, "replace done") order by id),
-  "t5": (from closedDs select array_replace(list_f, {"state": "OH", "country": "US"}, "replace done") order by id)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_replace([ [5,1], [6,2], [9,7], [6,2], [10,11]], [6,2], [0,0]))},
+  {"id": 2, "t2": (array_replace([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 4, "age": 10}], {"id": 2, "age": 29}, {"id": 8, "age": 200}))},
+  {"id": 3, "t3": (from openDs select array_replace(list_f, [9999,3], "replace done") order by id)},
+  {"id": 4, "t4": (from openDs select array_replace(list_f, {"state": "OH", "country": "US"}, "replace done") order by id)},
+  {"id": 5, "t5": (from closedDs select array_replace(list_f, {"state": "OH", "country": "US"}, "replace done") order by id)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.3.query.sqlpp
index 55c3569..277ab05 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.3.query.sqlpp
@@ -17,18 +17,20 @@
  * under the License.
  */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (select array_sort(t.`referred-topics`) from TweetMessages t order by t.tweetid),
-  "t2": (array_sort([19, 5, 7, 7, 5, 2])),
-  "t3": (array_sort([19, 5, 7, 7, 5.1, 5.0, 2])),
-  "t4": (array_sort([19, 5, "a", 7.5, "A", "John", "a"])),
-  "t5": (array_sort([19, missing, 7, null, 5, null])),
-  "t6": (array_sort([3])),
-  "t7": (array_sort("non_array")),
-  "t8": (array_sort([])),
-  "t9": (array_sort(missing)),
-  "t10": (array_sort(null)),
-  "t11": (select array_sort(d.followers) from d1 d)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (select array_sort(t.`referred-topics`) from TweetMessages t order by t.tweetid)},
+  {"id": 2, "t2": (array_sort([19, 5, 7, 7, 5, 2]))},
+  {"id": 3, "t3": (array_sort([19, 5, 7, 7, 5.1, 5.0, 2]))},
+  {"id": 4, "t4": (array_sort([19, 5, "a", 7.5, "A", "John", "a"]))},
+  {"id": 5, "t5": (array_sort([19, missing, 7, null, 5, null]))},
+  {"id": 6, "t6": (array_sort([3]))},
+  {"id": 7, "t7": (array_sort("non_array"))},
+  {"id": 8, "t8": (array_sort([]))},
+  {"id": 9, "t9": (array_sort(missing))},
+  {"id": 10, "t10": (array_sort(null))},
+  {"id": 11, "t11": (select array_sort(d.followers) from d1 d)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.4.query.sqlpp
index e3a0a4b..c765a31 100755
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array_fun/array_sort/array_sort.4.query.sqlpp
@@ -22,11 +22,13 @@
 * Expected Res : Success
 */
 
-use TinySocial;
+USE TinySocial;
 
-{
-  "t1": (array_sort([ [5,1], [6,2], [9,7], [6,2], [10,11], [6,2], [0,0] ])),
-  "t2": (array_sort([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 4, "age": 10}, {"id": 5, "age": 90}])),
-  "t3": (from openDs select array_sort(list_f) order by id),
-  "t4": (from closedDs select array_sort(list_f) order by id)
-};
\ No newline at end of file
+FROM [
+  {"id": 1, "t1": (array_sort([ [5,1], [6,2], [9,7], [6,2], [10,11], [6,2], [0,0] ]))},
+  {"id": 2, "t2": (array_sort([ {"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}, {"id": 4, "age": 10}, {"id": 5, "age": 90}]))},
+  {"id": 3, "t3": (from openDs select array_sort(list_f) order by id)},
+  {"id": 4, "t4": (from closedDs select array_sort(list_f) order by id)}
+] AS d
+SELECT VALUE d
+ORDER BY d.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp
index b79bd8b..f8169bb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.3.query.sqlpp
@@ -35,14 +35,5 @@ use TinySocial;
 
   "t2": object_unwrap({"a":"1"}),
   "t3": object_unwrap({"a":{"b":"1"}}),
-  "t4": object_unwrap({"a":[1, 2]}),
-
-  "t5": (
-    select object_unwrap(o) from (
-      select lang
-      from TwitterUsers u
-      order by lang
-      limit 1
-    ) o
-  )
+  "t4": object_unwrap({"a":[1, 2]})
 };
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.query.sqlpp
similarity index 88%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.query.sqlpp
index af175df..ad4fe11 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.query.sqlpp
@@ -22,4 +22,11 @@
  * Expected Res : Success
  */
 
-drop dataverse TinySocial;
\ No newline at end of file
+use TinySocial;
+
+select object_unwrap(o) from (
+    select lang
+    from TwitterUsers u
+    order by lang
+    limit 1
+) o;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.5.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.4.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_unwrap/object_unwarp.5.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_append/array_append.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_append/array_append.3.adm
index f1f17c2..029d71b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_append/array_append.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_append/array_append.3.adm
@@ -1 +1,10 @@
-{ "t1": [ { "$1": {{ "t-mobile", "customization", "sth", 5 }} }, { "$1": {{ "verizon", "voice-clarity", "sth", 5 }} }, { "$1": {{ "iphone", "platform", "sth", 5 }} }, { "$1": {{ "samsung", "voice-command", "sth", 5 }} }, { "$1": {{ "verizon", "shortcut-menu", "sth", 5 }} }, { "$1": {{ "motorola", "speed", "sth", 5 }} }, { "$1": {{ "sprint", "voice-command", "sth", 5 }} }, { "$1": {{ "motorola", "speed", "sth", 5 }} }, { "$1": {{ "iphone", "voice-clarity", "sth", 5 }} }, { "$1": {{ "samsu [...]
+{ "id": 1, "t1": [ { "$1": {{ "t-mobile", "customization", "sth", 5 }} }, { "$1": {{ "verizon", "voice-clarity", "sth", 5 }} }, { "$1": {{ "iphone", "platform", "sth", 5 }} }, { "$1": {{ "samsung", "voice-command", "sth", 5 }} }, { "$1": {{ "verizon", "shortcut-menu", "sth", 5 }} }, { "$1": {{ "motorola", "speed", "sth", 5 }} }, { "$1": {{ "sprint", "voice-command", "sth", 5 }} }, { "$1": {{ "motorola", "speed", "sth", 5 }} }, { "$1": {{ "iphone", "voice-clarity", "sth", 5 }} }, { "$1":  [...]
+{ "id": 2, "t2": [ { "$2": [ 3, "John", [ { "sth": 33 }, { "sth": 44 } ] ] } ] }
+{ "id": 3 }
+{ "id": 4, "t4": null }
+{ "id": 5, "t5": null }
+{ "id": 6 }
+{ "id": 7, "t7": [ 5, 10, 12.0, "sth" ] }
+{ "id": 8 }
+{ "id": 9, "t9": [ 3, 3, [ 9 ], null, "sth" ] }
+{ "id": 10, "t10": [ {  }, { "$3": [ "John Green", "Emily Jones", "sth1", "sth2" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_concat/array_concat.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_concat/array_concat.3.adm
index 2ee9cbc..5844b5a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_concat/array_concat.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_concat/array_concat.3.adm
@@ -1 +1,10 @@
-{ "t1": [ { "$1": {{ "t-mobile", "customization", "sth", 5, 3, 2 }} }, { "$1": {{ "verizon", "voice-clarity", "sth", 5, 3, 2 }} }, { "$1": {{ "iphone", "platform", "sth", 5, 3, 2 }} }, { "$1": {{ "samsung", "voice-command", "sth", 5, 3, 2 }} }, { "$1": {{ "verizon", "shortcut-menu", "sth", 5, 3, 2 }} }, { "$1": {{ "motorola", "speed", "sth", 5, 3, 2 }} }, { "$1": {{ "sprint", "voice-command", "sth", 5, 3, 2 }} }, { "$1": {{ "motorola", "speed", "sth", 5, 3, 2 }} }, { "$1": {{ "iphone", " [...]
+{ "id": 1, "t1": [ { "$1": {{ "t-mobile", "customization", "sth", 5, 3, 2 }} }, { "$1": {{ "verizon", "voice-clarity", "sth", 5, 3, 2 }} }, { "$1": {{ "iphone", "platform", "sth", 5, 3, 2 }} }, { "$1": {{ "samsung", "voice-command", "sth", 5, 3, 2 }} }, { "$1": {{ "verizon", "shortcut-menu", "sth", 5, 3, 2 }} }, { "$1": {{ "motorola", "speed", "sth", 5, 3, 2 }} }, { "$1": {{ "sprint", "voice-command", "sth", 5, 3, 2 }} }, { "$1": {{ "motorola", "speed", "sth", 5, 3, 2 }} }, { "$1": {{ "i [...]
+{ "id": 2, "t2": [ { "$2": [ 3, "John", { "sth": 33 }, { "sth": 44 } ] } ] }
+{ "id": 3 }
+{ "id": 4, "t4": null }
+{ "id": 5, "t5": null }
+{ "id": 6 }
+{ "id": 7, "t7": [ 5, 10, 12.0 ] }
+{ "id": 8 }
+{ "id": 9, "t9": [ 3, null, 3, [ 9 ], null, null, null ] }
+{ "id": 10, "t10": [ {  }, { "$3": [ "John Green", "Emily Jones", "sth", 5, 3, 2 ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.3.adm
index 7461980..c3c9571 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.3.adm
@@ -1 +1,11 @@
-{ "t1": [ { "$1": {{ "t-mobile", "customization", "verizon", "platform" }} }, { "$1": {{ "verizon", "voice-clarity", "platform" }} }, { "$1": {{ "iphone", "platform", "verizon" }} }, { "$1": {{ "samsung", "voice-command", "verizon", "platform" }} }, { "$1": {{ "verizon", "shortcut-menu", "platform" }} }, { "$1": {{ "motorola", "speed", "verizon", "platform" }} }, { "$1": {{ "sprint", "voice-command", "verizon", "platform" }} }, { "$1": {{ "motorola", "speed", "verizon", "platform" }} },  [...]
+{ "id": 1, "t1": [ { "$1": {{ "t-mobile", "customization", "verizon", "platform" }} }, { "$1": {{ "verizon", "voice-clarity", "platform" }} }, { "$1": {{ "iphone", "platform", "verizon" }} }, { "$1": {{ "samsung", "voice-command", "verizon", "platform" }} }, { "$1": {{ "verizon", "shortcut-menu", "platform" }} }, { "$1": {{ "motorola", "speed", "verizon", "platform" }} }, { "$1": {{ "sprint", "voice-command", "verizon", "platform" }} }, { "$1": {{ "motorola", "speed", "verizon", "platfor [...]
+{ "id": 2, "t2": [ 19, 5, 7, 2 ] }
+{ "id": 3, "t3": [ 19, 5, 7, 5.1, 2 ] }
+{ "id": 4, "t4": [ 19, 5, "a", 7.5, "A", "John" ] }
+{ "id": 5, "t5": [ 19, null, 7, 5 ] }
+{ "id": 6, "t6": [ 3 ] }
+{ "id": 7, "t7": null }
+{ "id": 8, "t8": [  ] }
+{ "id": 9 }
+{ "id": 10, "t10": null }
+{ "id": 10, "t11": [ {  }, { "$2": [ "John Green", "Emily Jones", "sth" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.4.adm
index 39263f4..ad26fe3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_distinct/array_distinct.4.adm
@@ -1 +1,4 @@
-{ "t1": [ [ 5, 1 ], [ 6, 2 ], [ 9, 7 ], [ 10, 11 ], [ 0, 0 ] ], "t2": [ { "id": 1, "age": 34 }, { "id": 2, "age": 29 }, { "id": 3, "age": 90 } ], "t3": [ { "$1": [ [ 1, 2, 1 ], [ 9999, 3 ], [ 5, 1 ], [ 4, 2 ], [ 1, 2 ] ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ], [ "blue", "orange" ] ] }, { "$1": [ { "state": "OH", "country": "US" }, { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$1": null }, {  } ], "t4": [ { "$2": [ { "state": "OH", "country": "US"  [...]
\ No newline at end of file
+{ "id": 1, "t1": [ [ 5, 1 ], [ 6, 2 ], [ 9, 7 ], [ 10, 11 ], [ 0, 0 ] ] }
+{ "id": 2, "t2": [ { "id": 1, "age": 34 }, { "id": 2, "age": 29 }, { "id": 3, "age": 90 } ] }
+{ "id": 3, "t3": [ { "$1": [ [ 1, 2, 1 ], [ 9999, 3 ], [ 5, 1 ], [ 4, 2 ], [ 1, 2 ] ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ], [ "blue", "orange" ] ] }, { "$1": [ { "state": "OH", "country": "US" }, { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$1": null }, {  } ] }
+{ "id": 4, "t4": [ { "$2": [ { "state": "OH", "country": "US" }, { "state": "CA", "country": "US", "zip_code": 92863 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_flatten/array_flatten.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_flatten/array_flatten.3.adm
index 72de628..e1bfda0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_flatten/array_flatten.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_flatten/array_flatten.3.adm
@@ -1 +1,17 @@
-{ "t1": [ 2, 3, 7, 1, 2, 8, [ 12, 13, 14 ] ], "t2": [ 2, 3, 7, 1, "a", 8, [ 12, "b", 14 ] ], "t3": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t4": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t5": [ 2, 3, [ 7, 1, "a" ], [ 8, [ 12, "b", 14 ] ] ], "t6": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t7": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ], "t8": null, "t9": [ 2, 3, null, 7, 1, null, "a", 8, 12, null, "b", 14, null ], "t11": null, "t13": null, "t15": null, "t16": null, "t17": [ {  }, { "$1": [ "John Green", "E [...]
+{ "id": 1, "t1": [ 2, 3, 7, 1, 2, 8, [ 12, 13, 14 ] ] }
+{ "id": 2, "t2": [ 2, 3, 7, 1, "a", 8, [ 12, "b", 14 ] ] }
+{ "id": 3, "t3": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ] }
+{ "id": 4, "t4": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ] }
+{ "id": 5, "t5": [ 2, 3, [ 7, 1, "a" ], [ 8, [ 12, "b", 14 ] ] ] }
+{ "id": 6, "t6": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ] }
+{ "id": 7, "t7": [ 2, 3, 7, 1, "a", 8, 12, "b", 14 ] }
+{ "id": 8, "t8": null }
+{ "id": 9, "t9": [ 2, 3, null, 7, 1, null, "a", 8, 12, null, "b", 14, null ] }
+{ "id": 10 }
+{ "id": 11, "t11": null }
+{ "id": 12 }
+{ "id": 13, "t13": null }
+{ "id": 14 }
+{ "id": 15, "t15": null }
+{ "id": 16, "t16": null }
+{ "id": 17, "t17": [ {  }, { "$1": [ "John Green", "Emily Jones", "sth" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_ifnull/array_ifnull.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_ifnull/array_ifnull.3.adm
index a7bc50e..c9ea1ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_ifnull/array_ifnull.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_ifnull/array_ifnull.3.adm
@@ -1 +1,13 @@
-{ "t1": [ { "$1": "t-mobile" }, { "$1": "verizon" }, { "$1": "iphone" }, { "$1": "samsung" }, { "$1": "verizon" }, { "$1": "motorola" }, { "$1": "sprint" }, { "$1": "motorola" }, { "$1": "iphone" }, { "$1": "samsung" }, { "$1": "t-mobile" }, { "$1": "verizon" } ], "t2": [ { "$2": { "sth": 44 } } ], "t7": null, "t8": null, "t10": null, "t13": null, "t3": 3, "t4": 2, "t5": 4, "t6": 4, "t9": null, "t11": null }
+{ "id": 1, "t1": [ { "$1": "t-mobile" }, { "$1": "verizon" }, { "$1": "iphone" }, { "$1": "samsung" }, { "$1": "verizon" }, { "$1": "motorola" }, { "$1": "sprint" }, { "$1": "motorola" }, { "$1": "iphone" }, { "$1": "samsung" }, { "$1": "t-mobile" }, { "$1": "verizon" } ] }
+{ "id": 2, "t2": [ { "$2": { "sth": 44 } } ] }
+{ "id": 3, "t3": 3 }
+{ "id": 4, "t4": 2 }
+{ "id": 5, "t5": 4 }
+{ "id": 6, "t6": 4 }
+{ "id": 7, "t7": null }
+{ "id": 8, "t8": null }
+{ "id": 9, "t9": null }
+{ "id": 10, "t10": null }
+{ "id": 11, "t11": null }
+{ "id": 12 }
+{ "id": 13, "t13": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_insert/array_insert.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_insert/array_insert.3.adm
index 0acb072..7fc5b4d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_insert/array_insert.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_insert/array_insert.3.adm
@@ -1 +1,24 @@
-{ "t1": [ "a", "b", 1, 2, 3 ], "t2": [ 1, 2, 3, "a", "b" ], "t3": [ 1, 1, 2, "a", "b", 4 ], "t4": [ 1, 1, 2, 7, "a", 7, "one more", 4 ], "t5": null, "t6": [ 1, 2, "a", "b", 3 ], "t7": null, "t8": null, "t10": null, "t11": [ 10, 12.0, "sth" ], "t12": null, "t13": [ 6, 9 ], "t14": null, "t15": null, "t17": [ 6, null, 9, null ], "t18": null, "t20": [ { "$1": {{ 5, "t-mobile", "customization" }} }, { "$1": {{ 5, "verizon", "voice-clarity" }} }, { "$1": {{ 5, "iphone", "platform" }} }, { "$1" [...]
+{ "id": 1, "t1": [ "a", "b", 1, 2, 3 ] }
+{ "id": 2, "t2": [ 1, 2, 3, "a", "b" ] }
+{ "id": 3, "t3": [ 1, 1, 2, "a", "b", 4 ] }
+{ "id": 4, "t4": [ 1, 1, 2, 7, "a", 7, "one more", 4 ] }
+{ "id": 5, "t5": null }
+{ "id": 6, "t6": [ 1, 2, "a", "b", 3 ] }
+{ "id": 7, "t7": null }
+{ "id": 8, "t8": null }
+{ "id": 9 }
+{ "id": 10, "t10": null }
+{ "id": 11, "t11": [ 10, 12.0, "sth" ] }
+{ "id": 12, "t12": null }
+{ "id": 13, "t13": [ 6, 9 ] }
+{ "id": 14, "t14": null }
+{ "id": 15, "t15": null }
+{ "id": 16 }
+{ "id": 17, "t17": [ 6, null, 9, null ] }
+{ "id": 18, "t18": null }
+{ "id": 19 }
+{ "id": 20, "t20": [ { "$1": {{ 5, "t-mobile", "customization" }} }, { "$1": {{ 5, "verizon", "voice-clarity" }} }, { "$1": {{ 5, "iphone", "platform" }} }, { "$1": {{ 5, "samsung", "voice-command" }} }, { "$1": {{ 5, "verizon", "shortcut-menu" }} }, { "$1": {{ 5, "motorola", "speed" }} }, { "$1": {{ 5, "sprint", "voice-command" }} }, { "$1": {{ 5, "motorola", "speed" }} }, { "$1": {{ 5, "iphone", "voice-clarity" }} }, { "$1": {{ 5, "samsung", "platform" }} }, { "$1": {{ 5, "t-mobile", " [...]
+{ "id": 21, "t21": [ {  }, { "$2": [ 5, 3, "John Green", "Emily Jones" ] } ] }
+{ "id": 22, "t22": null }
+{ "id": 23, "t23": null }
+{ "id": 24, "t24": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_prepend/array_prepend.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_prepend/array_prepend.3.adm
index e3b0b0b..bc68e5a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_prepend/array_prepend.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_prepend/array_prepend.3.adm
@@ -1 +1,10 @@
-{ "t1": [ { "$1": {{ "sth", 5, "t-mobile", "customization" }} }, { "$1": {{ "sth", 5, "verizon", "voice-clarity" }} }, { "$1": {{ "sth", 5, "iphone", "platform" }} }, { "$1": {{ "sth", 5, "samsung", "voice-command" }} }, { "$1": {{ "sth", 5, "verizon", "shortcut-menu" }} }, { "$1": {{ "sth", 5, "motorola", "speed" }} }, { "$1": {{ "sth", 5, "sprint", "voice-command" }} }, { "$1": {{ "sth", 5, "motorola", "speed" }} }, { "$1": {{ "sth", 5, "iphone", "voice-clarity" }} }, { "$1": {{ "sth", [...]
+{ "id": 1, "t1": [ { "$1": {{ "sth", 5, "t-mobile", "customization" }} }, { "$1": {{ "sth", 5, "verizon", "voice-clarity" }} }, { "$1": {{ "sth", 5, "iphone", "platform" }} }, { "$1": {{ "sth", 5, "samsung", "voice-command" }} }, { "$1": {{ "sth", 5, "verizon", "shortcut-menu" }} }, { "$1": {{ "sth", 5, "motorola", "speed" }} }, { "$1": {{ "sth", 5, "sprint", "voice-command" }} }, { "$1": {{ "sth", 5, "motorola", "speed" }} }, { "$1": {{ "sth", 5, "iphone", "voice-clarity" }} }, { "$1":  [...]
+{ "id": 2, "t2": [ { "$2": [ [ { "sth": 33 }, { "sth": 44 } ], 3, "John" ] } ] }
+{ "id": 3 }
+{ "id": 4, "t4": null }
+{ "id": 5, "t5": null }
+{ "id": 6 }
+{ "id": 7, "t7": [ 5, 10, 12.0, "sth", 77, "val" ] }
+{ "id": 8, "t8": [ 5, 10, null, "sth", 77, "val" ] }
+{ "id": 9 }
+{ "id": 10, "t10": [ {  }, { "$3": [ "sth1", "sth2", "John Green", "Emily Jones" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.3.adm
index b61b69c..ef6b645 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.3.adm
@@ -1 +1,11 @@
-{ "t1": [ { "$1": {{ "t-mobile", "customization", 5 }} }, { "$1": {{ "verizon", "voice-clarity", "t-mobile", 5 }} }, { "$1": {{ "iphone", "platform", "t-mobile", 5 }} }, { "$1": {{ "samsung", "voice-command", "t-mobile", 5 }} }, { "$1": {{ "verizon", "shortcut-menu", "t-mobile", 5 }} }, { "$1": {{ "motorola", "speed", "t-mobile", 5 }} }, { "$1": {{ "sprint", "voice-command", "t-mobile", 5 }} }, { "$1": {{ "motorola", "speed", "t-mobile", 5 }} }, { "$1": {{ "iphone", "voice-clarity", "t-m [...]
+{ "id": 1, "t1": [ { "$1": {{ "t-mobile", "customization", 5 }} }, { "$1": {{ "verizon", "voice-clarity", "t-mobile", 5 }} }, { "$1": {{ "iphone", "platform", "t-mobile", 5 }} }, { "$1": {{ "samsung", "voice-command", "t-mobile", 5 }} }, { "$1": {{ "verizon", "shortcut-menu", "t-mobile", 5 }} }, { "$1": {{ "motorola", "speed", "t-mobile", 5 }} }, { "$1": {{ "sprint", "voice-command", "t-mobile", 5 }} }, { "$1": {{ "motorola", "speed", "t-mobile", 5 }} }, { "$1": {{ "iphone", "voice-clari [...]
+{ "id": 2 }
+{ "id": 3, "t3": null }
+{ "id": 4 }
+{ "id": 5, "t5": [ 5, 10, 12.0, "sth" ] }
+{ "id": 6 }
+{ "id": 7, "t7": null }
+{ "id": 8, "t8": [ 3, 2, "sth", 9, 9 ] }
+{ "id": 9, "t9": [ 3, 2, "sth", 1, 5 ] }
+{ "id": 10, "t10": null }
+{ "id": 11, "t11": [ {  }, { "$2": [ "John Green", "Emily Jones", "sth1", "sth2" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.4.adm
index fd75f1e..9ea9644 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_put/array_put.4.adm
@@ -1 +1,5 @@
-{ "t1": [ 1, 2, 3, [ 9, 8 ], 5, [ "adding a list", "yes" ] ], "t2": [ [ 5, 1, 2 ], [ 3, 2 ], [ 90, 100 ], [ 80, 100 ] ], "t3": [ { "id": 1, "age": 34 }, { "id": 2, "age": 29 }, { "id": 3, "age": 90 }, { "id": 4, "age": 90 } ], "t4": [ { "$1": [ [ 1, 2, 1 ], [ 9999, 3 ], { "state": "OH", "country": "US" } ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ], [ 9999, 3 ], { "state": "OH", "country": "US" } ] }, { "$1": [ 1, 2, [ 9999, 3 ], { "state": "OH", "country": "US" } ]  [...]
\ No newline at end of file
+{ "id": 1, "t1": [ 1, 2, 3, [ 9, 8 ], 5, [ "adding a list", "yes" ] ] }
+{ "id": 2, "t2": [ [ 5, 1, 2 ], [ 3, 2 ], [ 90, 100 ], [ 80, 100 ] ] }
+{ "id": 3, "t3": [ { "id": 1, "age": 34 }, { "id": 2, "age": 29 }, { "id": 3, "age": 90 }, { "id": 4, "age": 90 } ] }
+{ "id": 4, "t4": [ { "$1": [ [ 1, 2, 1 ], [ 9999, 3 ], { "state": "OH", "country": "US" } ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ], [ 9999, 3 ], { "state": "OH", "country": "US" } ] }, { "$1": [ 1, 2, [ 9999, 3 ], { "state": "OH", "country": "US" } ] }, { "$1": [ { "state": "OH", "country": "US" }, { "state": "CA", "country": "US", "zip_code": 92863 }, [ 9999, 3 ] ] }, { "$1": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US [...]
+{ "id": 5, "t5": [ { "$2": [ { "state": "OH", "country": "US" }, { "state": "CA", "country": "US", "zip_code": 92863 }, [ 9999, 3 ] ] }, { "$2": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US", "zip_code": 92863 }, [ 9999, 3 ], { "state": "OH", "country": "US" } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_range/array_range.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_range/array_range.3.adm
index 362debf..5b10ac9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_range/array_range.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_range/array_range.3.adm
@@ -1 +1,20 @@
-{ "t1": [ 1, 2, 3, 4, 5 ], "t2": [ 0, 5, 10, 15, 20 ], "t3": [ 0.1, 1.1 ], "t4": [ 10, 7, 4 ], "t5": [ -1, -4, -7 ], "t6": [  ], "t7": [  ], "t8": [  ], "t9": [  ], "t10": [ 2 ], "t11": [ 2.0, 3.1, 4.2, 5.300000000000001, 6.4, 7.5, 8.6 ], "t12": null, "t15": null, "t16": null, "t17": null, "t18": null, "t19": null, "t20": null }
+{ "id": 1, "t1": [ 1, 2, 3, 4, 5 ] }
+{ "id": 2, "t2": [ 0, 5, 10, 15, 20 ] }
+{ "id": 3, "t3": [ 0.1, 1.1 ] }
+{ "id": 4, "t4": [ 10, 7, 4 ] }
+{ "id": 5, "t5": [ -1, -4, -7 ] }
+{ "id": 6, "t6": [  ] }
+{ "id": 7, "t7": [  ] }
+{ "id": 8, "t8": [  ] }
+{ "id": 9, "t9": [  ] }
+{ "id": 10, "t10": [ 2 ] }
+{ "id": 11, "t11": [ 2.0, 3.1, 4.2, 5.300000000000001, 6.4, 7.5, 8.6 ] }
+{ "id": 12, "t12": null }
+{ "id": 13 }
+{ "id": 14 }
+{ "id": 15, "t15": null }
+{ "id": 16, "t16": null }
+{ "id": 17, "t17": null }
+{ "id": 18, "t18": null }
+{ "id": 19, "t19": null }
+{ "id": 20, "t20": null }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.3.adm
index c8f9de7..c737763 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.3.adm
@@ -1 +1,10 @@
-{ "t1": [ { "$1": {{ "customization" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "verizon", "shortcut-menu" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "samsung", "platform" }} }, { "$1": {{ "shortcut-menu" }} }, { "$1": {{ "verizon", "voicemail-service" }} }  [...]
+{ "id": 1, "t1": [ { "$1": {{ "customization" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "verizon", "shortcut-menu" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "samsung", "platform" }} }, { "$1": {{ "shortcut-menu" }} }, { "$1": {{ "verizon", "voicemail-servi [...]
+{ "id": 2 }
+{ "id": 3, "t3": null }
+{ "id": 4 }
+{ "id": 5, "t5": [  ] }
+{ "id": 6 }
+{ "id": 7, "t7": null }
+{ "id": 8, "t8": [ 2 ] }
+{ "id": 9, "t9": null }
+{ "id": 10, "t10": [ {  }, { "$2": [ "Emily Jones" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.4.adm
index 7e42853e..85f4d41 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.4.adm
@@ -1 +1,5 @@
-{ "t1": [ 1, 2, 3, [ "str1", "str2" ] ], "t2": [ [ 5, 1, 2 ], [ 90, 100 ] ], "t3": [ { "id": 1, "age": 34 }, { "id": 3, "age": 90 } ], "t4": [ { "$1": [ [ 1, 2, 1 ] ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ] ] }, { "$1": [ 1, 2 ] }, { "$1": [ { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$1": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US", "zip_code": 92863 } ] }, { "$1": null }, {  } ], "t5": [ { "$2": [ {  [...]
\ No newline at end of file
+{ "id": 1, "t1": [ 1, 2, 3, [ "str1", "str2" ] ] }
+{ "id": 2, "t2": [ [ 5, 1, 2 ], [ 90, 100 ] ] }
+{ "id": 3, "t3": [ { "id": 1, "age": 34 }, { "id": 3, "age": 90 } ] }
+{ "id": 4, "t4": [ { "$1": [ [ 1, 2, 1 ] ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ] ] }, { "$1": [ 1, 2 ] }, { "$1": [ { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$1": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US", "zip_code": 92863 } ] }, { "$1": null }, {  } ] }
+{ "id": 5, "t5": [ { "$2": [ { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$2": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US", "zip_code": 92863 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.adm
new file mode 100644
index 0000000..3c38795
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_remove/array_remove.5.adm
@@ -0,0 +1,8 @@
+distribute result [$$d]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    unnest $$d <- scan-collection(ordered-list-constructor({"id": 1, "t1": array-remove(ordered-list-constructor(1, 2, 3, ordered-list-constructor(9, 8), ordered-list-constructor("str1", "str2"), ordered-list-constructor(90, 100)), array: [ 9, 8 ], array: [ 90, 100 ])}, {"id": 2, "t2": cast(array: [ array: [ 5, 1, 2 ], array: [ 90, 100 ] ])}, {"id": 3, "t3": cast(array-remove(ordered-list-constructor({"id": 1, "age": 34}, {"id": 2, "age": 29}, {"id": 3, "age": 90}), {"id": 4, "age": 90}, [...]
+    -- UNNEST  |UNPARTITIONED|
+      empty-tuple-source
+      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.3.adm
index 3fc58af..a364639 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.3.adm
@@ -1 +1,25 @@
-{ "t1": [ { "$1": {{ "coffee-mobile", "customization" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "verizon", "shortcut-menu" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "samsung", "platform" }} }, { "$1": {{ "coffee-mobile", "shortcut-menu" }} }, { "$1": {{ "v [...]
+{ "id": 1, "t1": [ { "$1": {{ "coffee-mobile", "customization" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "verizon", "shortcut-menu" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "samsung", "platform" }} }, { "$1": {{ "coffee-mobile", "shortcut-menu" }} }, { "$ [...]
+{ "id": 2, "t2": [ 3, -2, 1, 5, -2, 9, -2, 3, -2, 1, -2 ] }
+{ "id": 3, "t3": [ 3, -2, 1, 5, -4, 9, -4, 3, -4, 1, -4 ] }
+{ "id": 4, "t4": [ 3, -2, 1, 5, "2", 9, -4, 3, -4, 1, -4 ] }
+{ "id": 5, "t5": [ 3, -2, -2, 5, "2", 9, -2, 3, "A", 1, 2 ] }
+{ "id": 6, "t6": [ 3, -2, "a", 5, "2", 9, [ 2, 3, "A" ], 1, -2 ] }
+{ "id": 7, "t7": [ 3, -2, "a", 5, "2", 9, [ 2, 3, "A" ], 1, [ -2 ] ] }
+{ "id": 8, "t8": [ 3, -5, -5, 2, 2, -5, 2, 1, 1 ] }
+{ "id": 9, "t9": [ 3, 1, 1, 2, 2, 1, 2, 1, 1 ] }
+{ "id": 10, "t10": [ 3, -5, -5, 2, 2, -5, 2, -5, 1 ] }
+{ "id": 11, "t11": [ 3, -5, -5, 2, 2, -5, 2, -5, -5 ] }
+{ "id": 12, "t12": [ 3, -5, -5, 2, 2, -5, 2, -5, -5 ] }
+{ "id": 13, "t13": [ 3, -5, -5, 2, 2, -5, 2, -5, -5 ] }
+{ "id": 14, "t14": [ 3, -5, 1, 2, 2, 1, 2, 1, 1 ] }
+{ "id": 15, "t15": null }
+{ "id": 16, "t16": null }
+{ "id": 17, "t17": null }
+{ "id": 18, "t18": [  ] }
+{ "id": 19, "t19": [ 3, 2 ] }
+{ "id": 20 }
+{ "id": 21, "t21": [ null, 2, null ] }
+{ "id": 22, "t22": null }
+{ "id": 23, "t23": null }
+{ "id": 24 }
+{ "id": 25, "t25": [ {  }, { "$2": [ "John Green", "sth" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.4.adm
index 9932af1..666e7e6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_replace/array_replace.4.adm
@@ -1 +1,5 @@
-{ "t1": [ [ 5, 1 ], [ 0, 0 ], [ 9, 7 ], [ 0, 0 ], [ 10, 11 ] ], "t2": [ { "id": 1, "age": 34 }, { "id": 8, "age": 200 }, { "id": 3, "age": 90 }, { "id": 4, "age": 10 } ], "t3": [ { "$1": [ [ 1, 2, 1 ], "replace done" ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ] ] }, { "$1": [ 1, 2 ] }, { "$1": [ { "state": "OH", "country": "US" }, { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$1": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL" [...]
\ No newline at end of file
+{ "id": 1, "t1": [ [ 5, 1 ], [ 0, 0 ], [ 9, 7 ], [ 0, 0 ], [ 10, 11 ] ] }
+{ "id": 2, "t2": [ { "id": 1, "age": 34 }, { "id": 8, "age": 200 }, { "id": 3, "age": 90 }, { "id": 4, "age": 10 } ] }
+{ "id": 3, "t3": [ { "$1": [ [ 1, 2, 1 ], "replace done" ] }, { "$1": [ [ "white", "blue", "magenta" ], [ "red", "black" ] ] }, { "$1": [ 1, 2 ] }, { "$1": [ { "state": "OH", "country": "US" }, { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$1": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US", "zip_code": 92863 } ] }, { "$1": null }, {  } ] }
+{ "id": 4, "t4": [ { "$2": [ [ 1, 2, 1 ], [ 9999, 3 ] ] }, { "$2": [ [ "white", "blue", "magenta" ], [ "red", "black" ] ] }, { "$2": [ 1, 2 ] }, { "$2": [ "replace done", { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$2": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US", "zip_code": 92863 } ] }, { "$2": null }, {  } ] }
+{ "id": 5, "t5": [ { "$3": [ "replace done", { "state": "CA", "country": "US", "zip_code": 92863 } ] }, { "$3": [ { "state": "OR", "country": "US", "zip_code": null }, { "state": "IL", "country": "US", "zip_code": 92863 } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.3.adm
index 330b8d1..e201f6b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.3.adm
@@ -1 +1,11 @@
-{ "t1": [ { "$1": {{ "customization", "t-mobile" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "shortcut-menu", "verizon" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "platform", "samsung" }} }, { "$1": {{ "shortcut-menu", "t-mobile" }} }, { "$1": {{ "verizon", " [...]
+{ "id": 1, "t1": [ { "$1": {{ "customization", "t-mobile" }} }, { "$1": {{ "verizon", "voice-clarity" }} }, { "$1": {{ "iphone", "platform" }} }, { "$1": {{ "samsung", "voice-command" }} }, { "$1": {{ "shortcut-menu", "verizon" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "sprint", "voice-command" }} }, { "$1": {{ "motorola", "speed" }} }, { "$1": {{ "iphone", "voice-clarity" }} }, { "$1": {{ "platform", "samsung" }} }, { "$1": {{ "shortcut-menu", "t-mobile" }} }, { "$1": {{ "ve [...]
+{ "id": 2, "t2": [ 2, 5, 5, 7, 7, 19 ] }
+{ "id": 3, "t3": [ 2, 5.0, 5, 5.1, 7, 7, 19 ] }
+{ "id": 4, "t4": [ 5, 7.5, 19, "A", "John", "a", "a" ] }
+{ "id": 5, "t5": [ null, null, null, 5, 7, 19 ] }
+{ "id": 6, "t6": [ 3 ] }
+{ "id": 7, "t7": null }
+{ "id": 8, "t8": [  ] }
+{ "id": 9 }
+{ "id": 10, "t10": null }
+{ "id": 11, "t11": [ {  }, { "$2": [ "Abby", "Emily Jones", "John Green" ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.4.adm
index 2303896..5d52990 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.4.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/array_fun/array_sort/array_sort.4.adm
@@ -1 +1,4 @@
-{ "t1": [ [ 0, 0 ], [ 5, 1 ], [ 6, 2 ], [ 6, 2 ], [ 6, 2 ], [ 9, 7 ], [ 10, 11 ] ], "t2": [ { "id": 4, "age": 10 }, { "id": 2, "age": 29 }, { "id": 1, "age": 34 }, { "id": 3, "age": 90 }, { "id": 5, "age": 90 } ], "t3": [ { "$1": [ [ 1, 2 ], [ 1, 2, 1 ], [ 4, 2 ], [ 5, 1 ], [ 5, 1 ], [ 9999, 3 ] ] }, { "$1": [ [ "blue", "orange" ], [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$1": [ { "state": "CA", "country": "US", "zip_code": 12863 }, { "state": "CA", "country": "US", "zi [...]
\ No newline at end of file
+{ "id": 1, "t1": [ [ 0, 0 ], [ 5, 1 ], [ 6, 2 ], [ 6, 2 ], [ 6, 2 ], [ 9, 7 ], [ 10, 11 ] ] }
+{ "id": 2, "t2": [ { "id": 4, "age": 10 }, { "id": 2, "age": 29 }, { "id": 1, "age": 34 }, { "id": 3, "age": 90 }, { "id": 5, "age": 90 } ] }
+{ "id": 3, "t3": [ { "$1": [ [ 1, 2 ], [ 1, 2, 1 ], [ 4, 2 ], [ 5, 1 ], [ 5, 1 ], [ 9999, 3 ] ] }, { "$1": [ [ "blue", "orange" ], [ "red", "black" ], [ "white", "blue", "magenta" ] ] }, { "$1": [ { "state": "CA", "country": "US", "zip_code": 12863 }, { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] }, { "$1": null }, {  } ] }
+{ "id": 4, "t4": [ { "$2": [ { "state": "CA", "country": "US", "zip_code": 12863 }, { "state": "CA", "country": "US", "zip_code": 92863 }, { "state": "OH", "country": "US" } ] } ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm
index 87269dc..c185d5a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true ], "t2": "1", "t3": { "b": "1" }, "t4": [ 1, 2 ], "t5": [ { "$1": "en" } ] }
\ No newline at end of file
+{ "t1": [ true, true, true, true, true ], "t2": "1", "t3": { "b": "1" }, "t4": [ 1, 2 ] }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.4.adm
new file mode 100644
index 0000000..868480b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_unwrap/object_unwrap.4.adm
@@ -0,0 +1 @@
+{ "$1": "en" }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_number_01/to_number_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_number_01/to_number_01.1.adm
index 72416e4..7dc5028 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_number_01/to_number_01.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/types/to_number_01/to_number_01.1.adm
@@ -1 +1 @@
-{ "t1": 0, "t2": 1, "t3": 8, "t4": 16, "t5": 32, "t6": 64, "t7": 1.5, "t8": 2.25, "t9": 512, "t10": true, "t11": true, "t12": true, "t13": true, "t14": true, "t15": true, "t16": "INF", "t17": "-INF", "t18": "NaN", "t19": "INF", "t20": "-INF", "t21": "NaN" }
\ No newline at end of file
+{ "t1": 0, "t2": 1, "t3": 8, "t4": 16, "t5": 32, "t6": 64, "t7": 1.5, "t8": 2.25, "t10": true, "t11": true, "t12": true, "t13": true, "t14": true, "t15": true, "t16": "INF", "t17": "-INF", "t18": "NaN", "t19": "INF", "t20": "-INF", "t21": "NaN", "t9": 512 }
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
index e50ce1b..83af00d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveFieldsTypeComputer.java
@@ -66,25 +66,28 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
     private RecordRemoveFieldsTypeComputer() {
     }
 
-    private void getPathFromConstantExpression(FunctionIdentifier funcId, ILogicalExpression expression,
+    private static void getPathFromConstantExpression(FunctionIdentifier funcId, ILogicalExpression fieldNameExpression,
             Set<String> fieldNameSet, List<List<String>> pathList, SourceLocation sourceLoc)
             throws AlgebricksException {
-        ConstantExpression ce = (ConstantExpression) expression;
+        ConstantExpression ce = (ConstantExpression) fieldNameExpression;
         if (!(ce.getValue() instanceof AsterixConstantValue)) {
             throw new InvalidExpressionException(sourceLoc, funcId, 1, ce, LogicalExpressionTag.CONSTANT);
         }
-        IAObject item = ((AsterixConstantValue) ce.getValue()).getObject();
-        ATypeTag type = item.getType().getTypeTag();
+        IAObject fieldName = ((AsterixConstantValue) ce.getValue()).getObject();
+        getPathOfFieldName(funcId, fieldName, fieldNameSet, pathList, sourceLoc);
+    }
 
+    private static void getPathOfFieldName(FunctionIdentifier funcId, IAObject fieldName, Set<String> fieldNameSet,
+            List<List<String>> pathList, SourceLocation sourceLoc) throws UnsupportedTypeException {
+        ATypeTag type = fieldName.getType().getTypeTag();
         switch (type) {
             case STRING:
-                String fn = ((AString) item).getStringValue();
+                String fn = ((AString) fieldName).getStringValue();
                 fieldNameSet.add(fn);
                 break;
             case ARRAY:
-                AOrderedList pathOrdereList = (AOrderedList) item;
-                String fieldName = ((AString) pathOrdereList.getItem(0)).getStringValue();
-                fieldNameSet.add(fieldName);
+                AOrderedList pathOrdereList = (AOrderedList) fieldName;
+                fieldNameSet.add(((AString) pathOrdereList.getItem(0)).getStringValue());
                 List<String> path = new ArrayList<>();
                 for (int i = 0; i < pathOrdereList.size(); i++) {
                     path.add(((AString) pathOrdereList.getItem(i)).getStringValue());
@@ -96,7 +99,7 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
         }
     }
 
-    private List<String> getListFromExpression(FunctionIdentifier funcId, ILogicalExpression expression,
+    private static List<String> getListFromExpression(FunctionIdentifier funcId, ILogicalExpression expression,
             SourceLocation sourceLoc) throws AlgebricksException {
         AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) expression;
         List<Mutable<ILogicalExpression>> args = funcExp.getArguments();
@@ -121,7 +124,7 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
         return list;
     }
 
-    private void getPathFromFunctionExpression(FunctionIdentifier funcId, ILogicalExpression expression,
+    private static void getPathFromFunctionExpression(FunctionIdentifier funcId, ILogicalExpression expression,
             Set<String> fieldNameSet, List<List<String>> pathList, SourceLocation sourceLoc)
             throws AlgebricksException {
         List<String> path = getListFromExpression(funcId, expression, sourceLoc);
@@ -131,23 +134,32 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
 
     }
 
-    private void computeTypeFromNonConstantExpression(FunctionIdentifier funcId, ILogicalExpression expression,
+    private static void computeTypeFromExpression(FunctionIdentifier funcId, ILogicalExpression e,
             Set<String> fieldNameSet, List<List<String>> pathList) throws AlgebricksException {
-        AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) expression;
-        List<Mutable<ILogicalExpression>> args = funcExp.getArguments();
-
-        for (Mutable<ILogicalExpression> arg : args) {
-            ILogicalExpression le = arg.getValue();
-            switch (le.getExpressionTag()) {
-                case CONSTANT:
-                    getPathFromConstantExpression(funcId, le, fieldNameSet, pathList, funcExp.getSourceLocation());
-                    break;
-                case FUNCTION_CALL:
-                    getPathFromFunctionExpression(funcId, le, fieldNameSet, pathList, funcExp.getSourceLocation());
-                    break;
-                default:
-                    throw new InvalidExpressionException(funcExp.getSourceLocation(), funcId, 1, le,
-                            LogicalExpressionTag.CONSTANT, LogicalExpressionTag.FUNCTION_CALL);
+        // e is the field to remove, ["foo", "bar"] = foo.bar, either an ordered list function call or a constant array
+        SourceLocation sourceLocation = e.getSourceLocation();
+        if (e.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
+            AOrderedList list = (AOrderedList) ((AsterixConstantValue) ((ConstantExpression) e).getValue()).getObject();
+            for (int i = 0, size = list.size(); i < size; i++) {
+                getPathOfFieldName(funcId, list.getItem(i), fieldNameSet, pathList, sourceLocation);
+            }
+        } else {
+            AbstractFunctionCallExpression funcExp = (AbstractFunctionCallExpression) e;
+            List<Mutable<ILogicalExpression>> args = funcExp.getArguments();
+
+            for (Mutable<ILogicalExpression> arg : args) {
+                ILogicalExpression le = arg.getValue();
+                switch (le.getExpressionTag()) {
+                    case CONSTANT:
+                        getPathFromConstantExpression(funcId, le, fieldNameSet, pathList, sourceLocation);
+                        break;
+                    case FUNCTION_CALL:
+                        getPathFromFunctionExpression(funcId, le, fieldNameSet, pathList, sourceLocation);
+                        break;
+                    default:
+                        throw new InvalidExpressionException(sourceLocation, funcId, 1, le,
+                                LogicalExpressionTag.CONSTANT, LogicalExpressionTag.FUNCTION_CALL);
+                }
             }
         }
     }
@@ -177,20 +189,21 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
         }
 
         ATypeTag tt = inputOrderedListType.getItemType().getTypeTag();
-        if (tt == ATypeTag.STRING) { // If top-fieldlist
+        if (tt == ATypeTag.STRING) {
+            // all fields to be removed are top-level fields, e.g. remove_fields(obj, ["f1", "f2", "f3"])
             if (setFieldNameSet(arg1, fieldNameSet)) {
                 return buildOutputType(fieldPathStack, inputRecordType, fieldNameSet, pathList);
             } else {
                 return DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
             }
-        } else { // tt == ATypeTag.ANY, meaning the list is nested
-            computeTypeFromNonConstantExpression(funcId, arg1, fieldNameSet, pathList);
-            IAType resultType = buildOutputType(fieldPathStack, inputRecordType, fieldNameSet, pathList);
-            return resultType;
+        } else {
+            // some fields to be removed are nested, e.g. remove_fields(obj, ["f1", ["f2", "nested_field"], "f3"])
+            computeTypeFromExpression(funcId, arg1, fieldNameSet, pathList);
+            return buildOutputType(fieldPathStack, inputRecordType, fieldNameSet, pathList);
         }
     }
 
-    private boolean setFieldNameSet(ILogicalExpression expr, Set<String> fieldNameSet) {
+    private static boolean setFieldNameSet(ILogicalExpression expr, Set<String> fieldNameSet) {
         if (expr.getExpressionTag() == LogicalExpressionTag.CONSTANT) {
             AOrderedList orderedList =
                     (AOrderedList) (((AsterixConstantValue) ((ConstantExpression) expr).getValue()).getObject());
@@ -203,8 +216,8 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
         return false;
     }
 
-    private void addField(ARecordType inputRecordType, String fieldName, List<String> resultFieldNames,
-            List<IAType> resultFieldTypes) throws AlgebricksException {
+    private static void addField(ARecordType inputRecordType, String fieldName, List<String> resultFieldNames,
+            List<IAType> resultFieldTypes) {
         resultFieldNames.add(fieldName);
         if (inputRecordType.getFieldType(fieldName).getTypeTag() == ATypeTag.OBJECT) {
             ARecordType nestedType = (ARecordType) inputRecordType.getFieldType(fieldName);
@@ -215,8 +228,8 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
         }
     }
 
-    private IAType buildOutputType(Deque<String> fieldPathStack, ARecordType inputRecordType, Set<String> fieldNameSet,
-            List<List<String>> pathList) throws AlgebricksException {
+    private static IAType buildOutputType(Deque<String> fieldPathStack, ARecordType inputRecordType,
+            Set<String> fieldNameSet, List<List<String>> pathList) {
         List<String> resultFieldNames = new ArrayList<>();
         List<IAType> resultFieldTypes = new ArrayList<>();
 
@@ -250,7 +263,7 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
      * Comparison elements of two paths
      * Note: l2 uses a LIFO insert and removal.
      */
-    private <E> boolean isEqualPaths(List<E> l1, Deque<E> l2) {
+    private static <E> boolean isEqualPaths(List<E> l1, Deque<E> l2) {
         if ((l1 == null) || (l2 == null)) {
             return false;
         }
@@ -272,7 +285,7 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
         return true;
     }
 
-    private boolean isRemovePath(Deque<String> fieldPath, List<List<String>> pathList) {
+    private static boolean isRemovePath(Deque<String> fieldPath, List<List<String>> pathList) {
         for (List<String> removePath : pathList) {
             if (isEqualPaths(removePath, fieldPath)) {
                 return true;
@@ -285,8 +298,8 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
         A method to deep copy a record the path validation
              i.e., keep only fields that are valid
      */
-    private ARecordType deepCheckAndCopy(Deque<String> fieldPath, ARecordType srcRecType, List<List<String>> pathList,
-            boolean isOpen) throws AlgebricksException {
+    private static ARecordType deepCheckAndCopy(Deque<String> fieldPath, ARecordType srcRecType,
+            List<List<String>> pathList, boolean isOpen) {
         // Make sure the current path is valid before going further
         if (isRemovePath(fieldPath, pathList)) {
             return null;
@@ -343,5 +356,4 @@ public class RecordRemoveFieldsTypeComputer implements IResultTypeComputer {
                 throw new TypeMismatchException(sourceLoc, funcId, 0, type0.getTypeTag(), ATypeTag.OBJECT);
         }
     }
-
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
index 9e3ceaf..9796a1b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/FieldAccessByIndexEvalFactory.java
@@ -45,44 +45,37 @@ import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 public class FieldAccessByIndexEvalFactory implements IScalarEvaluatorFactory {
 
     private static final long serialVersionUID = 1L;
-
-    private IScalarEvaluatorFactory recordEvalFactory;
-    private IScalarEvaluatorFactory fieldIndexEvalFactory;
-    private int nullBitmapSize;
-    private ARecordType recordType;
+    private final IScalarEvaluatorFactory recordEvalFactory;
+    private final IScalarEvaluatorFactory fieldIndexEvalFactory;
+    private final int nullBitmapSize;
+    private final ARecordType recordType;
     private final SourceLocation sourceLoc;
 
-    public FieldAccessByIndexEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
+    FieldAccessByIndexEvalFactory(IScalarEvaluatorFactory recordEvalFactory,
             IScalarEvaluatorFactory fieldIndexEvalFactory, ARecordType recordType, SourceLocation sourceLoc) {
         this.recordEvalFactory = recordEvalFactory;
         this.fieldIndexEvalFactory = fieldIndexEvalFactory;
         this.recordType = recordType;
-        this.nullBitmapSize = RecordUtil.computeNullBitmapSize(recordType);
+        this.nullBitmapSize = recordType != null ? RecordUtil.computeNullBitmapSize(recordType) : 0;
         this.sourceLoc = sourceLoc;
     }
 
     @Override
     public IScalarEvaluator createScalarEvaluator(final IEvaluatorContext ctx) throws HyracksDataException {
         return new IScalarEvaluator() {
-            private ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-            private DataOutput out = resultStorage.getDataOutput();
-
-            private IPointable inputArg0 = new VoidPointable();
-            private IPointable inputArg1 = new VoidPointable();
-            private IScalarEvaluator eval0 = recordEvalFactory.createScalarEvaluator(ctx);
-            private IScalarEvaluator eval1 = fieldIndexEvalFactory.createScalarEvaluator(ctx);
+            private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+            private final DataOutput out = resultStorage.getDataOutput();
+            private final IPointable inputArg0 = new VoidPointable();
+            private final IPointable inputArg1 = new VoidPointable();
+            private final IScalarEvaluator eval0 = recordEvalFactory.createScalarEvaluator(ctx);
+            private final IScalarEvaluator eval1 = fieldIndexEvalFactory.createScalarEvaluator(ctx);
             private int fieldIndex;
             private int fieldValueOffset;
             private int fieldValueLength;
             private IAType fieldValueType;
             private ATypeTag fieldValueTypeTag;
 
-            /*
-             * inputArg0: the record
-             * inputArg1: the index
-             *
-             * This method outputs into IHyracksTaskContext context [field type tag (1 byte)][the field data]
-             */
+            // inputArg0: the record, inputArg1: the index
             @Override
             public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
                 try {
@@ -99,7 +92,8 @@ public class FieldAccessByIndexEvalFactory implements IScalarEvaluatorFactory {
                     byte[] indexBytes = inputArg1.getByteArray();
                     int indexOffset = inputArg1.getStartOffset();
 
-                    if (serRecord[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
+                    if (serRecord[offset] != ATypeTag.SERIALIZED_RECORD_TYPE_TAG || recordType == null) {
+                        // recordType = null should only mean first arg was not a record and compiler couldn't set it
                         throw new TypeMismatchException(sourceLoc, serRecord[offset],
                                 ATypeTag.SERIALIZED_RECORD_TYPE_TAG);
                     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index be4a1f0..5dc56dd 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -159,12 +159,9 @@ public final class FunctionTypeInferers {
                         IAType t2 = unionT.getActualType();
                         if (t2.getTypeTag() == ATypeTag.OBJECT) {
                             fd.setImmutableStates(t2);
-                            break;
                         }
                     }
-                    throw new NotImplementedException("field-access-by-index for data of type " + t);
-                default:
-                    throw new NotImplementedException("field-access-by-index for data of type " + t);
+                    break;
             }
         }
     }
@@ -191,7 +188,8 @@ public final class FunctionTypeInferers {
                     fd.setImmutableStates(RecordUtil.FULLY_OPEN_RECORD_TYPE, listFieldPath);
                     break;
                 default:
-                    throw new NotImplementedException("field-access-nested for data of type " + t);
+                    fd.setImmutableStates(null, listFieldPath);
+                    break;
             }
         }
     }