You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by wy...@apache.org on 2022/11/11 20:49:49 UTC

[asterixdb] branch master updated: [COMP][ASTERIXDB-3041] Index usage w/ OBJECT_* functions.

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

wyk 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 d6b5cb85cc [COMP][ASTERIXDB-3041] Index usage w/ OBJECT_* functions.
d6b5cb85cc is described below

commit d6b5cb85cc4122d37734159861fafeda77656319
Author: ggalvizo <gg...@uci.edu>
AuthorDate: Tue Oct 25 08:45:08 2022 -0700

    [COMP][ASTERIXDB-3041] Index usage w/ OBJECT_* functions.
    
    - user model changes: no
    - storage format changes: no
    - interface changes: no
    
    details:
    - adding specialized type computers for OBJECT_ADD,
    OBJECT_REMOVE, OBJECT_RENAME, and OBJECT_PUT.
    - updating runtimes of aforementioned functions to
    reuse its input type when possible.
    - updating access method introducing rule to treat
    function applications of aforementioned functions
    (and OBJECT_ADD_FIELDS, OBJECT_REMOVE_FIELDS) as
    its input record variables.
    
    Change-Id: I2c64319981f504736cea8732b3ec985d5c913dd3
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/16843
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Wail Alkowaileet <wa...@gmail.com>
    Tested-by: Wail Alkowaileet <wa...@gmail.com>
---
 .../optimizer/rules/am/AccessMethodUtils.java      | 153 +++++++++++++++++++--
 .../index-through-object.1.sqlpp                   |  36 +++++
 .../index-through-object.10.sqlpp                  |  40 ++++++
 .../index-through-object.11.sqlpp                  |  44 ++++++
 .../index-through-object.2.sqlpp                   |  37 +++++
 .../index-through-object.3.sqlpp                   |  36 +++++
 .../index-through-object.4.sqlpp                   |  36 +++++
 .../index-through-object.5.sqlpp                   |  36 +++++
 .../index-through-object.6.sqlpp                   |  38 +++++
 .../index-through-object.7.sqlpp                   |  36 +++++
 .../index-through-object.8.sqlpp                   |  36 +++++
 .../index-through-object.9.sqlpp                   |  41 ++++++
 .../index-through-object.1.plan                    |  18 +++
 .../index-through-object.10.plan                   |  20 +++
 .../index-through-object.11.plan                   |  39 ++++++
 .../index-through-object.2.plan                    |  18 +++
 .../index-through-object.3.plan                    |  18 +++
 .../index-through-object.4.plan                    |  18 +++
 .../index-through-object.5.plan                    |  11 ++
 .../index-through-object.6.plan                    |  20 +++
 .../index-through-object.7.plan                    |  11 ++
 .../index-through-object.8.plan                    |  18 +++
 .../index-through-object.9.plan                    |  47 +++++++
 .../objects/object_add/object_add.5.ddl.sqlpp      |  35 +++++
 .../objects/object_add/object_add.6.update.sqlpp   |  31 +++++
 .../objects/object_add/object_add.7.query.sqlpp    |  52 +++++++
 .../objects/object_put/object_put.5.ddl.sqlpp      |  35 +++++
 .../objects/object_put/object_put.6.update.sqlpp   |  31 +++++
 .../objects/object_put/object_put.7.query.sqlpp    |  60 ++++++++
 .../results/objects/object_add/object_add.3.adm    |   2 +-
 .../results/objects/object_add/object_add.7.adm    |   1 +
 .../results/objects/object_put/object_put.3.adm    |   2 +-
 .../results/objects/object_put/object_put.7.adm    |   1 +
 .../asterix/om/functions/BuiltinFunctions.java     |  12 +-
 .../impl/AbstractRecordFunctionTypeComputer.java   | 103 ++++++++++++++
 .../typecomputer/impl/RecordAddTypeComputer.java   |  84 +++++++++++
 .../typecomputer/impl/RecordPutTypeComputer.java   | 111 +++++++++++++++
 .../impl/RecordRemoveTypeComputer.java             |  74 ++++++++++
 .../impl/RecordRenameTypeComputer.java             | 104 ++++++++++++++
 .../records/AbstractRecordAddPutEvaluator.java     |  85 ++++--------
 .../records/AbstractRecordFunctionEvaluator.java   |  64 +++++++++
 .../functions/records/RecordAddDescriptor.java     |  16 +--
 .../functions/records/RecordAddEvaluator.java      |  29 ++--
 .../functions/records/RecordPutDescriptor.java     |  16 +--
 .../functions/records/RecordPutEvaluator.java      |  35 ++---
 .../functions/records/RecordRemoveDescriptor.java  |  13 +-
 .../functions/records/RecordRemoveEvaluator.java   | 127 +++++------------
 .../functions/records/RecordRenameDescriptor.java  |  13 +-
 .../functions/records/RecordRenameEvaluator.java   | 133 ++++++------------
 .../runtime/functions/FunctionTypeInferers.java    |  10 ++
 50 files changed, 1765 insertions(+), 321 deletions(-)

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 0196d912b3..bbbf3df8be 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -26,11 +26,15 @@ import static org.apache.asterix.om.functions.BuiltinFunctions.FIELD_ACCESS_NEST
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 
 import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
 import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -84,6 +88,7 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCa
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IAlgebricksConstantValue;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IExpressionTypeComputer;
 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.expressions.UnnestingFunctionCallExpression;
@@ -92,6 +97,7 @@ import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFun
 import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
 import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator.ExecutionMode;
@@ -146,6 +152,19 @@ public class AccessMethodUtils {
             BuiltinFunctions.YEAR_MONTH_DURATION_DEFAULT_NULL_CONSTRUCTOR,
             BuiltinFunctions.UUID_DEFAULT_NULL_CONSTRUCTOR, BuiltinFunctions.BINARY_BASE64_DEFAULT_NULL_CONSTRUCTOR);
 
+    // TODO (GLENN): We can definitely expand the whitelist here...
+    private final static Map<FunctionIdentifier, Set<Integer>> INDEX_USE_ON_FUNCTION_CALL_WHITELIST;
+    private final static Set<Integer> ALL_INDEX_FUNCTION_ARGUMENTS = Collections.emptySet();
+    static {
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST = new HashMap<>();
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_ADD, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.ADD_FIELDS, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_REMOVE, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_RENAME, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.REMOVE_FIELDS, Set.of(0));
+        INDEX_USE_ON_FUNCTION_CALL_WHITELIST.put(BuiltinFunctions.RECORD_CONCAT, ALL_INDEX_FUNCTION_ARGUMENTS);
+    }
+
     private final static Pair<List<String>, Integer> NO_FIELD_NAME = new Pair<>(Collections.emptyList(), 0);
 
     public static void appendPrimaryIndexTypes(Dataset dataset, IAType itemType, IAType metaItemType,
@@ -2982,8 +3001,26 @@ public class AccessMethodUtils {
             isByName = true;
         }
         if (isFieldAccess) {
-            LogicalVariable sourceVar =
-                    ((VariableReferenceExpression) funcExpr.getArguments().get(0).getValue()).getVariableReference();
+            ILogicalExpression funcExprArg0 = funcExpr.getArguments().get(0).getValue();
+            MutableInt sourceIndicator = new MutableInt(0);
+            LogicalVariable sourceVar;
+            if (funcExprArg0.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                // This might be a field-access on an indexable-function-call (or nested indexable-function-calls).
+                List<LogicalVariable> foundDatasourceVariables = new ArrayList<>();
+                if (canUseIndexOnFunction((AbstractFunctionCallExpression) funcExprArg0, sourceIndicator,
+                        foundDatasourceVariables, optFuncExpr, op.computeInputTypeEnvironment(context), context)) {
+                    // TODO (GLENN): In the case of OBJECT_CONCAT w/ potentially multiple datasource variables, we
+                    //               will not explore each variable. This method definitely needs refactoring in the
+                    //               future to handle such a case.
+                    sourceVar = foundDatasourceVariables.get(0);
+                } else {
+                    return NO_FIELD_NAME;
+                }
+            } else if (funcExprArg0.getExpressionTag() != LogicalExpressionTag.VARIABLE) {
+                return NO_FIELD_NAME;
+            } else {
+                sourceVar = ((VariableReferenceExpression) funcExprArg0).getVariableReference();
+            }
             if (optFuncExpr != null) {
                 optFuncExpr.setLogicalExpr(funcVarIndex, parentFuncExpr);
                 optFuncExpr.addStepExpr(funcVarIndex, funcExpr);
@@ -3021,12 +3058,27 @@ public class AccessMethodUtils {
             if (assignAndExpressionIndexes != null && assignAndExpressionIndexes[0] > -1) {
                 //We found the nested assign
 
-                //Recursive call on nested assign
+                // Is the next operator composed of functions that are not a field access? If so, do not recurse.
+                ILogicalOperator nextOp = subTree.getAssignsAndUnnests().get(assignAndExpressionIndexes[0]);
+                boolean isIndexOnFunction = false;
+                if (nextOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                    AssignOperator nextAssignOp = (AssignOperator) nextOp;
+                    ILogicalExpression leadingArgumentExpr = nextAssignOp.getExpressions().get(0).getValue();
+                    if (leadingArgumentExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                        IVariableTypeEnvironment typeEnv = nextAssignOp.computeInputTypeEnvironment(context);
+                        isIndexOnFunction = canUseIndexOnFunction((AbstractFunctionCallExpression) leadingArgumentExpr,
+                                sourceIndicator, new HashSet<>(), optFuncExpr, typeEnv, context);
+                    }
+                }
+
+                // Otherwise... recurse.
                 Pair<List<String>, Integer> parentFieldNames =
-                        getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
-                                assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context);
+                        !isIndexOnFunction
+                                ? getFieldNameAndStepsFromSubTree(optFuncExpr, subTree, assignAndExpressionIndexes[0],
+                                        assignAndExpressionIndexes[1], funcVarIndex, parentFuncExpr, context)
+                                : NO_FIELD_NAME;
 
-                if (parentFieldNames.first.isEmpty()) {
+                if (parentFieldNames.first.isEmpty() && !isIndexOnFunction) {
                     //Nested assign was not a field access.
                     //We will not use index
                     return NO_FIELD_NAME;
@@ -3048,13 +3100,23 @@ public class AccessMethodUtils {
                 if (optFuncExpr != null) {
                     optFuncExpr.setSourceVar(funcVarIndex, ((AssignOperator) op).getVariables().get(assignVarIndex));
                 }
-                //add fieldName to the nested fieldName, return
-                if (nestedAccessFieldName != null) {
-                    parentFieldNames.first.addAll(nestedAccessFieldName);
+
+                if (!isIndexOnFunction) {
+                    //add fieldName to the nested fieldName, return
+                    if (nestedAccessFieldName != null) {
+                        parentFieldNames.first.addAll(nestedAccessFieldName);
+                    } else {
+                        parentFieldNames.first.add(fieldName);
+                    }
+                    return (parentFieldNames);
+
                 } else {
-                    parentFieldNames.first.add(fieldName);
+                    if (nestedAccessFieldName != null) {
+                        return new Pair<>(nestedAccessFieldName, sourceIndicator.getValue());
+                    } else {
+                        return new Pair<>(new ArrayList<>(List.of(fieldName)), sourceIndicator.getValue());
+                    }
                 }
-                return (parentFieldNames);
             }
 
             if (optFuncExpr != null) {
@@ -3191,4 +3253,73 @@ public class AccessMethodUtils {
         return funId.equals(FIELD_ACCESS_BY_NAME) || funId.equals(FIELD_ACCESS_BY_INDEX)
                 || funId.equals(FIELD_ACCESS_NESTED);
     }
+
+    /**
+     * If we are accessing some field through a function application (or series of function applications) of the
+     * following:
+     * <p><pre>
+     * | OBJECT_ADD    | OBJECT_REMOVE | OBJECT_ADD_FIELDS    |
+     * | OBJECT_CONCAT | OBJECT_RENAME | OBJECT_REMOVE_FIELDS |
+     * </pre>
+     * ...then we still might be able to use an index. Check the output type of applying our function(s) and verify
+     * that the input is a data source variable.
+     */
+    public static boolean canUseIndexOnFunction(AbstractFunctionCallExpression funcExpr, MutableInt sourceIndicator,
+            Collection<LogicalVariable> foundDatasourceVariables, IOptimizableFuncExpr optFuncExpr,
+            IVariableTypeEnvironment typeEnv, IOptimizationContext context) throws AlgebricksException {
+        FunctionIdentifier functionID = funcExpr.getFunctionIdentifier();
+        if (!INDEX_USE_ON_FUNCTION_CALL_WHITELIST.containsKey(functionID)) {
+            return false;
+        }
+
+        // Our output should be an object (this is more of a sanity check given that we have a whitelist).
+        IExpressionTypeComputer expressionTypeComputer = context.getExpressionTypeComputer();
+        IMetadataProvider<?, ?> metadataProvider = context.getMetadataProvider();
+        IAType originalOutputType = (IAType) expressionTypeComputer.getType(funcExpr, metadataProvider, typeEnv);
+        IAType outputType = TypeComputeUtils.getActualType(originalOutputType);
+        ARecordType outputRecordType = TypeComputeUtils.extractRecordType(outputType);
+        if (outputRecordType == null) {
+            return false;
+        }
+
+        // Check the type of our input, according to record variables in each function's argument.
+        boolean isDataSourceVariableFound = false;
+        Set<Integer> indicesToCheck = INDEX_USE_ON_FUNCTION_CALL_WHITELIST.get(functionID);
+        if (indicesToCheck.equals(ALL_INDEX_FUNCTION_ARGUMENTS)) {
+            indicesToCheck = IntStream.range(0, funcExpr.getArguments().size()).boxed().collect(Collectors.toSet());
+        }
+        for (Integer functionCallArgumentIndex : indicesToCheck) {
+            ILogicalExpression inputRecordExpr = funcExpr.getArguments().get(functionCallArgumentIndex).getValue();
+            switch (inputRecordExpr.getExpressionTag()) {
+                case FUNCTION_CALL:
+                    AbstractFunctionCallExpression arg0FuncExpr = (AbstractFunctionCallExpression) inputRecordExpr;
+                    isDataSourceVariableFound |= canUseIndexOnFunction(arg0FuncExpr, sourceIndicator,
+                            foundDatasourceVariables, optFuncExpr, typeEnv, context);
+                    break;
+
+                case VARIABLE:
+                    // Base case. We should be using a data source variable here.
+                    VariableReferenceExpression inputRecordVarExpr = (VariableReferenceExpression) inputRecordExpr;
+                    LogicalVariable inputRecordVar = inputRecordVarExpr.getVariableReference();
+                    if (optFuncExpr != null) {
+                        for (int i = 0; i < optFuncExpr.getNumLogicalVars(); i++) {
+                            OptimizableOperatorSubTree operatorSubTree = optFuncExpr.getOperatorSubTree(i);
+                            if (operatorSubTree == null) {
+                                continue;
+                            }
+                            if (operatorSubTree.getDataSourceVariables().stream().anyMatch(inputRecordVar::equals)) {
+                                OptimizableOperatorSubTree.RecordTypeSource recordTypeSource =
+                                        operatorSubTree.getRecordTypeFor(inputRecordVar);
+                                foundDatasourceVariables.add(inputRecordVar);
+                                sourceIndicator.setValue(recordTypeSource.sourceIndicator);
+                                isDataSourceVariableFound = true;
+                                break;
+                            }
+                        }
+                    }
+                    break;
+            }
+        }
+        return isDataSourceVariableFound;
+    }
 }
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.sqlpp
new file mode 100644
index 0000000000..76f8416f6b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.1.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, single OBJECT_ADD function application.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_ADD(U, "favoriteColor", "Green")
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp
new file mode 100644
index 0000000000..7e018b919d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.10.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, three OBJECT_ADD function applications.
+ * Primary index should used w/ INLJ.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+FROM   Test.Users U1, Test.Users U2
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza"),
+       augmentedUser3 = OBJECT_ADD(augmentedUser2, "favoriteColor", "Red"),
+       augmentedUser4 = OBJECT_ADD(augmentedUser3, "favoriteDrink", "Wine")
+WHERE  augmentedUser1.name = "John" AND
+       augmentedUser2.name = "Sally" AND
+       TO_BIGINT(U1.bestFriend) /* +indexnl */ = augmentedUser4.user_id
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp
new file mode 100644
index 0000000000..782060fdb0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.11.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Two indexes on nested fields, one OBJECT_ADD function application each.
+ * Index should be used in both cases.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint, name: { first: string } };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX firstUsersNameIdx ON Users ( name.first );
+CREATE INDEX lastUsersNameIdx ON Users ( name.last: string );
+
+FROM   Test.Users U1
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green")
+WHERE  augmentedUser1.name.first = "Glenn"
+SELECT augmentedUser1.*
+
+UNION ALL
+
+FROM   Test.Users U2
+LET    augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza")
+WHERE  augmentedUser2.name.last = "John"
+SELECT augmentedUser2.*;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp
new file mode 100644
index 0000000000..4583e1cc85
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.2.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, two OBJECT_ADD function applications.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser1 = OBJECT_ADD(U, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(augmentedUser1, "favoriteCity", "Irvine")
+WHERE  augmentedUser2.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp
new file mode 100644
index 0000000000..0d81c0fd86
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.3.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on nested field, one OBJECT_ADD function application.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name.first : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_ADD(U, "favoriteColor", "Green")
+WHERE  augmentedUser.name.first = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp
new file mode 100644
index 0000000000..95a60e663c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.4.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, one OBJECT_REMOVE function application.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_REMOVE(U, "favoriteColor")
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp
new file mode 100644
index 0000000000..41af9221e1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.5.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, one OBJECT_PUT function application.
+ * Index should NOT used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_PUT(U, "name", "John")
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp
new file mode 100644
index 0000000000..8b573b5063
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.6.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, one OBJECT_ADD function application.
+ * Primary index should used w/ INLJ.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+
+FROM   Test.Users U1, Test.Users U2
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza")
+WHERE  augmentedUser1.name = "John" AND
+       augmentedUser2.name = "Sally" AND
+       TO_BIGINT(U1.bestFriend) /* +indexnl */ = augmentedUser2.user_id
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp
new file mode 100644
index 0000000000..cb48c1c2e8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.7.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, OBJECT_ADD followed by OBJECT_REMOVE.
+ * Index should NOT be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_ADD(OBJECT_REMOVE(U, "name"), "name", "Glenn")
+WHERE  augmentedUser.firstName = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp
new file mode 100644
index 0000000000..8d780bf714
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.8.sqlpp
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, single OBJECT_CONCAT function application with multiple records.
+ * Index "usersNameIdx" should be used.
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U
+LET    augmentedUser = OBJECT_CONCAT({"favoriteColor": "Green"}, U, {"birthdate": "10/09/1996"})
+WHERE  augmentedUser.name = "Glenn"
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.sqlpp
new file mode 100644
index 0000000000..ad827e0552
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/index-through-object/index-through-object.9.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Index on top-level field, three OBJECT_ADD function applications.
+ * Index should be used twice (and not used for E1).
+ */
+
+DROP DATAVERSE Test IF EXISTS;
+CREATE DATAVERSE Test;
+USE Test;
+
+CREATE TYPE UsersType AS { user_id: bigint };
+CREATE DATASET Users (UsersType) PRIMARY KEY user_id;
+CREATE INDEX usersNameIdx ON Users ( name : string );
+
+FROM   Test.Users U1, Test.Users U2, [{"name": "Glenn"}] E1
+LET    augmentedUser1 = OBJECT_ADD(U1, "favoriteColor", "Green"),
+       augmentedUser2 = OBJECT_ADD(U2, "favoriteFood", "Pizza"),
+       augmentedUser3 = OBJECT_ADD(E1, "favoriteColor", "Blue")
+WHERE  augmentedUser1.name = "John" AND
+       augmentedUser2.name = "Sally" AND
+       augmentedUser3.name = "Glenn" AND
+       augmentedUser1.bestFriend = augmentedUser2.user_id
+SELECT *;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
new file mode 100644
index 0000000000..ff509f4d83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.1.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
new file mode 100644
index 0000000000..f5ab2c287b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.10.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$80]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
new file mode 100644
index 0000000000..6971ab466a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.11.plan
@@ -0,0 +1,39 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- UNION_ALL  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$78(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (Test.Users.firstUsersNameIdx)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_SELECT  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH (Test.Users.lastUsersNameIdx)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
new file mode 100644
index 0000000000..90816ce15a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.2.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
new file mode 100644
index 0000000000..693056009b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.3.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
new file mode 100644
index 0000000000..ff509f4d83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.4.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
new file mode 100644
index 0000000000..f0f6a0394c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.5.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
new file mode 100644
index 0000000000..b4aee6baab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.6.plan
@@ -0,0 +1,20 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
+                      -- HASH_PARTITION_EXCHANGE [$$58]  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_SELECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
new file mode 100644
index 0000000000..f0f6a0394c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.7.plan
@@ -0,0 +1,11 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- DATASOURCE_SCAN (Test.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
new file mode 100644
index 0000000000..693056009b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.8.plan
@@ -0,0 +1,18 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- ASSIGN  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
new file mode 100644
index 0000000000..1d2e55b9cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/index-through-object/index-through-object.9.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- NESTED_LOOP  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$82][$$83]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$82]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$106(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$83]  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH (Test.Users.Users)  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$110(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH (Test.Users.usersNameIdx)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+            -- BROADCAST_EXCHANGE  |PARTITIONED|
+              -- STREAM_SELECT  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- UNNEST  |UNPARTITIONED|
+                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.sqlpp
new file mode 100644
index 0000000000..2eefbe8e05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.5.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       UsersBaseTypeClosed AS CLOSED { _id: int };
+CREATE TYPE       UsersBaseTypeOpen AS { _id: int };
+CREATE TYPE       UsersFriendsTypeClosed AS CLOSED {
+  _id: int,
+  best_friend: UsersBaseTypeClosed,
+  friends: [UsersBaseTypeClosed]
+};
+CREATE TYPE       UsersFriendsTypeOpen AS { _id: int };
+
+CREATE DATASET    UsersClosed (UsersFriendsTypeClosed) PRIMARY KEY _id;
+CREATE DATASET    UsersOpen (UsersFriendsTypeOpen) PRIMARY KEY _id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp
new file mode 100644
index 0000000000..bda29eb758
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.6.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE               TestDataverse;
+
+INSERT INTO       UsersClosed [
+  { "_id": 1, "best_friend": { "_id": 1 }, "friends": [] },
+  { "_id": 2, "best_friend": { "_id": 1 }, "friends": [{ "_id": 1 }] },
+  { "_id": 3, "best_friend": { "_id": 2 }, "friends": [{ "_id": 1 }, { "_id": 2 }] }
+];
+INSERT INTO       UsersOpen [
+  { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [] },
+  { "_id": 5, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [{ "_id": 4 }] },
+  { "_id": 6, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [{ "_id": 4 }, { "_id": 5 }] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp
new file mode 100644
index 0000000000..71f6974766
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_add/object_add.7.query.sqlpp
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// For OBJECT_ADD.
+USE     TestDataverse;
+LET     openUValues = (
+          FROM    UsersOpen U2
+          WHERE   U2._id = 4
+          SELECT  VALUE U2
+        ),
+        closedUValues = (
+          FROM    UsersClosed U1
+          WHERE   U1._id = 1
+          SELECT  VALUE U1
+        )
+SELECT  VALUE {
+  // New constant field into closed and open record.
+  "t1c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "name", "John") ORDER BY U._id ),
+  "t1o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "name", "John") ORDER BY U._id ),
+
+  // New record field from open record into closed and open record.
+  "t2c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+  "t2o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+
+  // New record field from closed record into closed and open record.
+  "t3c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+  "t3o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+
+  // New list field from open dataset into closed and open record.
+  "t4c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriends", openUValues) ORDER BY U._id ),
+  "t4o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriends", openUValues) ORDER BY U._id ),
+
+  // New list field from closed dataset into closed and open record.
+  "t5c": ( FROM UsersClosed U SELECT VALUE OBJECT_ADD(U, "newFriends", closedUValues) ORDER BY U._id ),
+  "t5o": ( FROM UsersOpen U SELECT VALUE OBJECT_ADD(U, "newFriends", closedUValues) ORDER BY U._id )
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp
new file mode 100644
index 0000000000..2eefbe8e05
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.5.ddl.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       UsersBaseTypeClosed AS CLOSED { _id: int };
+CREATE TYPE       UsersBaseTypeOpen AS { _id: int };
+CREATE TYPE       UsersFriendsTypeClosed AS CLOSED {
+  _id: int,
+  best_friend: UsersBaseTypeClosed,
+  friends: [UsersBaseTypeClosed]
+};
+CREATE TYPE       UsersFriendsTypeOpen AS { _id: int };
+
+CREATE DATASET    UsersClosed (UsersFriendsTypeClosed) PRIMARY KEY _id;
+CREATE DATASET    UsersOpen (UsersFriendsTypeOpen) PRIMARY KEY _id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp
new file mode 100644
index 0000000000..bda29eb758
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.6.update.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE               TestDataverse;
+
+INSERT INTO       UsersClosed [
+  { "_id": 1, "best_friend": { "_id": 1 }, "friends": [] },
+  { "_id": 2, "best_friend": { "_id": 1 }, "friends": [{ "_id": 1 }] },
+  { "_id": 3, "best_friend": { "_id": 2 }, "friends": [{ "_id": 1 }, { "_id": 2 }] }
+];
+INSERT INTO       UsersOpen [
+  { "_id": 4, "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [] },
+  { "_id": 5, "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [{ "_id": 4 }] },
+  { "_id": 6, "favorite_color": "Orange", "best_friend": { "_id": 5 }, "friends": [{ "_id": 4 }, { "_id": 5 }] }
+];
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp
new file mode 100644
index 0000000000..52943a08e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_put/object_put.7.query.sqlpp
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// For OBJECT_PUT.
+USE     TestDataverse;
+LET     openUValues = (
+          FROM    UsersOpen U2
+          WHERE   U2._id = 4
+          SELECT  VALUE U2
+        ),
+        closedUValues = (
+          FROM    UsersClosed U1
+          WHERE   U1._id = 1
+          SELECT  VALUE U1
+        )
+SELECT  VALUE {
+  // New constant field into closed and open record.
+  "t1c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "name", "John") ORDER BY U._id ),
+  "t1o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "name", "John") ORDER BY U._id ),
+
+  // New record field from open record into closed and open record.
+  "t2c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+  "t2o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriend", openUValues[0]) ORDER BY U._id ) ,
+
+  // New record field from closed record into closed and open record.
+  "t3c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+  "t3o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriend", closedUValues[0]) ORDER BY U._id ),
+
+  // New list field from open dataset into closed and open record.
+  "t4c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriends", openUValues) ORDER BY U._id ),
+  "t4o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriends", openUValues) ORDER BY U._id ),
+
+  // New list field from closed dataset into closed and open record.
+  "t5c": ( FROM UsersClosed U SELECT VALUE OBJECT_PUT(U, "newFriends", closedUValues) ORDER BY U._id ),
+  "t5o": ( FROM UsersOpen U SELECT VALUE OBJECT_PUT(U, "newFriends", closedUValues) ORDER BY U._id ),
+
+  // Old field of open type into closed record (we should overwrite).
+  "t6c": ( FROM   UsersClosed U
+           LET    newFriends = [
+             { "_id": 8 }, { "_id": 9 }
+           ]
+           SELECT VALUE OBJECT_PUT(U, "friends", newFriends)
+           ORDER BY U._id )
+};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
index 800859b704..94bb2b5fbf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, " [...]
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": 1 }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "field": "value", "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": 39339, " [...]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm
new file mode 100644
index 0000000000..3bbbabbb5c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_add/object_add.7.adm
@@ -0,0 +1 @@
+{ "t1c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "name": "John" }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "name": "John" }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "name": "John" } ], "t1o": [ { "_id": 4, "name": "John", "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "name": "John", "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_i [...]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
index 4b97741bfe..2947405493 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.3.adm
@@ -1 +1 @@
-{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "name": "Chang Ewing", "followers_count": 32136, "field": "value" } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": [...]
+{ "t1": [ true, true, true, true, true, true, true ], "t2": { "a": 1, "field": "value" }, "t3": { "a": 1, "field": null }, "t4": { "a": 1, "field": { "x": [ "y", "z" ] } }, "t5": { "a": "replaced" }, "t6": { "a": 1 }, "t7": [ { "screen-name": "ChangEwing_573", "lang": "en", "friends_count": 182, "statuses_count": 394, "field": "value", "name": "Chang Ewing", "followers_count": 32136 } ], "t8": [ { "tweetid": "1", "user": { "screen-name": "NathanGiesen@211", "lang": "en", "friends_count": [...]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm
new file mode 100644
index 0000000000..32b297d4ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/object_put/object_put.7.adm
@@ -0,0 +1 @@
+{ "t1c": [ { "_id": 1, "best_friend": { "_id": 1 }, "friends": [  ], "name": "John" }, { "_id": 2, "best_friend": { "_id": 1 }, "friends": [ { "_id": 1 } ], "name": "John" }, { "_id": 3, "best_friend": { "_id": 2 }, "friends": [ { "_id": 1 }, { "_id": 2 } ], "name": "John" } ], "t1o": [ { "_id": 4, "name": "John", "favorite_color": "Green", "best_friend": { "_id": 4 }, "friends": [  ] }, { "_id": 5, "name": "John", "favorite_color": "Blue", "best_friend": { "_id": 4 }, "friends": [ { "_i [...]
\ No newline at end of file
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index 56971ebbe2..2f3ff5955a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -119,8 +119,12 @@ import org.apache.asterix.om.typecomputer.impl.OrderedListOfAStringTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAnyTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.PropagateTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordAddFieldsTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordAddTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordMergeTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordPutTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordRemoveFieldsTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordRemoveTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.RecordRenameTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ScalarArrayAggTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ScalarVersionOfAggregateResultType;
 import org.apache.asterix.om.typecomputer.impl.SleepTypeComputer;
@@ -2425,12 +2429,12 @@ public class BuiltinFunctions {
         addFunction(RECORD_PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(PAIRS, OrderedListOfAnyTypeComputer.INSTANCE_NULLABLE, true);
         addFunction(GEOMETRY_CONSTRUCTOR, AGeometryTypeComputer.INSTANCE, true);
-        addFunction(RECORD_REMOVE, OpenARecordTypeComputer.INSTANCE, true);
-        addFunction(RECORD_RENAME, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_REMOVE, RecordRemoveTypeComputer.INSTANCE, true);
+        addFunction(RECORD_RENAME, RecordRenameTypeComputer.INSTANCE, true);
         addFunction(RECORD_UNWRAP, AnyTypeComputer.INSTANCE, true);
         addFunction(RECORD_REPLACE, OpenARecordTypeComputer.INSTANCE, true);
-        addFunction(RECORD_ADD, OpenARecordTypeComputer.INSTANCE, true);
-        addFunction(RECORD_PUT, OpenARecordTypeComputer.INSTANCE, true);
+        addFunction(RECORD_ADD, RecordAddTypeComputer.INSTANCE, true);
+        addFunction(RECORD_PUT, RecordPutTypeComputer.INSTANCE, true);
         addFunction(RECORD_VALUES, OrderedListOfAnyTypeComputer.INSTANCE, true);
 
         // temporal type accessors
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java
new file mode 100644
index 0000000000..580cab4221
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/AbstractRecordFunctionTypeComputer.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.IResultTypeComputer;
+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.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeHelper;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.metadata.IMetadataProvider;
+
+/**
+ * Base type computer for the following record-functions:
+ * 1. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_ADD}
+ * 2. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_PUT}
+ * 3. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_REMOVE}
+ * 4. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_RENAME}
+ */
+public abstract class AbstractRecordFunctionTypeComputer implements IResultTypeComputer {
+    @Override
+    public IAType computeType(ILogicalExpression expression, IVariableTypeEnvironment env,
+            IMetadataProvider<?, ?> metadataProvider) throws AlgebricksException {
+        AbstractFunctionCallExpression functionCallExpression = (AbstractFunctionCallExpression) expression;
+
+        // Get our record type.
+        ILogicalExpression arg0 = functionCallExpression.getArguments().get(0).getValue();
+        IAType type0 = (IAType) env.getType(arg0);
+        IAType actualType0 = TypeComputeUtils.getActualType(type0);
+        ATypeTag tag0 = actualType0.getTypeTag();
+        if (tag0 == ATypeTag.ANY) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, true, true);
+        } else if (tag0 == ATypeTag.MISSING) {
+            // Our output is always going to be MISSING.
+            return BuiltinType.AMISSING;
+        } else if (tag0 != ATypeTag.OBJECT) {
+            // Our output is always going to be NULL.
+            return BuiltinType.ANULL;
+        }
+        boolean isOutputMissable = TypeHelper.canBeMissing(type0);
+        boolean isOutputNullable = TypeHelper.canBeNull(type0);
+        ARecordType inputRecordType = TypeComputeUtils.extractRecordType(actualType0);
+
+        // Our second argument should be of type "string".
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        IAType type1 = (IAType) env.getType(arg1);
+        IAType actualType1 = TypeComputeUtils.getActualType(type1);
+        ATypeTag tag1 = actualType1.getTypeTag();
+        if (tag1 == ATypeTag.ANY) {
+            // We cannot infer the type of our second argument-- our output may be MISSING or NULL.
+            return wrapTypeWithUnknown(type0, true, true);
+        } else if (tag1 == ATypeTag.MISSING) {
+            // Our output is always going to be MISSING.
+            return BuiltinType.AMISSING;
+        } else if (tag1 != ATypeTag.STRING) {
+            // Our output is always going to be NULL.
+            return BuiltinType.ANULL;
+        }
+        isOutputMissable |= TypeHelper.canBeMissing(type1);
+        isOutputNullable |= TypeHelper.canBeNull(type1);
+
+        // Compute our type.
+        return computeTypeImpl(functionCallExpression, env, inputRecordType, isOutputMissable, isOutputNullable);
+    }
+
+    protected abstract IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression,
+            IVariableTypeEnvironment env, ARecordType inputRecordType, boolean isOutputMissable,
+            boolean isOutputNullable) throws AlgebricksException;
+
+    protected static IAType wrapTypeWithUnknown(IAType originalType, boolean isMissable, boolean isNullable) {
+        if (isNullable && isMissable) {
+            return AUnionType.createUnknownableType(originalType);
+        } else if (isNullable) { // && !isMissable
+            return AUnionType.createNullableType(originalType);
+        } else if (isMissable) { // && !isNullable
+            return AUnionType.createMissableType(originalType);
+        } else {
+            return originalType;
+        }
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.java
new file mode 100644
index 0000000000..481534041a
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordAddTypeComputer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordAddTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordAddTypeComputer INSTANCE = new RecordAddTypeComputer();
+
+    private RecordAddTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+            throws AlgebricksException {
+        // If our third argument is missing, then just return the type of our first argument.
+        ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+        IAType type2 = (IAType) env.getType(arg2);
+        IAType actualType2 = TypeComputeUtils.getActualType(type2);
+        ATypeTag tag2 = actualType2.getTypeTag();
+        if (tag2 == ATypeTag.MISSING) {
+            IAType type0 = (IAType) env.getType(functionCallExpression.getArguments().get(0).getValue());
+            return wrapTypeWithUnknown(type0, isOutputMissable, isOutputNullable);
+        }
+
+        // We expect a constant for our second argument.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression constantExpression = (ConstantExpression) arg1;
+        AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+        String newFieldName = ((AString) constantValue.getObject()).getStringValue();
+
+        // If our field is found, return the original record type. Otherwise, add our new field.
+        ARecordType outputRecordType = inputRecordType;
+        if (!Arrays.asList(inputRecordType.getFieldNames()).contains(newFieldName)) {
+            String[] fieldNames = new String[inputRecordType.getFieldNames().length + 1];
+            IAType[] fieldTypes = new IAType[inputRecordType.getFieldTypes().length + 1];
+            int currentCursorPosition = 0;
+            for (; currentCursorPosition < inputRecordType.getFieldNames().length; currentCursorPosition++) {
+                fieldNames[currentCursorPosition] = inputRecordType.getFieldNames()[currentCursorPosition];
+                fieldTypes[currentCursorPosition] = inputRecordType.getFieldTypes()[currentCursorPosition];
+            }
+            fieldNames[currentCursorPosition] = newFieldName;
+            fieldTypes[currentCursorPosition] = type2;
+            String inputTypeName = inputRecordType.getTypeName();
+            String outputTypeName = inputTypeName != null ? inputTypeName + "_add_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, fieldNames, fieldTypes, inputRecordType.isOpen());
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java
new file mode 100644
index 0000000000..a92907de21
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordPutTypeComputer.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordPutTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordPutTypeComputer INSTANCE = new RecordPutTypeComputer();
+
+    private RecordPutTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+            throws AlgebricksException {
+        // Extract the type of our third argument. If it is MISSING, then we are performing a field removal.
+        ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+        IAType type2 = (IAType) env.getType(arg2);
+        IAType actualType2 = TypeComputeUtils.getActualType(type2);
+        boolean isFieldRemoval = actualType2.getTypeTag() == ATypeTag.MISSING;
+
+        // We expect a constant for our second argument.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression constantExpression = (ConstantExpression) arg1;
+        AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+        String newFieldName = ((AString) constantValue.getObject()).getStringValue();
+
+        // Remove or replace our field name and type (dependent on the type of our third argument).
+        boolean fieldFound = false;
+        List<String> outputFieldNames = new ArrayList<>();
+        List<IAType> outputFieldTypes = new ArrayList<>();
+        for (int i = 0; i < inputRecordType.getFieldNames().length; i++) {
+            String inputFieldName = inputRecordType.getFieldNames()[i];
+            IAType inputFieldType = inputRecordType.getFieldTypes()[i];
+            if (!inputFieldName.equals(newFieldName)) {
+                outputFieldNames.add(inputFieldName);
+                outputFieldTypes.add(inputFieldType);
+
+            } else {
+                fieldFound = true;
+                if (!isFieldRemoval) {
+                    // Replace our input field type.
+                    outputFieldNames.add(inputFieldName);
+                    outputFieldTypes.add(type2);
+                }
+            }
+        }
+
+        // Build our output record type.
+        ARecordType outputRecordType;
+        String inputTypeName = inputRecordType.getTypeName();
+        boolean doesRecordHaveTypeName = inputTypeName != null;
+        if (fieldFound && isFieldRemoval) {
+            // We have removed our argument field.
+            String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_remove_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+                    outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+        } else if (fieldFound) { // && !isFieldRemoval
+            // We have replaced our argument field.
+            String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_replaced_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+                    outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+        } else if (!isFieldRemoval) { // && !wasFieldFound
+            // We need to insert our argument field.
+            outputFieldNames.add(newFieldName);
+            outputFieldTypes.add(type2);
+            String outputTypeName = doesRecordHaveTypeName ? inputTypeName + "_add_" + newFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, outputFieldNames.toArray(String[]::new),
+                    outputFieldTypes.toArray(IAType[]::new), inputRecordType.isOpen());
+        } else { // isFieldRemoval && !wasFieldFound
+            // We have not found the field to remove.
+            outputRecordType = inputRecordType;
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java
new file mode 100644
index 0000000000..c447e65b87
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRemoveTypeComputer.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordRemoveTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordRemoveTypeComputer INSTANCE = new RecordRemoveTypeComputer();
+
+    private RecordRemoveTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable) {
+        // We expect a CONSTANT expression. Otherwise, defer the removal to runtime.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression constantExpression = (ConstantExpression) arg1;
+        AsterixConstantValue constantValue = (AsterixConstantValue) constantExpression.getValue();
+        String fieldName = ((AString) constantValue.getObject()).getStringValue();
+
+        // If our field is found, remove it. Otherwise, return the original record type.
+        ARecordType outputRecordType = inputRecordType;
+        if (Arrays.asList(inputRecordType.getFieldNames()).contains(fieldName)) {
+            String[] fieldNames = new String[inputRecordType.getFieldNames().length - 1];
+            IAType[] fieldTypes = new IAType[inputRecordType.getFieldTypes().length - 1];
+            int currentOutputCursor = 0;
+            for (int i = 0; i < inputRecordType.getFieldNames().length; i++) {
+                String inputName = inputRecordType.getFieldNames()[i];
+                IAType inputType = inputRecordType.getFieldTypes()[i];
+                if (!inputName.equals(fieldName)) {
+                    fieldNames[currentOutputCursor] = inputName;
+                    fieldTypes[currentOutputCursor] = inputType;
+                    currentOutputCursor++;
+                }
+            }
+            String inputTypeName = inputRecordType.getTypeName();
+            String outputTypeName = inputTypeName != null ? inputTypeName + "_remove_" + fieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, fieldNames, fieldTypes, inputRecordType.isOpen());
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java
new file mode 100644
index 0000000000..4302082042
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/RecordRenameTypeComputer.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.om.typecomputer.impl;
+
+import java.util.Arrays;
+
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+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.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeHelper;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+
+public class RecordRenameTypeComputer extends AbstractRecordFunctionTypeComputer {
+    public static final RecordRenameTypeComputer INSTANCE = new RecordRenameTypeComputer();
+
+    private RecordRenameTypeComputer() {
+    }
+
+    @Override
+    public IAType computeTypeImpl(AbstractFunctionCallExpression functionCallExpression, IVariableTypeEnvironment env,
+            ARecordType inputRecordType, boolean isOutputMissable, boolean isOutputNullable)
+            throws AlgebricksException {
+        // Our third argument should be of type "string".
+        ILogicalExpression arg2 = functionCallExpression.getArguments().get(2).getValue();
+        IAType type2 = (IAType) env.getType(arg2);
+        IAType actualType2 = TypeComputeUtils.getActualType(type2);
+        ATypeTag tag2 = actualType2.getTypeTag();
+        if (tag2 == ATypeTag.ANY) {
+            // We cannot infer the type of our third argument-- our output may be MISSING or NULL.
+            return AUnionType.createUnknownableType(inputRecordType, inputRecordType.getTypeName() + "?");
+        } else if (tag2 == ATypeTag.MISSING) {
+            // Our output is always going to be MISSING.
+            return BuiltinType.AMISSING;
+        } else if (tag2 != ATypeTag.STRING) {
+            // Our output is always going to be NULL.
+            return BuiltinType.ANULL;
+        }
+        isOutputMissable |= TypeHelper.canBeMissing(type2);
+        isOutputNullable |= TypeHelper.canBeNull(type2);
+
+        // We expect a CONSTANT expression for both arguments. Otherwise, defer the replacement to runtime.
+        ILogicalExpression arg1 = functionCallExpression.getArguments().get(1).getValue();
+        if (arg1.getExpressionTag() != LogicalExpressionTag.CONSTANT
+                || arg2.getExpressionTag() != LogicalExpressionTag.CONSTANT) {
+            return wrapTypeWithUnknown(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE, isOutputMissable,
+                    isOutputNullable);
+        }
+        ConstantExpression arg1ConstantExpression = (ConstantExpression) arg1;
+        ConstantExpression arg2ConstantExpression = (ConstantExpression) arg2;
+        AsterixConstantValue arg1ConstantValue = (AsterixConstantValue) arg1ConstantExpression.getValue();
+        AsterixConstantValue arg2ConstantValue = (AsterixConstantValue) arg2ConstantExpression.getValue();
+        String oldFieldName = ((AString) arg1ConstantValue.getObject()).getStringValue();
+        String newFieldName = ((AString) arg2ConstantValue.getObject()).getStringValue();
+
+        // If our field is found, replace it. Otherwise, return the original record type.
+        Mutable<Boolean> fieldFound = new MutableObject<>(false);
+        String[] newFieldNames = Arrays.stream(inputRecordType.getFieldNames()).map(f -> {
+            if (f.equals(oldFieldName)) {
+                fieldFound.setValue(true);
+                return newFieldName;
+            } else {
+                return f;
+            }
+        }).toArray(String[]::new);
+        ARecordType outputRecordType;
+        if (!fieldFound.getValue()) {
+            outputRecordType = inputRecordType;
+        } else {
+            String inputTypeName = inputRecordType.getTypeName();
+            String outputTypeName = inputTypeName != null ? inputTypeName + "_replaced_" + oldFieldName : null;
+            outputRecordType = new ARecordType(outputTypeName, newFieldNames, inputRecordType.getFieldTypes(),
+                    inputRecordType.isOpen());
+        }
+        return wrapTypeWithUnknown(outputRecordType, isOutputMissable, isOutputNullable);
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
index 47e84eb38d..ed7da3ff39 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordAddPutEvaluator.java
@@ -16,101 +16,64 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.runtime.evaluators.functions.CastTypeEvaluator;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-abstract class AbstractRecordAddPutEvaluator implements IScalarEvaluator {
-
-    private final CastTypeEvaluator inputRecordCaster;
-    private final CastTypeEvaluator argRecordCaster;
+abstract class AbstractRecordAddPutEvaluator extends AbstractRecordFunctionEvaluator {
     private final IScalarEvaluator eval0;
     private final IScalarEvaluator eval1;
     private final IScalarEvaluator eval2;
-    final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    final DataOutput resultOutput = resultStorage.getDataOutput();
-    final IPointable inputRecordPointable = new VoidPointable();
-    final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
-    final IPointable newFieldValuePointable = new VoidPointable();
-    final IBinaryComparator stringBinaryComparator =
-            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-    final RecordBuilder outRecordBuilder = new RecordBuilder();
-    final ARecordVisitablePointable inputOpenRecordPointable;
-    boolean newFieldValueIsMissing = false;
+    protected boolean newFieldValueIsMissing = false;
 
     AbstractRecordAddPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
-            IAType[] argTypes) {
+            ARecordType outRecType, ARecordType inRecType) {
+        super(outRecType, inRecType);
         this.eval0 = eval0;
         this.eval1 = eval1;
         this.eval2 = eval2;
-        inputOpenRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        inputRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[0], eval0);
-        argRecordCaster = new CastTypeEvaluator(BuiltinType.ANY, argTypes[2], eval2);
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        eval0.evaluate(tuple, inputRecordPointable);
+        eval0.evaluate(tuple, inputPointable);
         eval1.evaluate(tuple, newFieldNamePointable);
         eval2.evaluate(tuple, newFieldValuePointable);
-        if (containsMissing(inputRecordPointable, newFieldNamePointable)) {
-            writeTypeTag(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
-            result.set(resultStorage);
+        ATypeTag inputTypeTag = PointableHelper.getTypeTag(inputPointable);
+        ATypeTag newFieldNameTypeTag = PointableHelper.getTypeTag(newFieldNamePointable);
+        if (inputTypeTag == ATypeTag.MISSING || newFieldNameTypeTag == ATypeTag.MISSING) {
+            PointableHelper.setMissing(result);
             return;
         }
-        final ATypeTag inputObjectType = PointableHelper.getTypeTag(inputRecordPointable);
-        final ATypeTag newFieldNameValueType = PointableHelper.getTypeTag(newFieldNamePointable);
-        if (inputObjectType != ATypeTag.OBJECT || newFieldNameValueType != ATypeTag.STRING) {
+        if (inputTypeTag != ATypeTag.OBJECT || newFieldNameTypeTag != ATypeTag.STRING) {
             PointableHelper.setNull(result);
             return;
         }
-        inputRecordCaster.evaluate(tuple, inputRecordPointable);
-        final ATypeTag newFieldValueTag = PointableHelper.getTypeTag(newFieldValuePointable);
-        if (newFieldValueTag.isDerivedType()) {
-            argRecordCaster.evaluate(tuple, newFieldValuePointable);
+        newFieldValueIsMissing = PointableHelper.getTypeTag(newFieldValuePointable) == ATypeTag.MISSING;
+        outputRecordTypeInfo.reset(outRecType);
+        if (inputRecordPointable == null) {
+            inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
         }
-        newFieldValueIsMissing = newFieldValueTag == ATypeTag.MISSING;
-        buildOutputRecord();
+        buildOutputRecord(result);
         result.set(resultStorage);
     }
 
-    protected abstract void buildOutputRecord() throws HyracksDataException;
-
-    private boolean containsMissing(IPointable... pointables) {
-        for (int i = 0; i < pointables.length; i++) {
-            if (PointableHelper.getTypeTag(pointables[i]) == ATypeTag.MISSING) {
-                return true;
-            }
-        }
-        return false;
-    }
+    protected abstract void buildOutputRecord(IPointable result) throws HyracksDataException;
 
-    private void writeTypeTag(byte typeTag) throws HyracksDataException {
-        try {
-            resultOutput.writeByte(typeTag);
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
+    protected void addField(IPointable fieldName, IPointable fieldValue) throws HyracksDataException {
+        int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(), fieldName.getStartOffset() + 1,
+                fieldName.getLength() - 1);
+        if (pos >= 0) {
+            outRecordBuilder.addField(pos, fieldValue);
+        } else {
+            outRecordBuilder.addField(fieldName, fieldValue);
         }
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java
new file mode 100644
index 0000000000..2bb192d192
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/AbstractRecordFunctionEvaluator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.asterix.runtime.evaluators.functions.records;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.builders.RecordBuilder;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.PointableAllocator;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.runtime.RuntimeRecordTypeInfo;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+
+/**
+ * Base evaluator class for the following functions:
+ * 1. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_ADD}
+ * 2. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_PUT}
+ * 3. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_REMOVE}
+ * 4. {@link org.apache.asterix.om.functions.BuiltinFunctions#RECORD_RENAME}
+ */
+abstract class AbstractRecordFunctionEvaluator implements IScalarEvaluator {
+    protected final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+    protected final DataOutput resultOutput = resultStorage.getDataOutput();
+    protected final RecordBuilder outRecordBuilder = new RecordBuilder();
+
+    protected final IPointable newFieldNamePointable = new VoidPointable();
+    protected final IPointable newFieldValuePointable = new VoidPointable();
+    protected final IBinaryComparator stringBinaryComparator =
+            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+
+    protected final PointableAllocator pointableAllocator = new PointableAllocator();
+    protected final IPointable inputPointable = new VoidPointable();
+    protected final ARecordType inRecType;
+    protected ARecordVisitablePointable inputRecordPointable;
+
+    protected final ARecordType outRecType;
+    protected final RuntimeRecordTypeInfo outputRecordTypeInfo = new RuntimeRecordTypeInfo();
+
+    AbstractRecordFunctionEvaluator(ARecordType outRecType, ARecordType inRecType) {
+        this.outRecType = outRecType;
+        this.inRecType = inRecType;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
index 67f2c8da4f..075b80380b 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddDescriptor.java
@@ -22,6 +22,8 @@ import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
@@ -32,7 +34,6 @@ import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class RecordAddDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -41,19 +42,18 @@ public class RecordAddDescriptor extends AbstractScalarFunctionDynamicDescriptor
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private IAType[] argTypes;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        argTypes = new IAType[states.length];
-        for (int i = 0; i < states.length; i++) {
-            argTypes[i] = (IAType) states[i];
-        }
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -67,7 +67,7 @@ public class RecordAddDescriptor extends AbstractScalarFunctionDynamicDescriptor
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordAddEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+                return new RecordAddEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
index 52a69bf02a..3df16d00d2 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordAddEvaluator.java
@@ -16,42 +16,41 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
 import java.util.List;
 
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
 
 class RecordAddEvaluator extends AbstractRecordAddPutEvaluator {
-
-    RecordAddEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
-        super(eval0, eval1, eval2, argTypes);
+    RecordAddEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, ARecordType outRecType,
+            ARecordType inRecType) {
+        super(eval0, eval1, eval2, outRecType, inRecType);
     }
 
     @Override
-    protected void buildOutputRecord() throws HyracksDataException {
+    protected void buildOutputRecord(IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        inputOpenRecordPointable.set(inputRecordPointable);
-        final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+        outRecordBuilder.reset(outRecType);
+        inputRecordPointable.set(inputPointable);
+        final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
         boolean newFieldFound = false;
         for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
             final IVisitablePointable fieldName = fieldNames.get(i);
-            if (PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+            final IVisitablePointable fieldValue = fieldValues.get(i);
+            if (!newFieldFound && PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
                 newFieldFound = true;
             }
-            outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            addField(fieldName, fieldValue);
         }
         if (!newFieldValueIsMissing && !newFieldFound) {
-            outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
+            addField(newFieldNamePointable, newFieldValuePointable);
         }
         outRecordBuilder.write(resultOutput, true);
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
index e4d72e5ad3..870a2e82ca 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutDescriptor.java
@@ -22,6 +22,8 @@ import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
@@ -32,7 +34,6 @@ import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class RecordPutDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -41,19 +42,18 @@ public class RecordPutDescriptor extends AbstractScalarFunctionDynamicDescriptor
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.SET_ARGUMENTS_TYPE;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private IAType[] argTypes;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        argTypes = new IAType[states.length];
-        for (int i = 0; i < states.length; i++) {
-            argTypes[i] = (IAType) states[i];
-        }
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -67,7 +67,7 @@ public class RecordPutDescriptor extends AbstractScalarFunctionDynamicDescriptor
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordPutEvaluator(argEvals[0], argEvals[1], argEvals[2], argTypes);
+                return new RecordPutEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
index 857bd2ee04..3a9f3f8612 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordPutEvaluator.java
@@ -16,51 +16,46 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
 import java.util.List;
 
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
 
 class RecordPutEvaluator extends AbstractRecordAddPutEvaluator {
-
-    RecordPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, IAType[] argTypes) {
-        super(eval0, eval1, eval2, argTypes);
+    RecordPutEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2, ARecordType outRecType,
+            ARecordType inRecType) {
+        super(eval0, eval1, eval2, outRecType, inRecType);
     }
 
     @Override
-    protected void buildOutputRecord() throws HyracksDataException {
+    protected void buildOutputRecord(IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        inputOpenRecordPointable.set(inputRecordPointable);
-        final List<IVisitablePointable> fieldNames = inputOpenRecordPointable.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputOpenRecordPointable.getFieldValues();
+        outRecordBuilder.reset(outRecType);
+        inputRecordPointable.set(inputPointable);
+        final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+        final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
         boolean newFieldFound = false;
         for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
             final IVisitablePointable fieldName = fieldNames.get(i);
-            if (!PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
-                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            final IVisitablePointable fieldValue = fieldValues.get(i);
+            if (!newFieldFound && !PointableHelper.isEqual(fieldName, newFieldNamePointable, stringBinaryComparator)) {
+                addField(fieldName, fieldValue);
             } else {
                 newFieldFound = true;
                 if (!newFieldValueIsMissing) {
-                    putNewField();
+                    addField(newFieldNamePointable, newFieldValuePointable);
                 }
             }
         }
         if (!newFieldFound) {
-            putNewField();
+            addField(newFieldNamePointable, newFieldValuePointable);
         }
         outRecordBuilder.write(resultOutput, true);
     }
-
-    private void putNewField() throws HyracksDataException {
-        outRecordBuilder.addField(newFieldNamePointable, newFieldValuePointable);
-    }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
index 8e44ada15c..818b650c79 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveDescriptor.java
@@ -23,7 +23,9 @@ import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,7 +36,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 @MissingNullInOutFunction
 public class RecordRemoveDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -43,16 +44,18 @@ public class RecordRemoveDescriptor extends AbstractScalarFunctionDynamicDescrip
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private ARecordType recordType;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        recordType = (ARecordType) states[0];
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -66,7 +69,7 @@ public class RecordRemoveDescriptor extends AbstractScalarFunctionDynamicDescrip
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordRemoveEvaluator(argEvals[0], argEvals[1], recordType);
+                return new RecordRemoveEvaluator(argEvals[0], argEvals[1], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
index 51f66b079a..e560c10290 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRemoveEvaluator.java
@@ -16,140 +16,87 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.pointables.cast.ACastVisitor;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-class RecordRemoveEvaluator implements IScalarEvaluator {
-
-    private final IPointable inputRecordPointable = new VoidPointable();
-    private final UTF8StringPointable fieldToRemovePointable = new UTF8StringPointable();
-    private final IBinaryComparator stringBinaryComparator =
-            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    private final DataOutput resultOutput = resultStorage.getDataOutput();
-    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+class RecordRemoveEvaluator extends AbstractRecordFunctionEvaluator {
     private final IScalarEvaluator eval0;
     private final IScalarEvaluator eval1;
-    private final ARecordVisitablePointable openRecordPointable;
-    private ARecordVisitablePointable inputRecordVisitable;
-    private boolean requiresCast = false;
-    private ACastVisitor castVisitor;
-    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+    private final IPointable fieldToRemovePointable = new VoidPointable();
 
-    RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType recordType) {
+    RecordRemoveEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, ARecordType outRecType,
+            ARecordType inRecType) {
+        super(outRecType, inRecType);
         this.eval0 = eval0;
         this.eval1 = eval1;
-        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        if (recordType != null) {
-            inputRecordVisitable = new ARecordVisitablePointable(recordType);
-            if (hasDerivedType(recordType.getFieldTypes())) {
-                requiresCast = true;
-                castVisitor = new ACastVisitor();
-                castVisitorArg =
-                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
-            }
-        }
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        boolean returnNull = false;
-        eval0.evaluate(tuple, inputRecordPointable);
+        eval0.evaluate(tuple, inputPointable);
         eval1.evaluate(tuple, fieldToRemovePointable);
-
-        if (PointableHelper.checkAndSetMissingOrNull(result, inputRecordPointable, fieldToRemovePointable)) {
+        if (PointableHelper.checkAndSetMissingOrNull(result, inputPointable, fieldToRemovePointable)) {
             return;
         }
 
-        byte[] data = inputRecordPointable.getByteArray();
-        int offset = inputRecordPointable.getStartOffset();
+        // Check the type of our first argument.
+        byte[] data = inputPointable.getByteArray();
+        int offset = inputPointable.getStartOffset();
         byte typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-            returnNull = true;
+            PointableHelper.setNull(result);
+            return;
         }
 
+        // Check the type of our second argument.
         data = fieldToRemovePointable.getByteArray();
         offset = fieldToRemovePointable.getStartOffset();
         typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            returnNull = true;
-        }
-        if (returnNull) {
             PointableHelper.setNull(result);
             return;
         }
-        evaluate();
-        result.set(resultStorage);
-    }
 
-    private void evaluate() throws HyracksDataException {
-        resultStorage.reset();
         try {
-            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
-            buildOutputRecord(inputRecord);
-        } catch (IOException e) {
-            throw HyracksDataException.create(e);
-        }
-    }
-
-    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
-        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
-            final IVisitablePointable fieldName = fieldNames.get(i);
-            if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
-                outRecordBuilder.addField(fieldName, fieldValues.get(i));
+            outRecordBuilder.reset(outRecType);
+            outputRecordTypeInfo.reset(outRecType);
+            if (inputRecordPointable == null) {
+                inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
             }
-        }
-        outRecordBuilder.write(resultOutput, true);
-    }
-
-    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
-        inputRecordVisitable.set(inputRecordPointable);
-        if (requiresCast) {
-            return castToOpenRecord();
-        }
-        return inputRecordVisitable;
-    }
-
-    private boolean hasDerivedType(IAType[] types) {
-        for (IAType type : types) {
-            if (type.getTypeTag().isDerivedType()) {
-                return true;
+            inputRecordPointable.set(inputPointable);
+            final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+            final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
+            for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+                final IVisitablePointable fieldName = fieldNames.get(i);
+                final IVisitablePointable fieldValue = fieldValues.get(i);
+                if (!PointableHelper.isEqual(fieldName, fieldToRemovePointable, stringBinaryComparator)) {
+                    int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(),
+                            fieldName.getStartOffset() + 1, fieldName.getLength() - 1);
+                    if (pos >= 0) {
+                        outRecordBuilder.addField(pos, fieldValue);
+                    } else {
+                        outRecordBuilder.addField(fieldName, fieldValue);
+                    }
+                }
             }
-        }
-        return false;
-    }
+            outRecordBuilder.write(resultOutput, true);
 
-    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
-        inputRecordVisitable.accept(castVisitor, castVisitorArg);
-        return openRecordPointable;
+        } catch (IOException e) {
+            throw HyracksDataException.create(e);
+        }
+        result.set(resultStorage);
     }
 }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
index 8b345aa700..ea4f6db37f 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameDescriptor.java
@@ -23,7 +23,9 @@ import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptor;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
 import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.TypeComputeUtils;
 import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.functions.FunctionTypeInferers;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
@@ -34,7 +36,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 @MissingNullInOutFunction
 public class RecordRenameDescriptor extends AbstractScalarFunctionDynamicDescriptor {
-
     public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
         @Override
         public IFunctionDescriptor createFunctionDescriptor() {
@@ -43,16 +44,18 @@ public class RecordRenameDescriptor extends AbstractScalarFunctionDynamicDescrip
 
         @Override
         public IFunctionTypeInferer createFunctionTypeInferer() {
-            return FunctionTypeInferers.RecordAccessorTypeInferer.INSTANCE_LAX;
+            return FunctionTypeInferers.RECORD_MODIFY_INFERER;
         }
     };
 
     private static final long serialVersionUID = 1L;
-    private ARecordType recordType;
+    private ARecordType outRecType;
+    private ARecordType inRecType;
 
     @Override
     public void setImmutableStates(Object... states) {
-        recordType = (ARecordType) states[0];
+        outRecType = TypeComputeUtils.extractRecordType((IAType) states[0]);
+        inRecType = TypeComputeUtils.extractRecordType((IAType) states[1]);
     }
 
     @Override
@@ -66,7 +69,7 @@ public class RecordRenameDescriptor extends AbstractScalarFunctionDynamicDescrip
                 for (int i = 0; i < args.length; i++) {
                     argEvals[i] = args[i].createScalarEvaluator(ctx);
                 }
-                return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], recordType);
+                return new RecordRenameEvaluator(argEvals[0], argEvals[1], argEvals[2], outRecType, inRecType);
             }
         };
     }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
index ac09ffa030..1461536f95 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/records/RecordRenameEvaluator.java
@@ -16,156 +16,105 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.runtime.evaluators.functions.records;
 
-import java.io.DataOutput;
 import java.io.IOException;
 import java.util.List;
 
-import org.apache.asterix.builders.RecordBuilder;
-import org.apache.asterix.om.pointables.ARecordVisitablePointable;
-import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
-import org.apache.asterix.om.pointables.cast.ACastVisitor;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
-import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.accessors.UTF8StringBinaryComparatorFactory;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.UTF8StringPointable;
 import org.apache.hyracks.data.std.primitive.VoidPointable;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
 import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
 
-class RecordRenameEvaluator implements IScalarEvaluator {
-
-    private final IPointable inputRecordPointable = new VoidPointable();
-    private final UTF8StringPointable oldFieldNamePointable = new UTF8StringPointable();
-    private final UTF8StringPointable newFieldNamePointable = new UTF8StringPointable();
-    private final IBinaryComparator stringBinaryComparator =
-            UTF8StringBinaryComparatorFactory.INSTANCE.createBinaryComparator();
-    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
-    private final DataOutput resultOutput = resultStorage.getDataOutput();
-    private final RecordBuilder outRecordBuilder = new RecordBuilder();
+class RecordRenameEvaluator extends AbstractRecordFunctionEvaluator {
     private final IScalarEvaluator eval0;
     private final IScalarEvaluator eval1;
     private final IScalarEvaluator eval2;
-    private final ARecordVisitablePointable openRecordPointable;
-    private ARecordVisitablePointable inputRecordVisitable;
-    private boolean requiresCast = false;
-    private ACastVisitor castVisitor;
-    private Triple<IVisitablePointable, IAType, Boolean> castVisitorArg;
+    private final IPointable oldFieldNamePointable = new VoidPointable();
 
     RecordRenameEvaluator(IScalarEvaluator eval0, IScalarEvaluator eval1, IScalarEvaluator eval2,
-            ARecordType recordType) {
+            ARecordType outRecType, ARecordType inRecType) {
+        super(outRecType, inRecType);
         this.eval0 = eval0;
         this.eval1 = eval1;
         this.eval2 = eval2;
-        openRecordPointable = new ARecordVisitablePointable(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        if (recordType != null) {
-            inputRecordVisitable = new ARecordVisitablePointable(recordType);
-            if (hasDerivedType(recordType.getFieldTypes())) {
-                requiresCast = true;
-                castVisitor = new ACastVisitor();
-                castVisitorArg =
-                        new Triple<>(openRecordPointable, openRecordPointable.getInputRecordType(), Boolean.FALSE);
-            }
-        }
     }
 
     @Override
     public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
         resultStorage.reset();
-        boolean returnNull = false;
-        eval0.evaluate(tuple, inputRecordPointable);
+        eval0.evaluate(tuple, inputPointable);
         eval1.evaluate(tuple, oldFieldNamePointable);
         eval2.evaluate(tuple, newFieldNamePointable);
-
-        if (PointableHelper.checkAndSetMissingOrNull(result, inputRecordPointable, oldFieldNamePointable,
+        if (PointableHelper.checkAndSetMissingOrNull(result, inputPointable, oldFieldNamePointable,
                 newFieldNamePointable)) {
             return;
         }
 
-        byte[] data = inputRecordPointable.getByteArray();
-        int offset = inputRecordPointable.getStartOffset();
+        // Check the type of our first argument.
+        byte[] data = inputPointable.getByteArray();
+        int offset = inputPointable.getStartOffset();
         byte typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_RECORD_TYPE_TAG) {
-            returnNull = true;
+            PointableHelper.setNull(result);
+            return;
         }
 
+        // Check the type of our second argument.
         data = oldFieldNamePointable.getByteArray();
         offset = oldFieldNamePointable.getStartOffset();
         typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            returnNull = true;
+            PointableHelper.setNull(result);
+            return;
         }
 
+        // Check the type of our third argument.
         data = newFieldNamePointable.getByteArray();
         offset = newFieldNamePointable.getStartOffset();
         typeTag = data[offset];
         if (typeTag != ATypeTag.SERIALIZED_STRING_TYPE_TAG) {
-            returnNull = true;
-        }
-        if (returnNull) {
             PointableHelper.setNull(result);
             return;
         }
-        evaluate();
-        result.set(resultStorage);
-    }
 
-    private void evaluate() throws HyracksDataException {
-        resultStorage.reset();
         try {
-            final ARecordVisitablePointable inputRecord = getInputRecordVisitablePointable();
-            buildOutputRecord(inputRecord);
+            outRecordBuilder.reset(outRecType);
+            outputRecordTypeInfo.reset(outRecType);
+            if (inputRecordPointable == null) {
+                inputRecordPointable = pointableAllocator.allocateRecordValue(inRecType);
+            }
+            inputRecordPointable.set(inputPointable);
+            final List<IVisitablePointable> fieldNames = inputRecordPointable.getFieldNames();
+            final List<IVisitablePointable> fieldValues = inputRecordPointable.getFieldValues();
+            for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
+                final IVisitablePointable fieldName = fieldNames.get(i);
+                final IVisitablePointable fieldValue = fieldValues.get(i);
+                if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
+                    addField(fieldName, fieldValue);
+                } else {
+                    addField(newFieldNamePointable, fieldValue);
+                }
+            }
+            outRecordBuilder.write(resultOutput, true);
         } catch (IOException e) {
             throw HyracksDataException.create(e);
         }
+        result.set(resultStorage);
     }
 
-    private void buildOutputRecord(ARecordVisitablePointable inputRecord) throws HyracksDataException {
-        outRecordBuilder.reset(DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
-        outRecordBuilder.init();
-        final List<IVisitablePointable> fieldNames = inputRecord.getFieldNames();
-        final List<IVisitablePointable> fieldValues = inputRecord.getFieldValues();
-        for (int i = 0, fieldCount = fieldNames.size(); i < fieldCount; i++) {
-            final IVisitablePointable fieldName = fieldNames.get(i);
-            if (!PointableHelper.isEqual(fieldName, oldFieldNamePointable, stringBinaryComparator)) {
-                outRecordBuilder.addField(fieldName, fieldValues.get(i));
-            } else {
-                outRecordBuilder.addField(newFieldNamePointable, fieldValues.get(i));
-            }
-        }
-        outRecordBuilder.write(resultOutput, true);
-    }
-
-    private ARecordVisitablePointable getInputRecordVisitablePointable() throws HyracksDataException {
-        inputRecordVisitable.set(inputRecordPointable);
-        if (requiresCast) {
-            return castToOpenRecord();
-        }
-        return inputRecordVisitable;
-    }
-
-    private boolean hasDerivedType(IAType[] types) {
-        for (IAType type : types) {
-            if (type.getTypeTag().isDerivedType()) {
-                return true;
-            }
+    private void addField(IPointable fieldName, IPointable fieldValue) throws HyracksDataException {
+        int pos = outputRecordTypeInfo.getFieldIndex(fieldName.getByteArray(), fieldName.getStartOffset() + 1,
+                fieldName.getLength() - 1);
+        if (pos >= 0) {
+            outRecordBuilder.addField(pos, fieldValue);
+        } else {
+            outRecordBuilder.addField(fieldName, fieldValue);
         }
-        return false;
     }
-
-    private ARecordVisitablePointable castToOpenRecord() throws HyracksDataException {
-        inputRecordVisitable.accept(castVisitor, castVisitorArg);
-        return openRecordPointable;
-    }
-
 }
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 f96a6e9461..5abebd2a18 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
@@ -116,6 +116,16 @@ public final class FunctionTypeInferers {
         }
     };
 
+    public static final IFunctionTypeInferer RECORD_MODIFY_INFERER = (expr, fd, context, compilerProps) -> {
+        AbstractFunctionCallExpression f = (AbstractFunctionCallExpression) expr;
+        IAType outType = (IAType) context.getType(expr);
+        IAType inType = (IAType) context.getType(f.getArguments().get(0).getValue());
+        if (inType.getTypeTag().equals(ATypeTag.ANY)) {
+            inType = DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE;
+        }
+        fd.setImmutableStates(outType, inType);
+    };
+
     public static final class CastTypeInferer implements IFunctionTypeInferer {
         @Override
         public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,