ASTERIXDB-1755: add UPSERT in SQL++.

Detailed list of changes included:
- Fix AQL RETURNING clause to use an expression instead of a query;
- Add a rule to check whether a RETURNING expression contains dataset accesses;
- Perform AST-level rewrites for INSERT/UPSERT as a whole, including the
  body query and the returning expression.

Change-Id: I02b2be7ff2653573eccb48037895f5c8c4bc8c74
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1401
Tested-by: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <jenkins@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <buyingyi@gmail.com>
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index f12a02f..ec73185 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
 import org.apache.asterix.optimizer.rules.CancelUnnestWithNestedListifyRule;
 import org.apache.asterix.optimizer.rules.CheckFilterExpressionTypeRule;
+import org.apache.asterix.optimizer.rules.CheckInsertUpsertReturningRule;
 import org.apache.asterix.optimizer.rules.ConstantFoldingRule;
 import org.apache.asterix.optimizer.rules.CountVarToCountOneRule;
 import org.apache.asterix.optimizer.rules.DisjunctivePredicateToJoinRule;
@@ -163,6 +164,7 @@
     public static final List<IAlgebraicRewriteRule> buildNormalizationRuleCollection() {
         List<IAlgebraicRewriteRule> normalization = new LinkedList<>();
         normalization.add(new ResolveVariableRule());
+        normalization.add(new CheckInsertUpsertReturningRule());
         normalization.add(new IntroduceUnnestForCollectionToSequenceRule());
         normalization.add(new EliminateSubplanRule());
         normalization.add(new EnforceOrderByAfterSubplan());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
new file mode 100644
index 0000000..d181dd3
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckInsertUpsertReturningRule.java
@@ -0,0 +1,51 @@
+/*
+ * 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.optimizer.rules;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.optimizer.rules.util.InsertUpsertCheckUtil;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * This rule checks whether a returning clause for INSERT/UPSERT contains a dataset access.
+ * NOTE: the rule should be invoked before rewriting Unnests (with dataset functions) into DataSourceScans.
+ */
+public class CheckInsertUpsertReturningRule implements IAlgebraicRewriteRule {
+
+    private boolean checked = false;
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        if (checked) {
+            return false;
+        }
+        if (InsertUpsertCheckUtil.check(opRef.getValue())) {
+            throw new CompilationException(ErrorCode.ERROR_COMPILATION_INVALID_RETURNING_EXPRESSION);
+        }
+        checked = true;
+        return false;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/InsertUpsertCheckUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/InsertUpsertCheckUtil.java
new file mode 100644
index 0000000..178e8b9
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/InsertUpsertCheckUtil.java
@@ -0,0 +1,102 @@
+/*
+ * 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.optimizer.rules.util;
+
+import java.util.List;
+
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.UnnestingFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+
+public class InsertUpsertCheckUtil {
+
+    private InsertUpsertCheckUtil() {
+    }
+
+    /**
+     *
+     * Checks the query plan rooted at <code>op</code> to see whether there is an invalid returning expression
+     * for insert/upsert, i.e., an returning expression that contains dataset accesses.
+     *
+     * @param op,
+     *            the operator in consideration
+     * @return true if the returning expression after insert/upsert is invalid; false otherwise.
+     */
+    public static boolean check(ILogicalOperator op) {
+        return checkTopDown(op, false);
+    }
+
+    // Checks the query plan rooted at <code>op</code> top down to see whether there is an invalid returning expression
+    // for insert/upsert, i.e., a returning expression that contains dataset accesses.
+    private static boolean checkTopDown(ILogicalOperator op, boolean hasSubplanAboveWithDatasetAccess) {
+        boolean metSubplanWithDataScan = hasSubplanAboveWithDatasetAccess;
+        if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+            SubplanOperator subplanOp = (SubplanOperator) op;
+            metSubplanWithDataScan = containsDatasetAccess(subplanOp);
+        }
+        if (op.getOperatorTag() == LogicalOperatorTag.INSERT_DELETE_UPSERT && metSubplanWithDataScan) {
+            return true;
+        }
+        for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {
+            if (checkTopDown(inputOpRef.getValue(), metSubplanWithDataScan)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    // Checks whether a subplan operator contains a dataset accesses in its nested pipeline.
+    private static boolean containsDatasetAccess(SubplanOperator subplanOp) {
+        List<ILogicalPlan> nestedPlans = subplanOp.getNestedPlans();
+        for (ILogicalPlan nestedPlan : nestedPlans) {
+            for (Mutable<ILogicalOperator> opRef : nestedPlan.getRoots()) {
+                if (containsDatasetAccessInternal(opRef.getValue())) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    // Checks whether a query plan rooted at <code>op</code> contains dataset accesses.
+    private static boolean containsDatasetAccessInternal(ILogicalOperator op) {
+        if (op.getOperatorTag() == LogicalOperatorTag.UNNEST) {
+            UnnestOperator unnestOp = (UnnestOperator) op;
+            ILogicalExpression unnestExpr = unnestOp.getExpressionRef().getValue();
+            UnnestingFunctionCallExpression unnestingFuncExpr = (UnnestingFunctionCallExpression) unnestExpr;
+            return unnestingFuncExpr.getFunctionIdentifier().equals(BuiltinFunctions.DATASET);
+        }
+        if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN && containsDatasetAccess((SubplanOperator) op)) {
+            return true;
+        }
+        for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
+            if (containsDatasetAccessInternal(childRef.getValue())) {
+                return true;
+            }
+        }
+        return false;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
index de17448..7d1ce7d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/AqlExpressionToPlanTranslator.java
@@ -169,13 +169,16 @@
 
     @Override
     protected boolean expressionNeedsNoNesting(Expression expr) {
-        boolean noForFLWOR = false;
-        // No nesting is needed for a FLWOR expression without a FOR clause.
-        if (expr.getKind() == Kind.FLWOGR_EXPRESSION) {
+        boolean isFLWOGR = expr.getKind() == Kind.FLWOGR_EXPRESSION;
+        boolean letOnly = true;
+        // No nesting is needed for a FLWOR expression that only has LETs and RETURN.
+        if (isFLWOGR) {
             FLWOGRExpression flwor = (FLWOGRExpression) expr;
-            noForFLWOR = flwor.noForClause();
+            for (Clause clause : flwor.getClauseList()) {
+                letOnly &= clause.getClauseType() == Clause.ClauseType.LET_CLAUSE;
+            }
         }
-        return noForFLWOR || super.expressionNeedsNoNesting(expr);
+        return (isFLWOGR && letOnly) || super.expressionNeedsNoNesting(expr);
     }
 
     private Pair<ILogicalOperator, LogicalVariable> produceFlworPlan(boolean noForClause, boolean isTop,
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
index 98c717c..c21b5db 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/CompiledStatements.java
@@ -295,17 +295,17 @@
         private final String datasetName;
         private final Query query;
         private final int varCounter;
-        VariableExpr var;
-        Query returnQuery;
+        private final VariableExpr var;
+        private final Expression returnExpression;
 
         public CompiledInsertStatement(String dataverseName, String datasetName, Query query, int varCounter,
-                VariableExpr var, Query returnQuery) {
+                VariableExpr var, Expression returnExpression) {
             this.dataverseName = dataverseName;
             this.datasetName = datasetName;
             this.query = query;
             this.varCounter = varCounter;
             this.var = var;
-            this.returnQuery = returnQuery;
+            this.returnExpression = returnExpression;
         }
 
         @Override
@@ -330,8 +330,8 @@
             return var;
         }
 
-        public Query getReturnQuery() {
-            return returnQuery;
+        public Expression getReturnExpression() {
+            return returnExpression;
         }
 
         @Override
@@ -343,8 +343,8 @@
     public static class CompiledUpsertStatement extends CompiledInsertStatement {
 
         public CompiledUpsertStatement(String dataverseName, String datasetName, Query query, int varCounter,
-                VariableExpr var, Query returnQuery) {
-            super(dataverseName, datasetName, query, varCounter, var, returnQuery);
+                VariableExpr var, Expression returnExpression) {
+            super(dataverseName, datasetName, query, varCounter, var, returnExpression);
         }
 
         @Override
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index fe5e590..cf722cc 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -71,6 +71,7 @@
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
+import org.apache.asterix.lang.sqlpp.clause.Projection;
 import org.apache.asterix.metadata.MetadataException;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSource;
@@ -300,18 +301,12 @@
         Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, base);
         ArrayList<Mutable<ILogicalOperator>> globalPlanRoots = new ArrayList<>();
         ILogicalOperator topOp = p.first;
+        List<LogicalVariable> liveVars = new ArrayList<>();
+        VariableUtilities.getLiveVariables(topOp, liveVars);
+        LogicalVariable unnestVar = liveVars.get(0);
+        LogicalVariable resVar = unnestVar;
 
         if (outputDatasetName == null) {
-            LogicalVariable resVar;
-            if (topOp instanceof ProjectOperator) {
-                resVar = ((ProjectOperator) topOp).getVariables().get(0);
-            } else if (topOp instanceof AssignOperator) {
-                resVar = ((AssignOperator) topOp).getVariables().get(0);
-            } else if (topOp instanceof AggregateOperator) {
-                resVar = ((AggregateOperator) topOp).getVariables().get(0);
-            } else {
-                throw new AlgebricksException("Invalid returning query");
-            }
             FileSplit outputFileSplit = metadataProvider.getOutputFile();
             if (outputFileSplit == null) {
                 outputFileSplit = getDefaultOutputFileLocation();
@@ -333,10 +328,6 @@
                 topOp.getAnnotations().put("output-record-type", outputRecordType);
             }
         } else {
-            ProjectOperator project = (ProjectOperator) topOp;
-            LogicalVariable unnestVar = project.getVariables().get(0);
-            LogicalVariable resVar = project.getVariables().get(0);
-
             /**
              * add the collection-to-sequence right before the project,
              * because dataset only accept non-collection records
@@ -350,9 +341,10 @@
                     new MutableObject<>(new ScalarFunctionCallExpression(
                             FunctionUtil.getFunctionInfo(BuiltinFunctions.COLLECTION_TO_SEQUENCE),
                             new MutableObject<>(new VariableReferenceExpression(resVar)))));
-            assignCollectionToSequence.getInputs().add(new MutableObject<>(project.getInputs().get(0).getValue()));
-            project.getInputs().get(0).setValue(assignCollectionToSequence);
-            project.getVariables().set(0, seqVar);
+            assignCollectionToSequence.getInputs().add(new MutableObject<>(topOp.getInputs().get(0).getValue()));
+            topOp.getInputs().get(0).setValue(assignCollectionToSequence);
+            ProjectOperator projectOperator = (ProjectOperator) topOp;
+            projectOperator.getVariables().set(0, seqVar);
             resVar = seqVar;
             DatasetDataSource targetDatasource = validateDatasetInfo(metadataProvider, stmt.getDataverseName(),
                     stmt.getDatasetName());
@@ -391,10 +383,10 @@
 
                 additionalFilteringAssign = new AssignOperator(additionalFilteringVars,
                         additionalFilteringAssignExpressions);
-                additionalFilteringAssign.getInputs().add(new MutableObject<>(project));
+                additionalFilteringAssign.getInputs().add(new MutableObject<>(topOp));
                 assign.getInputs().add(new MutableObject<>(additionalFilteringAssign));
             } else {
-                assign.getInputs().add(new MutableObject<>(project));
+                assign.getInputs().add(new MutableObject<>(topOp));
             }
 
             Mutable<ILogicalExpression> varRef = new MutableObject<>(new VariableReferenceExpression(resVar));
@@ -406,7 +398,7 @@
                     break;
                 case Statement.Kind.UPSERT:
                     leafOperator = translateUpsert(targetDatasource, varRef, varRefsForLoading,
-                            additionalFilteringExpressions, assign, additionalFilteringField, unnestVar, project, exprs,
+                            additionalFilteringExpressions, assign, additionalFilteringField, unnestVar, topOp, exprs,
                             resVar, additionalFilteringAssign, stmt);
                     break;
                 case Statement.Kind.DELETE:
@@ -419,7 +411,7 @@
                     break;
                 case Statement.Kind.SUBSCRIBE_FEED:
                     leafOperator = translateSubscribeFeed((CompiledSubscribeFeedStatement) stmt, targetDatasource,
-                            unnestVar, project, exprs, resVar, varRefsForLoading, varRef, assign,
+                            unnestVar, topOp, exprs, resVar, varRefsForLoading, varRef, assign,
                             additionalFilteringField, additionalFilteringAssign, additionalFilteringExpressions);
                     break;
                 default:
@@ -463,7 +455,7 @@
     }
 
     private ILogicalOperator translateSubscribeFeed(CompiledSubscribeFeedStatement sfs,
-            DatasetDataSource targetDatasource, LogicalVariable unnestVar, ProjectOperator project,
+            DatasetDataSource targetDatasource, LogicalVariable unnestVar, ILogicalOperator topOp,
             ArrayList<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar,
             List<Mutable<ILogicalExpression>> varRefsForLoading, Mutable<ILogicalExpression> varRef,
             ILogicalOperator assign, List<String> additionalFilteringField, AssignOperator additionalFilteringAssign,
@@ -478,6 +470,7 @@
         boolean isChangeFeed = ExternalDataUtils.isChangeFeed(feed.getAdapterConfiguration());
         boolean isUpsertFeed = ExternalDataUtils.isUpsertFeed(feed.getAdapterConfiguration());
 
+        ProjectOperator project = (ProjectOperator) topOp;
         if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
             metaAndKeysVars = new ArrayList<>();
             metaAndKeysExprs = new ArrayList<>();
@@ -546,8 +539,8 @@
         }
         if (targetDatasource.getDataset().hasMetaPart() || isChangeFeed) {
             metaAndKeysAssign = new AssignOperator(metaAndKeysVars, metaAndKeysExprs);
-            metaAndKeysAssign.getInputs().add(project.getInputs().get(0));
-            project.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
+            metaAndKeysAssign.getInputs().add(topOp.getInputs().get(0));
+            topOp.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
         }
         feedModificationOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         ILogicalOperator leafOperator = new DelegateOperator(new CommitOperator(true));
@@ -558,18 +551,20 @@
     private ILogicalOperator translateUpsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
             List<Mutable<ILogicalExpression>> varRefsForLoading,
             List<Mutable<ILogicalExpression>> additionalFilteringExpressions, ILogicalOperator assign,
-            List<String> additionalFilteringField, LogicalVariable unnestVar, ProjectOperator project,
+            List<String> additionalFilteringField, LogicalVariable unnestVar, ILogicalOperator topOp,
             List<Mutable<ILogicalExpression>> exprs, LogicalVariable resVar, AssignOperator additionalFilteringAssign,
             ICompiledDmlStatement stmt) throws AlgebricksException {
-        if (!targetDatasource.getDataset().allow(project, Dataset.OP_UPSERT)) {
+        if (!targetDatasource.getDataset().allow(topOp, Dataset.OP_UPSERT)) {
             throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
                     + ": upsert into dataset is not supported on Datasets with Meta records");
         }
+        ProjectOperator project = (ProjectOperator) topOp;
         CompiledUpsertStatement compiledUpsert = (CompiledUpsertStatement) stmt;
+        Expression returnExpression = compiledUpsert.getReturnExpression();
         InsertDeleteUpsertOperator upsertOp;
-        ILogicalOperator leafOperator;
+        ILogicalOperator rootOperator;
         if (targetDatasource.getDataset().hasMetaPart()) {
-            if (compiledUpsert.getReturnQuery() != null) {
+            if (returnExpression != null) {
                 throw new AlgebricksException("Returning not allowed on datasets with Meta records");
 
             }
@@ -628,12 +623,9 @@
                 upsertOp.getInputs().add(assign.getInputs().get(0));
             }
             metaAndKeysAssign = new AssignOperator(metaAndKeysVars, metaAndKeysExprs);
-            metaAndKeysAssign.getInputs().add(project.getInputs().get(0));
-            project.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
+            metaAndKeysAssign.getInputs().add(topOp.getInputs().get(0));
+            topOp.getInputs().set(0, new MutableObject<>(metaAndKeysAssign));
             upsertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
-            leafOperator = new DelegateOperator(new CommitOperator(true));
-            leafOperator.getInputs().add(new MutableObject<>(upsertOp));
-
         } else {
             upsertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef, varRefsForLoading,
                     InsertDeleteUpsertOperator.Kind.UPSERT, false);
@@ -647,46 +639,13 @@
                 upsertOp.setPrevFilterVar(context.newVar());
                 upsertOp.setPrevFilterType(recordType.getFieldType(additionalFilteringField.get(0)));
             }
-
-            if (compiledUpsert.getReturnQuery() != null) {
-                leafOperator = createReturningQuery(compiledUpsert, upsertOp);
-
-            } else {
-                leafOperator = new DelegateOperator(new CommitOperator(true));
-                leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(upsertOp));
-            }
         }
-        return leafOperator;
+        rootOperator = new DelegateOperator(new CommitOperator(returnExpression == null));
+        rootOperator.getInputs().add(new MutableObject<>(upsertOp));
 
-    }
+        // Compiles the return expression.
+        return processReturningExpression(rootOperator, upsertOp, compiledUpsert);
 
-    private ILogicalOperator createReturningQuery(CompiledInsertStatement compiledInsert,
-            InsertDeleteUpsertOperator insertOp) throws AlgebricksException {
-        //Make the id of the insert var point to the record variable
-        context.newVar(compiledInsert.getVar());
-        context.setVar(compiledInsert.getVar(),
-                ((VariableReferenceExpression) insertOp.getPayloadExpression().getValue()).getVariableReference());
-        // context
-
-        ILogicalPlan planAfterInsert = translate(compiledInsert.getReturnQuery(), null, null, insertOp);
-
-        ILogicalOperator finalRoot = planAfterInsert.getRoots().get(0).getValue();
-        ILogicalOperator op;
-        for (op = finalRoot;; op = op.getInputs().get(0).getValue()) {
-            if (op.getInputs().size() != 1) {
-                throw new AlgebricksException("Cannot have a multi-branch returning query");
-            }
-            if (op.getInputs().get(0).getValue() instanceof InsertDeleteUpsertOperator) {
-                break;
-            }
-        }
-
-        op.getInputs().clear();
-        ILogicalOperator leafOperator = new DelegateOperator(new CommitOperator(false));
-        leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
-        op.getInputs().add(new MutableObject<>(leafOperator));
-        leafOperator = finalRoot;
-        return leafOperator;
     }
 
     private ILogicalOperator translateInsert(DatasetDataSource targetDatasource, Mutable<ILogicalExpression> varRef,
@@ -697,20 +656,52 @@
             throw new AlgebricksException(targetDatasource.getDataset().getDatasetName()
                     + ": insert into dataset is not supported on Datasets with Meta records");
         }
-        ILogicalOperator leafOperator;
+        // Adds the insert operator.
         InsertDeleteUpsertOperator insertOp = new InsertDeleteUpsertOperator(targetDatasource, varRef,
                 varRefsForLoading, InsertDeleteUpsertOperator.Kind.INSERT, false);
         insertOp.setAdditionalFilteringExpressions(additionalFilteringExpressions);
         insertOp.getInputs().add(new MutableObject<>(assign));
-        CompiledInsertStatement compiledInsert = (CompiledInsertStatement) stmt;
-        if (compiledInsert.getReturnQuery() != null) {
-            leafOperator = createReturningQuery(compiledInsert, insertOp);
 
-        } else {
-            leafOperator = new DelegateOperator(new CommitOperator(true));
-            leafOperator.getInputs().add(new MutableObject<ILogicalOperator>(insertOp));
+        // Adds the commit operator.
+        CompiledInsertStatement compiledInsert = (CompiledInsertStatement) stmt;
+        Expression returnExpression = compiledInsert.getReturnExpression();
+        ILogicalOperator rootOperator = new DelegateOperator(
+                new CommitOperator(returnExpression == null ? true : false));
+        rootOperator.getInputs().add(new MutableObject<>(insertOp));
+
+        // Compiles the return expression.
+        return processReturningExpression(rootOperator, insertOp, compiledInsert);
+    }
+
+    // Stitches the translated operators for the returning expression into the query plan.
+    private ILogicalOperator processReturningExpression(ILogicalOperator inputOperator,
+            InsertDeleteUpsertOperator insertOp, CompiledInsertStatement compiledInsert) throws AlgebricksException {
+        Expression returnExpression = compiledInsert.getReturnExpression();
+        if (returnExpression == null) {
+            return inputOperator;
         }
-        return leafOperator;
+        ILogicalOperator rootOperator = inputOperator;
+
+        //Makes the id of the insert var point to the record variable.
+        context.newVar(compiledInsert.getVar());
+        context.setVar(compiledInsert.getVar(),
+                ((VariableReferenceExpression) insertOp.getPayloadExpression().getValue()).getVariableReference());
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p = langExprToAlgExpression(returnExpression,
+                    new MutableObject<>(rootOperator));
+
+        // Adds an assign operator for the returning expression.
+        LogicalVariable resultVar = context.newVar();
+        AssignOperator assignOperator = new AssignOperator(resultVar, new MutableObject<>(p.first));
+        assignOperator.getInputs().add(p.second);
+
+        // Adds a distribute result operator.
+        List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
+        expressions.add(new MutableObject<>(new VariableReferenceExpression(resultVar)));
+        ResultSetSinkId rssId = new ResultSetSinkId(metadataProvider.getResultSetId());
+        ResultSetDataSink sink = new ResultSetDataSink(rssId, null);
+        rootOperator = new DistributeResultOperator(expressions, sink);
+        rootOperator.getInputs().add(new MutableObject<>(assignOperator));
+        return rootOperator;
     }
 
     private DatasetDataSource validateDatasetInfo(MetadataProvider metadataProvider, String dataverseName,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 643ba71..f9794e4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -21,7 +21,6 @@
 
 import java.io.IOException;
 import java.io.PrintWriter;
-import java.rmi.Remote;
 import java.rmi.RemoteException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -53,6 +52,7 @@
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.Query;
@@ -145,8 +145,8 @@
         }
     }
 
-    public Pair<Query, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions, MetadataProvider metadataProvider,
-            Query q, SessionConfig conf) throws AsterixException {
+    public Pair<IReturningStatement, Integer> reWriteQuery(List<FunctionDecl> declaredFunctions,
+            MetadataProvider metadataProvider, IReturningStatement q, SessionConfig conf) throws AsterixException {
         if (q == null) {
             return null;
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 4fab8d7..906525c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -90,6 +90,7 @@
 import org.apache.asterix.lang.common.base.IRewriterFactory;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.expression.TypeExpression;
 import org.apache.asterix.lang.common.statement.CompactStatement;
 import org.apache.asterix.lang.common.statement.ConnectFeedStatement;
@@ -340,7 +341,7 @@
                         break;
                     case Statement.Kind.INSERT:
                     case Statement.Kind.UPSERT:
-                        if (((InsertStatement) stmt).getReturnQuery() != null) {
+                        if (((InsertStatement) stmt).getReturnExpression() != null) {
                             metadataProvider.setResultSetId(new ResultSetId(resultSetIdCounter++));
                             metadataProvider.setResultAsyncMode(resultDelivery == ResultDelivery.ASYNC
                                     || resultDelivery == ResultDelivery.DEFERRED);
@@ -1859,51 +1860,21 @@
             IStatementExecutor.Stats stats, boolean compileOnly) throws Exception {
 
         InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
-
         String dataverseName = getActiveDataverse(stmtInsertUpsert.getDataverseName());
         Query query = stmtInsertUpsert.getQuery();
         MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
         boolean bActiveTxn = true;
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
-        if (stmtInsertUpsert.getReturnQuery() != null) {
-            if (!stmtInsertUpsert.getReturnQuery().getDatasets().isEmpty()) {
-                throw new AsterixException("Cannot use datasets in an insert returning query");
-            }
-            // returnQuery Rewriting (happens under the same ongoing metadata transaction)
-            Pair<Query, Integer> rewrittenReturnQuery = apiFramework.reWriteQuery(declaredFunctions, metadataProvider,
-                    stmtInsertUpsert.getReturnQuery(), sessionConfig);
-
-            stmtInsertUpsert.getQuery().setVarCounter(rewrittenReturnQuery.first.getVarCounter());
-            stmtInsertUpsert.setRewrittenReturnQuery(rewrittenReturnQuery.first);
-            stmtInsertUpsert.addToVarCounter(rewrittenReturnQuery.second);
-        }
-
         MetadataLockManager.INSTANCE.insertDeleteUpsertBegin(dataverseName,
                 dataverseName + "." + stmtInsertUpsert.getDatasetName(), query.getDataverses(), query.getDatasets());
         try {
             metadataProvider.setWriteTransaction(true);
-            CompiledInsertStatement clfrqs = null;
-            switch (stmtInsertUpsert.getKind()) {
-                case Statement.Kind.INSERT:
-                    clfrqs = new CompiledInsertStatement(dataverseName, stmtInsertUpsert.getDatasetName().getValue(),
-                            query, stmtInsertUpsert.getVarCounter(), stmtInsertUpsert.getVar(),
-                            stmtInsertUpsert.getReturnQuery());
-                    break;
-                case Statement.Kind.UPSERT:
-                    clfrqs = new CompiledUpsertStatement(dataverseName, stmtInsertUpsert.getDatasetName().getValue(),
-                            query, stmtInsertUpsert.getVarCounter(), stmtInsertUpsert.getVar(),
-                            stmtInsertUpsert.getReturnQuery());
-                    break;
-                default:
-                    throw new AlgebricksException("Unsupported statement type " + stmtInsertUpsert.getKind());
-            }
-            JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, clfrqs);
-
+            JobSpecification jobSpec = rewriteCompileInsertUpsert(hcc, metadataProvider, stmtInsertUpsert);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
 
             if (jobSpec != null && !compileOnly) {
-                if (stmtInsertUpsert.getReturnQuery() != null) {
+                if (stmtInsertUpsert.getReturnExpression() != null) {
                     handleQueryResult(metadataProvider, hcc, hdc, jobSpec, resultDelivery, stats);
                 } else {
                     JobUtils.runJob(hcc, jobSpec, true);
@@ -1964,18 +1935,46 @@
     public JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector,
             MetadataProvider metadataProvider, Query query,
             ICompiledDmlStatement stmt)
-            throws AsterixException, RemoteException, AlgebricksException, ACIDException {
+            throws RemoteException, AlgebricksException, ACIDException {
 
         // Query Rewriting (happens under the same ongoing metadata transaction)
-        Pair<Query, Integer> reWrittenQuery = apiFramework.reWriteQuery(declaredFunctions, metadataProvider, query,
-                sessionConfig);
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
+                metadataProvider, query, sessionConfig);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
-        JobSpecification spec = apiFramework.compileQuery(clusterInfoCollector, metadataProvider, reWrittenQuery.first,
-                reWrittenQuery.second, stmt == null ? null : stmt.getDatasetName(), sessionConfig, stmt);
+        return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
+                rewrittenResult.second, stmt == null ? null : stmt.getDatasetName(), sessionConfig, stmt);
+    }
 
-        return spec;
+    private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
+            MetadataProvider metadataProvider, InsertStatement insertUpsert)
+            throws RemoteException, AlgebricksException, ACIDException {
 
+        // Insert/upsert statement rewriting (happens under the same ongoing metadata transaction)
+        Pair<IReturningStatement, Integer> rewrittenResult = apiFramework.reWriteQuery(declaredFunctions,
+                metadataProvider, insertUpsert, sessionConfig);
+
+        InsertStatement rewrittenInsertUpsert = (InsertStatement) rewrittenResult.first;
+        String dataverseName = getActiveDataverse(rewrittenInsertUpsert.getDataverseName());
+        String datasetName = rewrittenInsertUpsert.getDatasetName().getValue();
+        CompiledInsertStatement clfrqs;
+        switch (insertUpsert.getKind()) {
+            case Statement.Kind.INSERT:
+                clfrqs = new CompiledInsertStatement(dataverseName, datasetName, rewrittenInsertUpsert.getQuery(),
+                        rewrittenInsertUpsert.getVarCounter(), rewrittenInsertUpsert.getVar(),
+                        rewrittenInsertUpsert.getReturnExpression());
+                break;
+            case Statement.Kind.UPSERT:
+                clfrqs = new CompiledUpsertStatement(dataverseName, datasetName, rewrittenInsertUpsert.getQuery(),
+                        rewrittenInsertUpsert.getVarCounter(), rewrittenInsertUpsert.getVar(),
+                        rewrittenInsertUpsert.getReturnExpression());
+                break;
+            default:
+                throw new AlgebricksException("Unsupported statement type " + rewrittenInsertUpsert.getKind());
+        }
+        // Insert/upsert statement compilation (happens under the same ongoing metadata transaction)
+        return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, rewrittenInsertUpsert.getQuery(),
+                rewrittenResult.second, datasetName, sessionConfig, clfrqs);
     }
 
     protected void handleCreateFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 5ede01c..2b9b357 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -185,7 +185,8 @@
     private void rewrite(IQueryRewriter rewriter, List<FunctionDecl> declaredFunctions, Query topExpr,
             MetadataProvider metadataProvider, LangRewritingContext context) throws AsterixException {
         PA.invokeMethod(rewriter,
-                "setup(java.util.List, org.apache.asterix.lang.common.statement.Query, org.apache.asterix.metadata.declared.MetadataProvider, "
+                "setup(java.util.List, org.apache.asterix.lang.common.base.IReturningStatement, "
+                        + "org.apache.asterix.metadata.declared.MetadataProvider, "
                         + "org.apache.asterix.lang.common.rewrites.LangRewritingContext)",
                 declaredFunctions, topExpr, metadataProvider, context);
         PA.invokeMethod(rewriter, "inlineColumnAlias()");
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-return-custom-result.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-return-custom-result.plan
index 391b248..ef733cb 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-return-custom-result.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/insert-return-custom-result.plan
@@ -1,17 +1,20 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+      -- SUBPLAN  |PARTITIONED|
+              {
+                -- ASSIGN  |LOCAL|
+                  -- ASSIGN  |LOCAL|
+                    -- IN_MEMORY_STABLE_SORT [$$16(ASC)]  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+              }
+        -- COMMIT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- COMMIT  |PARTITIONED|
-                -- STREAM_PROJECT  |PARTITIONED|
-                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- INSERT_DELETE  |PARTITIONED|
-                      -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+              -- INSERT_DELETE  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$16]  |PARTITIONED|
+                  -- ASSIGN  |UNPARTITIONED|
+                    -- STREAM_PROJECT  |UNPARTITIONED|
+                      -- UNNEST  |UNPARTITIONED|
                         -- ASSIGN  |UNPARTITIONED|
-                          -- STREAM_PROJECT  |UNPARTITIONED|
-                            -- UNNEST  |UNPARTITIONED|
-                              -- ASSIGN  |UNPARTITIONED|
-                                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.1.ddl.aql
new file mode 100644
index 0000000..9e77afb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.1.ddl.aql
@@ -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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: uuid,
+  message-text: string
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid autogenerated;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.3.query.aql
new file mode 100644
index 0000000..8e9cff5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.3.query.aql
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use dataverse test;
+
+insert into dataset TweetMessageuuids (
+     { "message-text":"hello"}
+) returning 1;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-udf/insert-returning-udf.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-udf/insert-returning-udf.1.ddl.aql
new file mode 100644
index 0000000..f579a73
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-udf/insert-returning-udf.1.ddl.aql
@@ -0,0 +1,42 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: uuid,
+  message-text: string
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid autogenerated;
+
+create function project($foo){
+    let $result := $foo.message-text
+    return $result
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-udf/insert-returning-udf.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-udf/insert-returning-udf.3.query.aql
new file mode 100644
index 0000000..ecbb50d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/insert-returning-udf/insert-returning-udf.3.query.aql
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use dataverse test;
+
+insert into dataset TweetMessageuuids as $message (
+{ "message-text":"hello"}
+) returning project($message);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/upsert-return-custom-result/upsert-return-custom-result.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/upsert-return-custom-result/upsert-return-custom-result.3.query.aql
index 9abf667..45a9595 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/upsert-return-custom-result/upsert-return-custom-result.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/dml/upsert-return-custom-result/upsert-return-custom-result.3.query.aql
@@ -23,7 +23,7 @@
  * Date            : Mar 2015
  */
 
- use dataverse test;
+use dataverse test;
 
 upsert into dataset TweetMessageuuids as $a (
 let $x :=
@@ -33,6 +33,7 @@
 {"tweetid":4,"message-text":"what","location":create-point(3.0,6.0)},
 {"tweetid":5,"message-text":"good","location":create-point(5.0,6.0)}]
 for $y in $x
+where $y.tweetid=5
 return $y
 ) returning
 let $x := create-circle($a.location,5.0)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-return-records/insert-return-records.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-return-records/insert-return-records.1.ddl.sqlpp
new file mode 100644
index 0000000..5c66655
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-return-records/insert-return-records.1.ddl.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.
+ */
+/*
+ * Test case Name  : insert-return-records
+ * Description     : Check records returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: int,
+  `message-text`: string
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-return-records/insert-return-records.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-return-records/insert-return-records.3.query.sqlpp
new file mode 100644
index 0000000..09a3b59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-return-records/insert-return-records.3.query.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.
+ */
+/*
+ * Test case Name  : insert-return-records
+ * Description     : Check records returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use test;
+
+insert into TweetMessageuuids as message (
+    [
+     { "tweetid":1,"message-text":"hello"},
+     {"tweetid":2,"message-text":"goodbye"},
+     {"tweetid":3,"message-text":"the end"},
+     {"tweetid":4,"message-text":"what"},
+     {"tweetid":5,"message-text":"good"}
+    ]
+) returning message;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.1.ddl.sqlpp
new file mode 100644
index 0000000..f20fcbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.1.ddl.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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: uuid,
+  `message-text`: string
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid autogenerated;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.3.query.sqlpp
new file mode 100644
index 0000000..c8d14e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname-implicit/insert-returning-fieldname-implicit.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use test;
+
+insert into TweetMessageuuids (
+     { "message-text":"hello"}
+) returning `message-text`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname/insert-returning-fieldname.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname/insert-returning-fieldname.1.ddl.sqlpp
new file mode 100644
index 0000000..f20fcbe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname/insert-returning-fieldname.1.ddl.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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: uuid,
+  `message-text`: string
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid autogenerated;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname/insert-returning-fieldname.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname/insert-returning-fieldname.3.query.sqlpp
new file mode 100644
index 0000000..d56a954
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-fieldname/insert-returning-fieldname.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use test;
+
+insert into TweetMessageuuids as message (
+     { "message-text":"hello"}
+) returning message.`message-text`;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-udf/insert-returning-udf.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-udf/insert-returning-udf.1.ddl.sqlpp
new file mode 100644
index 0000000..885c347
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-udf/insert-returning-udf.1.ddl.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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: uuid,
+  `message-text`: string
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid autogenerated;
+
+create function project(foo){
+    foo.`message-text`
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-udf/insert-returning-udf.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-udf/insert-returning-udf.3.query.sqlpp
new file mode 100644
index 0000000..2d2d70f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-returning-udf/insert-returning-udf.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : insert-returning-fieldname
+ * Description     : Check fields returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+use test;
+
+insert into TweetMessageuuids as message (
+     { "message-text":"hello"}
+) returning project(message);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-with-bad-return/insert-with-bad-return.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-with-bad-return/insert-with-bad-return.1.ddl.sqlpp
new file mode 100644
index 0000000..5c66655
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-with-bad-return/insert-with-bad-return.1.ddl.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.
+ */
+/*
+ * Test case Name  : insert-return-records
+ * Description     : Check records returned on insert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: int,
+  `message-text`: string
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-with-bad-return/insert-with-bad-return.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-with-bad-return/insert-with-bad-return.3.query.sqlpp
new file mode 100644
index 0000000..aa91b42
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/insert-with-bad-return/insert-with-bad-return.3.query.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.
+ */
+/*
+ * Test case Name  : insert-with-bad-return
+ * Description     : Throw an error
+ * Expected Result : Error
+ * Date            : Oct 2016
+ */
+
+use test;
+
+insert into TweetMessageuuids as message (
+    [
+       { "tweetid":1,"message-text":"hello"},
+       {"tweetid":2,"message-text":"goodbye"},
+       {"tweetid":3,"message-text":"the end"},
+       {"tweetid":4,"message-text":"what"},
+       {"tweetid":5,"message-text":"good"}
+    ]
+) returning
+(
+ from TweetMessageuuids as result
+ where result.`message-text`=message
+ select value result
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-dataset-with-meta/upsert-dataset-with-meta.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-dataset-with-meta/upsert-dataset-with-meta.1.ddl.sqlpp
new file mode 100644
index 0000000..d77f990
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-dataset-with-meta/upsert-dataset-with-meta.1.ddl.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.
+ */
+/*
+ * Description  : Upsert into a dataset with meta type
+ * Expected Res : Failure
+ * Date         : 15th Mar 2016
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type RecordType as open{
+};
+
+create type MetaType as open{
+id:string
+};
+
+create dataset DatasetWithMeta(RecordType) with meta(MetaType)primary key meta().id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-dataset-with-meta/upsert-dataset-with-meta.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-dataset-with-meta/upsert-dataset-with-meta.2.update.sqlpp
new file mode 100644
index 0000000..af7ab4b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-dataset-with-meta/upsert-dataset-with-meta.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Upsert into a dataset with meta type
+ * Expected Res : Failure
+ * Date         : 15th Mar 2016
+ */
+
+use test;
+
+upsert into DatasetWithMeta (
+{"id": 2, "name": "Person Two", "hobbies": {{"Rock", "Jazz"}}}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-return-custom-result/upsert-return-custom-result.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-return-custom-result/upsert-return-custom-result.1.ddl.sqlpp
new file mode 100644
index 0000000..40768db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-return-custom-result/upsert-return-custom-result.1.ddl.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.
+ */
+/*
+ * Test case Name  : upsert-return-custom-result
+ * Description     : Check records returned on upsert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TweetMessageTypeuuid as closed {
+  tweetid: int,
+  `message-text`: string,
+  location:point
+}
+
+create dataset TweetMessageuuids(TweetMessageTypeuuid)
+primary key tweetid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-return-custom-result/upsert-return-custom-result.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-return-custom-result/upsert-return-custom-result.3.query.sqlpp
new file mode 100644
index 0000000..b504205
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/dml/upsert-return-custom-result/upsert-return-custom-result.3.query.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Test case Name  : upsert-return-custom-result
+ * Description     : Check records returned on upsert
+ * Expected Result : Success
+ * Date            : Mar 2015
+ */
+use test;
+
+upsert into TweetMessageuuids as a (
+   with x as
+    [
+      { "tweetid":1,"message-text":"hello","location":create_point(6.0,6.0)},
+      {"tweetid":2,"message-text":"goodbye","location":create_point(1.0,1.0)},
+      {"tweetid":3,"message-text":"the end","location":create_point(6.0,3.0)},
+      {"tweetid":4,"message-text":"what","location":create_point(3.0,6.0)},
+      {"tweetid":5,"message-text":"good","location":create_point(5.0,6.0)}
+     ]
+   from x as y
+   where y.tweetid=5
+   select value y
+)
+returning
+(
+  {
+    "x": create_circle(a.location,5.0),
+    "tweetid": a.tweetid
+  }
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.1.ddl.sqlpp
new file mode 100644
index 0000000..d426687
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+drop dataverse experiments if exists;
+create dataverse experiments;
+use experiments;
+
+create type TwitterUser if not exists as open{
+    `screen-name`: string,
+    friends_count: int32,
+    name: string,
+    followers_count: int32
+};
+
+create dataset TwitterUsers(TwitterUser) primary key `screen-name`;
+
+create feed UserFeed using socket_adapter
+(
+    ("sockets"="127.0.0.1:10001"),
+    ("address-type"="IP"),
+    ("type-name"="TwitterUser"),
+    ("format"="adm"),
+    ("upsert-feed"="true")
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
new file mode 100644
index 0000000..32d8389
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+use experiments;
+set `wait-for-completion-feed` "false";
+
+connect feed UserFeed to dataset TwitterUsers;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.3.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.3.server.sqlpp
new file mode 100644
index 0000000..3da77f7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.3.server.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+start client 10001 file-client localhost ../asterix-app/data/tinysocial/twu_update.adm 500 50 1000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.4.sleep.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.4.sleep.sqlpp
new file mode 100644
index 0000000..99ad0d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.4.sleep.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+10000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
new file mode 100644
index 0000000..8e2dff8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.5.update.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+use experiments;
+disconnect feed UserFeed from dataset TwitterUsers;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.6.query.sqlpp
new file mode 100644
index 0000000..77ee294
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.6.query.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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+use experiments;
+
+select value x
+from TwitterUsers x
+order by x.`screen-name`;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.7.server.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.7.server.sqlpp
new file mode 100644
index 0000000..d331474
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.7.server.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+
+stop 10001
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.8.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.8.ddl.sqlpp
new file mode 100644
index 0000000..8ba3fcd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/upsert-feed/upsert-feed.8.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Create a feed with upsert option. Push record twice
+ * with minor changes. The updated record will not cause duplicate
+ * key exception and the data will be updated.
+ * Expected Res : Success
+ * Date         : 13th Aug 2016
+ */
+use experiments;
+drop dataverse experiments;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.sqlpp
new file mode 100644
index 0000000..12ca0ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type OrderOpenType as open {
+  o_orderkey: int64
+}
+
+create dataset OrdersOpen(OrderOpenType)
+primary key o_orderkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.update.sqlpp
new file mode 100644
index 0000000..072b093
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use test;
+
+insert into OrdersOpen (
+  {"o_orderkey": 1,
+  "o_custkey": 1}
+)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.sqlpp
new file mode 100644
index 0000000..0309109
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use test;
+
+create index idx_Orders_Custkey on
+OrdersOpen(o_custkey:int32?) enforced;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.update.sqlpp
new file mode 100644
index 0000000..8e73c27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use test;
+
+delete from OrdersOpen v
+where v.o_orderkey = 1;
+
+insert into OrdersOpen (
+  {"o_orderkey": 1,
+   "o_custkey": 2}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
new file mode 100644
index 0000000..a1c1923
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+USE test;
+
+WITH l AS (SELECT VALUE o.o_orderKey
+           FROM OrdersOpen o
+           WHERE o.o_custkey >= -1
+          )
+SELECT VALUE COLL_COUNT((
+  SELECT VALUE l
+  FROM l
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.sqlpp
new file mode 100644
index 0000000..8ed8c77
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type OrderOpenType as open {
+  o_orderkey: int64
+}
+
+create dataset OrdersOpen(OrderOpenType)
+primary key o_orderkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.update.sqlpp
new file mode 100644
index 0000000..17f1479
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use test;
+
+insert into OrdersOpen (
+  {"o_orderkey": 1,
+   "o_custkey": 1}
+)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.sqlpp
new file mode 100644
index 0000000..d7885cd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use test;
+
+create index idx_Orders_Custkey on
+OrdersOpen(o_custkey:int32?) enforced;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.update.sqlpp
new file mode 100644
index 0000000..0cab666
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.update.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use test;
+
+upsert into OrdersOpen (
+  {"o_orderkey": 1,
+   "o_custkey": 2}
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
new file mode 100644
index 0000000..0e6f4e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+USE test;
+
+WITH l AS (SELECT VALUE o.o_orderKey
+           FROM OrdersOpen o
+           WHERE o.o_custkey >= -1
+          )
+SELECT VALUE COLL_COUNT((
+  SELECT VALUE l
+  FROM l
+));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.1.ddl.sqlpp
new file mode 100644
index 0000000..751ba27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.1.ddl.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Test filters with upsert pipeline
+ * Expected Res : Success
+ * Date         : 13th Jan 2016
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type FacebookMessageType as closed {
+        `message-id`: int64,
+        `author-id`: int64,
+        `in-response-to`: int64?,
+        `sender-location`: point?,
+        message: string,
+        `send-time`: datetime
+}
+
+create dataset FacebookMessages(FacebookMessageType)
+primary key `message-id`;
+
+create dataset FilteredFacebookMessages(FacebookMessageType)
+primary key `message-id` with filter on `send-time`;
+
+create index AutherIdx on FilteredFacebookMessages(`author-id`);
+create index MessageIdx on FilteredFacebookMessages(message);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.2.update.sqlpp
new file mode 100644
index 0000000..008c03c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.2.update.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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 test;
+
+load dataset FilteredFacebookMessages using localfs
+(("path"="asterix_nc1://data/fbm-with-send-time.adm"),("format"="adm"));
+
+load dataset FacebookMessages using localfs
+(("path"="asterix_nc1://data/more-fbm-with-send-time.adm"),("format"="adm"));
+
+upsert into FilteredFacebookMessages(
+ FROM FacebookMessages AS x
+ SELECT VALUE x
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.3.query.sqlpp
new file mode 100644
index 0000000..08b9d51
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/filtered-dataset/filtered-dataset.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test;
+
+FROM FilteredFacebookMessages m
+WHERE m.`send-time` > datetime("2012-08-20T10:10:00")
+SELECT VALUE m;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.1.ddl.sqlpp
new file mode 100644
index 0000000..1eaeba3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.1.ddl.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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a foreign datatype
+ * Expected Res : Success
+ * Date         : Aug 18th 2016
+ */
+
+drop dataverse b if exists;
+drop dataverse a if exists;
+create dataverse a;
+create dataverse b;
+use a;
+
+create type TypeA as closed{
+id:int32,
+age:int32
+};
+
+use b;
+
+create dataset UpsertTo(a.TypeA) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.2.update.sqlpp
new file mode 100644
index 0000000..588c261
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.2.update.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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a foreign datatype
+ * Expected Res : Success
+ * Date         : Aug 18th 2016
+ */
+
+// upsert UpsertFrom into UpsertTo
+use b;
+
+upsert into UpsertTo(
+{"id":1,"age":7}
+);
+
+upsert into UpsertTo(
+{"id":2,"age":8}
+);
+
+upsert into UpsertTo(
+{"id":1,"age":9}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.3.query.sqlpp
new file mode 100644
index 0000000..e53f140
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a foreign datatype
+ * Expected Res : Success
+ * Date         : Aug 18th 2016
+ */
+
+use b;
+
+SELECT VALUE x
+FROM UpsertTo x
+ORDER BY x.id;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.4.ddl.sqlpp
new file mode 100644
index 0000000..71eaed1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/issue1587-foreignDataType/issue1587-foreignDataType.4.ddl.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a foreign datatype
+ * Expected Res : Success
+ * Date         : Aug 18th 2016
+ */
+drop dataverse b;
+drop dataverse a;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.1.ddl.sqlpp
new file mode 100644
index 0000000..f9649c7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.1.ddl.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.
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+
+create dataset UpsertTo(MyRecord)
+  primary key id;
+
+ create dataset UpsertFrom(MyRecord)
+ primary key id;
+
+create index btree_index on UpsertTo(kwds);
+create index rtree_index on UpsertTo(point) type rtree;
+create index inverted_index on UpsertTo(kwds) type keyword;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.2.update.sqlpp
new file mode 100644
index 0000000..4a316a8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.2.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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 test;
+
+load dataset UpsertTo
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm"));
+
+load dataset UpsertFrom
+using localfs
+(("path"="asterix_nc1://data/spatial/moreSpatialData.json"),("format"="adm"));
+
+
+upsert into UpsertTo(
+    select value x
+    from UpsertFrom x
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.3.query.sqlpp
new file mode 100644
index 0000000..ad45ca2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/multiple-secondaries/multiple-secondaries.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test;
+
+SELECT o.id id
+FROM UpsertTo o
+WHERE spatial_intersect(o.point, create_polygon([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]))
+ORDER BY o.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.1.ddl.sqlpp
new file mode 100644
index 0000000..5d0b5e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.1.ddl.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+use test;
+
+create type OrderTypetmp as closed {
+  o_orderkey: int64,
+  o_custkey: int64,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int64,
+  o_comment: string
+}
+
+create type OrderType as closed {
+nested : OrderTypetmp
+}
+
+create dataset UpsertTo(OrderTypetmp)
+  primary key o_orderkey;
+
+create dataset UpsertFrom(OrderTypetmp)
+  primary key o_orderkey;
+
+  create dataset Orders(OrderType)
+  primary key nested.o_orderkey;
+
+create index idx_Orders_Custkey on Orders(nested.o_custkey);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.2.update.sqlpp
new file mode 100644
index 0000000..01a883b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.2.update.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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 test;
+
+load dataset UpsertTo
+using localfs
+(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),
+("format"="delimited-text"),
+("delimiter"="|")) pre-sorted;
+
+load dataset UpsertFrom
+using localfs
+(("path"="asterix_nc1://data/tpch0.001/other-orders.tbl"),
+("format"="delimited-text"),
+("delimiter"="|")) pre-sorted;
+
+insert into Orders
+(
+  select c as nested
+  from UpsertTo c
+);
+
+upsert into Orders
+(
+  select c as nested
+  from UpsertFrom c
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.3.query.sqlpp
new file mode 100644
index 0000000..03357e5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nested-index/nested-index.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test;
+
+select o.nested.o_orderkey, o.nested.o_custkey
+from Orders o
+where o.nested.o_custkey < 60
+order by o.nested.o_orderkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.1.ddl.sqlpp
new file mode 100644
index 0000000..6451ed4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.1.ddl.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.
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type AddressType as open {
+  number: int64,
+  street: string,
+  city: string
+}
+
+create type CustomerType as open {
+  cid: int64,
+  name: string,
+  age: int64?,
+  address: AddressType?,
+  interests: {{string}},
+  children: [ { name: string, age: int64? } ]
+}
+
+create dataset Customers(CustomerType) primary key cid;
+create dataset MoreCustomers(CustomerType) primary key cid;
+create index age_index on Customers(age);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.2.update.sqlpp
new file mode 100644
index 0000000..f3cf58a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 test;
+
+load dataset Customers
+using localfs
+(("path"="asterix_nc1://data/semistructured/tiny01/customer.adm"),("format"="adm"));
+
+load dataset MoreCustomers
+using localfs
+(("path"="asterix_nc1://data/semistructured/tiny01/more-customer.adm"),("format"="adm"));
+
+upsert into Customers(
+  from MoreCustomers x
+  select value x
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.3.query.sqlpp
new file mode 100644
index 0000000..600faa3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/nullable-index/nullable-index.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test;
+
+select value c
+from Customers c
+where c.age < 20
+order by c.cid;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.1.ddl.sqlpp
new file mode 100644
index 0000000..e964d0b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.1.ddl.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 test if exists;
+create dataverse test;
+use test;
+
+create type OrderType as closed {
+  o_orderkey: int64,
+  o_custkey: int64,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int64,
+  o_comment: string
+}
+
+create type OrderOpenType as open {
+  o_orderkey: int64,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: int64,
+  o_comment: string
+}
+
+create dataset Orders(OrderType)
+  primary key o_orderkey;
+
+  create dataset OtherOrders(OrderType)
+  primary key o_orderkey;
+
+create dataset OrdersOpen(OrderOpenType)
+primary key o_orderkey;
+
+create index idx_Orders_Custkey on OrdersOpen(o_custkey:int32?) enforced;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.2.update.sqlpp
new file mode 100644
index 0000000..bebc8d0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.2.update.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.
+ */
+use test;
+
+load dataset Orders
+using localfs
+(("path"="asterix_nc1://data/tpch0.001/orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+load dataset OtherOrders
+using localfs
+(("path"="asterix_nc1://data/tpch0.001/other-orders.tbl"),("format"="delimited-text"),("delimiter"="|")) pre-sorted;
+
+insert into OrdersOpen (
+  select value x
+  from OtherOrders x
+);
+
+upsert into OrdersOpen (
+  select value x
+  from Orders x
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.3.query.sqlpp
new file mode 100644
index 0000000..ce78713
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/open-index/open-index.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test;
+
+select o.o_orderkey, o.o_custkey
+from OrdersOpen o
+where o.o_custkey > 40
+order by o.o_orderkey;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.1.ddl.sqlpp
new file mode 100644
index 0000000..c970044
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.1.ddl.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.
+ */
+/*
+ * Description  : Upsert into a dataset which doesn't have any secondary indexes
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestType as closed{
+id:int32,
+age:int32,
+name:string,
+salary:double
+};
+
+create dataset UpsertTo(TestType) primary key id;
+create dataset UpsertFrom(TestType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.2.update.sqlpp
new file mode 100644
index 0000000..64239cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.2.update.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.
+ */
+/*
+ * Description  : Upsert into a dataset which doesn't have any secondary indexes
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+use test;
+// load first dataset
+load dataset UpsertTo using
+localfs(("format"="delimited-text"),
+  ("path"="asterix_nc1://data/upsert/raw-data/overlapping.data"),
+  ("delimiter"=","));
+// load second dataset
+load dataset UpsertFrom using
+localfs(("format"="delimited-text"),
+  ("path"="asterix_nc1://data/upsert/raw-data/test-data.txt,asterix_nc1://data/upsert/raw-data/more-data.txt"),
+  ("delimiter"=","));
+
+// upsert UpsertFrom into UpsertTo
+upsert into UpsertTo(
+ from UpsertFrom x
+ select value x
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.3.query.sqlpp
new file mode 100644
index 0000000..d8a2308
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-index/primary-index.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Upsert into a dataset which doesn't have any secondary indexes
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+USE test;
+
+FROM UpsertTo x
+SELECT VALUE x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.1.ddl.sqlpp
new file mode 100644
index 0000000..b232d8f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.1.ddl.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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a b-tree secondary index
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestType as closed{
+id:int32,
+age:int32,
+name:string,
+salary:double
+};
+
+create dataset UpsertTo(TestType) primary key id;
+create index ageindex on UpsertTo(age);
+create dataset UpsertFrom(TestType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.2.update.sqlpp
new file mode 100644
index 0000000..2f8b334
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.2.update.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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a b-tree secondary index
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+use test;
+// load first dataset
+load dataset UpsertTo using
+localfs(("format"="delimited-text"),
+  ("path"="asterix_nc1://data/upsert/raw-data/overlapping.data"),
+  ("delimiter"=","));
+// load second dataset
+load dataset UpsertFrom using
+localfs(("format"="delimited-text"),
+  ("path"="asterix_nc1://data/upsert/raw-data/test-data.txt,asterix_nc1://data/upsert/raw-data/more-data.txt"),
+  ("delimiter"=","));
+
+// upsert UpsertFrom into UpsertTo
+upsert into UpsertTo(
+ select value x
+ from UpsertFrom x
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.3.query.sqlpp
new file mode 100644
index 0000000..102aada
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-btree/primary-secondary-btree.3.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Upsert into a dataset which has a b-tree secondary index
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+ // So far this one doesn't use the btree index, need another query
+use test;
+
+select value x
+from UpsertTo x
+where x.age > 5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.1.ddl.sqlpp
new file mode 100644
index 0000000..e8e421a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.1.ddl.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.
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type DBLPType as closed {
+  id: int64,
+  dblpid: string,
+  title: string,
+  authors: string,
+  misc: string
+}
+
+create dataset UpsertToDBLP(DBLPType)
+  primary key id;
+
+create dataset UpsertFromDBLP(DBLPType)
+  primary key id;
+
+create index keyword_index on UpsertToDBLP(title) type keyword;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.2.update.sqlpp
new file mode 100644
index 0000000..29a7396
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 test;
+
+load dataset UpsertToDBLP
+using localfs
+(("path"="asterix_nc1://data/dblp-small/dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+load dataset UpsertFromDBLP
+using localfs
+(("path"="asterix_nc1://data/dblp-small/more-dblp-small-id.txt"),("format"="delimited-text"),("delimiter"=":"));
+
+upsert into UpsertToDBLP(
+ select value x
+ from UpsertFromDBLP x
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.3.query.sqlpp
new file mode 100644
index 0000000..31e091c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-inverted/primary-secondary-inverted.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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 test;
+
+SELECT VALUE o
+FROM UpsertToDBLP o
+WHERE contains(o.title, "SQL")
+ORDER BY o.id;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.1.ddl.sqlpp
new file mode 100644
index 0000000..01f6725
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.1.ddl.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.
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+}
+
+create dataset UpsertTo(MyRecord)
+ primary key id;
+
+create dataset UpsertFrom(MyRecord)
+ primary key id;
+
+create index rtree_index_point on UpsertTo(point) type rtree;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.2.update.sqlpp
new file mode 100644
index 0000000..b0a3bbc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.2.update.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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 test;
+
+load dataset UpsertTo
+using localfs
+(("path"="asterix_nc1://data/spatial/spatialData.json"),("format"="adm"));
+
+load dataset UpsertFrom
+using localfs
+(("path"="asterix_nc1://data/spatial/moreSpatialData.json"),("format"="adm"));
+
+upsert into UpsertTo(
+  from UpsertFrom x
+  select value x
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.3.query.sqlpp
new file mode 100644
index 0000000..42d4040
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/primary-secondary-rtree/primary-secondary-rtree.3.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * 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 test;
+
+select o.id
+from UpsertTo o
+where spatial_intersect(o.point, create_polygon([4.0,1.0,4.0,4.0,12.0,4.0,12.0,1.0]));
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.1.ddl.sqlpp
new file mode 100644
index 0000000..23d3b4d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.1.ddl.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.
+ */
+/*
+ * Description  : Upsert into a dataset with self read
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type TestType as closed{
+id:int32,
+age:int32,
+name:string,
+salary:double
+};
+
+create dataset UpsertTo(TestType) primary key id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.2.update.sqlpp
new file mode 100644
index 0000000..0f071ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.2.update.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.
+ */
+/*
+ * Description  : Upsert into a dataset with self read
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+use test;
+// load first dataset
+load dataset UpsertTo using
+localfs(("format"="delimited-text"),
+  ("path"="asterix_nc1://data/upsert/raw-data/overlapping.data"),
+  ("delimiter"=","));
+
+// upsert from UpsertTo into UpsertTo
+upsert into UpsertTo(
+  select x.id as id, x.age+1 as age, x.name as name, x.salary*1.1 as salary
+  from UpsertTo x
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.3.query.sqlpp
new file mode 100644
index 0000000..e57ea39
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/upsert/upsert-with-self-read/upsert-with-self-read.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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.
+ */
+/*
+ * Description  : Upsert into a dataset with self read
+ * Expected Res : Success
+ * Date         : Sep 15th 2015
+ */
+
+use test;
+
+select value x
+from UpsertTo x;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.ddl.sqlpp
new file mode 100644
index 0000000..bb96a11
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.1.ddl.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * 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 channels if exists;
+create dataverse channels;
+use channels;
+
+create type userLocation as closed
+{ userId: int, roomNumber: int }
+
+create dataset userLocations(userLocation)
+primary key userId;
+
+create function currentOccupancy(room)
+{
+  (
+    SELECT VALUE location.userId
+    FROM userLocations location
+    WHERE location.roomNumber = room
+  )
+};
+
+create type subscription as { id:uuid, param0:int }
+
+create dataset subscriptions(subscription)
+primary key id autogenerated;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.2.update.sqlpp
new file mode 100644
index 0000000..be4cb91
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.2.update.sqlpp
@@ -0,0 +1,56 @@
+/*
+ * 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 channels;
+
+upsert into userLocations(
+[
+{"userId":1, "roomNumber":123}
+,
+{"userId":2, "roomNumber":123}
+,
+{"userId":3, "roomNumber":123}
+,
+{"userId":4, "roomNumber":123}
+,
+{"userId":5, "roomNumber":350}
+,
+{"userId":6, "roomNumber":350}
+,
+{"userId":7, "roomNumber":350}
+,
+{"userId":8, "roomNumber":350}
+,
+{"userId":9, "roomNumber":350}
+,
+{"userId":10,"roomNumber":210}
+,
+{"userId":11,"roomNumber":210}
+,
+{"userId":12,"roomNumber":210}
+,
+{"userId":13,"roomNumber":210}
+,
+{"userId":14,"roomNumber":210}
+]
+);
+
+insert into subscriptions(
+{"param0":123}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.3.query.sqlpp
new file mode 100644
index 0000000..f206462
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/user-defined-functions/query-ASTERIXDB-1298/query-ASTERIXDB-1298.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 channels;
+
+SELECT VALUE result
+FROM subscriptions sub,
+     currentOccupancy(sub.param0) result
+ORDER BY result;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/upsert-return-custom-result/upsert-return-custom-result.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/upsert-return-custom-result/upsert-return-custom-result.1.adm
index 8706beb..78ac3c4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/upsert-return-custom-result/upsert-return-custom-result.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/dml/upsert-return-custom-result/upsert-return-custom-result.1.adm
@@ -1,5 +1 @@
-{ "x": circle("6.0,6.0 5.0"), "tweetid": 1 }
-{ "x": circle("1.0,1.0 5.0"), "tweetid": 2 }
-{ "x": circle("3.0,6.0 5.0"), "tweetid": 4 }
-{ "x": circle("6.0,3.0 5.0"), "tweetid": 3 }
-{ "x": circle("5.0,6.0 5.0"), "tweetid": 5 }
\ No newline at end of file
+{ "x": circle("5.0,6.0 5.0"), "tweetid": 5 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index e4d4ee3..3f33680 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -1721,9 +1721,20 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-udf">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-fieldname-implicit">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+        <expected-error>Need a binding variable for the enclosed expression</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
       <compilation-unit name="insert-with-bad-return">
         <output-dir compare="Text">insert-with-bad-return</output-dir>
-        <expected-error>Error: Cannot use datasets in an insert returning query</expected-error>
+        <expected-error>A returning expression cannot contain dataset access</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 8c92b6b..ff1c530 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -1749,6 +1749,33 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="dml">
+      <compilation-unit name="insert-return-records">
+        <output-dir compare="Text">insert-return-records</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-fieldname">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-udf">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-returning-fieldname-implicit">
+        <output-dir compare="Text">insert-returning-fieldname</output-dir>
+        <expected-error>Need an alias for the enclosed expression</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="insert-with-bad-return">
+        <output-dir compare="Text">insert-with-bad-return</output-dir>
+        <expected-error>A returning expression cannot contain dataset access</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
       <compilation-unit name="load-with-index">
         <output-dir compare="Text">load-with-index</output-dir>
       </compilation-unit>
@@ -1948,6 +1975,17 @@
         <output-dir compare="Text">delete-multi-statement</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-dataset-with-meta">
+        <output-dir compare="Text">upsert-dataset-with-meta</output-dir>
+        <expected-error>upsert into dataset is not supported on Datasets with Meta records</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="dml">
+      <compilation-unit name="upsert-return-custom-result">
+        <output-dir compare="Text">upsert-return-custom-result</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="employee">
     <test-case FilePath="employee">
@@ -3371,6 +3409,18 @@
         </compilation-unit>
       </test-case>
     </test-group>
+    <test-group name="open-index-enforced/type-checking">
+      <test-case FilePath="open-index-enforced/type-checking">
+        <compilation-unit name="enforced-type-delete">
+          <output-dir compare="Text">enforced-type-delete</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/type-checking">
+        <compilation-unit name="enforced-type-upsert">
+          <output-dir compare="Text">enforced-type-upsert</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
   </test-group>
   <test-group name="nested-open-index">
     <test-group name="nested-open-index/index-join">
@@ -7290,6 +7340,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="user-defined-functions">
+      <compilation-unit name="query-ASTERIXDB-1298">
+        <output-dir compare="Text">query-ASTERIXDB-1298</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="user-defined-functions">
       <compilation-unit name="query-ASTERIXDB-1652">
         <expected-error>In function call "test.length(...)", the dataverse "test" cannot be found!</expected-error>
         <output-dir compare="Text">query-ASTERIXDB-1652-2</output-dir>
@@ -7698,6 +7753,11 @@
         <output-dir compare="Text">issue_230_feeds</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="feeds">
+      <compilation-unit name="upsert-feed">
+        <output-dir compare="Text">upsert-feed</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="hdfs">
     <test-case FilePath="hdfs">
@@ -8478,4 +8538,61 @@
       </compilation-unit>
     </test-case>
   </test-group>
+  <test-group name="upsert">
+    <test-case FilePath="upsert">
+      <compilation-unit name="filtered-dataset">
+        <output-dir compare="Text">filtered-dataset</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="issue1587-foreignDataType">
+        <output-dir compare="Text">issue1587-foreignDataType</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="nested-index">
+        <output-dir compare="Text">nested-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-secondary-rtree">
+        <output-dir compare="Text">primary-secondary-rtree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="upsert-with-self-read">
+        <output-dir compare="Text">upsert-with-self-read</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="nullable-index">
+        <output-dir compare="Text">nullable-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="open-index">
+        <output-dir compare="Text">open-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-index">
+        <output-dir compare="Text">primary-index</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-secondary-btree">
+        <output-dir compare="Text">primary-secondary-btree</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="primary-secondary-inverted">
+        <output-dir compare="Text">primary-secondary-inverted</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="upsert">
+      <compilation-unit name="multiple-secondaries">
+        <output-dir compare="Text">multiple-secondaries</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
 </test-suite>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 743d5fb..d5dfd87 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -58,6 +58,7 @@
     public static final int ERROR_COMPILATION_INVALID_EXPRESSION = 1006;
     public static final int ERROR_COMPILATION_INVALID_PARAMETER_NUMBER = 1007;
     public static final int ERROR_COMPILATION_DUPLICATE_FIELD_NAME = 1008;
+    public static final int ERROR_COMPILATION_INVALID_RETURNING_EXPRESSION = 1009;
 
     private static final String ERROR_MESSAGE_ID_CONFLICT = "Two Extensions share the same Id: %1$s";
     private static final String ERROR_MESSAGE_COMPONENT_CONFLICT = "Extension Conflict between %1$s and %2$s both "
@@ -83,6 +84,8 @@
             + " its %2$s input parameter to be a %3$s expression, but the actual expression is %4$s";
     private static final String ERROR_MESSAGE_INVALID_PARAMETER_NUMBER = "Invalid parameter number: function %1$s "
             + "cannot take %2$s parameters";
+    private static final String ERROR_MESSAGE_INVALID_RETURNING_EXPRESSION = "A returning expression cannot"
+            + " contain dataset access";
 
     private static Map<Integer, String> errorMessageMap = new HashMap<>();
 
@@ -109,6 +112,7 @@
         errorMessageMap.put(ERROR_COMPILATION_INVALID_EXPRESSION, ERROR_MESSAGE_INVALID_EXPRESSION);
         errorMessageMap.put(ERROR_COMPILATION_INVALID_PARAMETER_NUMBER, ERROR_MESSAGE_INVALID_PARAMETER_NUMBER);
         errorMessageMap.put(ERROR_COMPILATION_DUPLICATE_FIELD_NAME, ERROR_MESSAGE_DUPLICATE_FIELD);
+        errorMessageMap.put(ERROR_COMPILATION_INVALID_RETURNING_EXPRESSION, ERROR_MESSAGE_INVALID_RETURNING_EXPRESSION);
 
         // lifecycle management errors
         errorMessageMap.put(ERROR_EXTENSION_ID_CONFLICT, ERROR_MESSAGE_ID_CONFLICT);
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
index cea3ad7..d70f399 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/rewrites/AqlQueryRewriter.java
@@ -31,6 +31,7 @@
 import org.apache.asterix.lang.aql.parser.AQLParserFactory;
 import org.apache.asterix.lang.aql.parser.FunctionParser;
 import org.apache.asterix.lang.aql.rewrites.visitor.AqlBuiltinFunctionRewriteVisitor;
+import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.util.CommonFunctionMapUtil;
 import org.apache.asterix.lang.aql.visitor.AQLInlineUdfsVisitor;
 import org.apache.asterix.lang.aql.visitor.base.IAQLVisitor;
@@ -44,7 +45,6 @@
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
@@ -53,59 +53,59 @@
 class AqlQueryRewriter implements IQueryRewriter {
 
     private final FunctionParser functionParser = new FunctionParser(new AQLParserFactory());
-    private Query topExpr;
+    private IReturningStatement topStatement;
     private List<FunctionDecl> declaredFunctions;
     private LangRewritingContext context;
     private MetadataProvider metadataProvider;
 
-    private void setup(List<FunctionDecl> declaredFunctions, Query topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context) {
-        this.topExpr = topExpr;
+    private void setup(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
+            MetadataProvider metadataProvider, LangRewritingContext context) {
+        this.topStatement = topStatement;
         this.context = context;
         this.declaredFunctions = declaredFunctions;
         this.metadataProvider = metadataProvider;
     }
 
     @Override
-    public void rewrite(List<FunctionDecl> declaredFunctions, Query topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context) throws AsterixException {
-        setup(declaredFunctions, topExpr, metadataProvider, context);
-        if (topExpr.isTopLevel()) {
+    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
+            MetadataProvider metadataProvider, LangRewritingContext context) throws AsterixException {
+        setup(declaredFunctions, topStatement, metadataProvider, context);
+        if (topStatement.isTopLevel()) {
             wrapInLets();
         }
         inlineDeclaredUdfs();
         rewriteFunctionName();
-        topExpr.setVarCounter(context.getVarCounter());
+        topStatement.setVarCounter(context.getVarCounter());
     }
 
     private void wrapInLets() {
         // If the top expression of the main statement is not a FLWOR, it wraps
         // it into a let clause.
-        if (topExpr == null) {
+        if (topStatement == null) {
             return;
         }
-        Expression body = topExpr.getBody();
+        Expression body = topStatement.getBody();
         if (body.getKind() != Kind.FLWOGR_EXPRESSION) {
             VarIdentifier var = context.newVariable();
             VariableExpr v = new VariableExpr(var);
             LetClause c1 = new LetClause(v, body);
-            ArrayList<Clause> clauseList = new ArrayList<Clause>(1);
+            ArrayList<Clause> clauseList = new ArrayList<>(1);
             clauseList.add(c1);
             FLWOGRExpression newBody = new FLWOGRExpression(clauseList, new VariableExpr(var));
-            topExpr.setBody(newBody);
+            topStatement.setBody(newBody);
         }
     }
 
     private void rewriteFunctionName() throws AsterixException {
-        if (topExpr == null) {
+        if (topStatement == null) {
             return;
         }
         AqlBuiltinFunctionRewriteVisitor visitor = new AqlBuiltinFunctionRewriteVisitor();
-        topExpr.accept(visitor, null);
+        topStatement.accept(visitor, null);
     }
 
     private void inlineDeclaredUdfs() throws AsterixException {
-        if (topExpr == null) {
+        if (topStatement == null) {
             return;
         }
         List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
@@ -113,15 +113,17 @@
             funIds.add(fdecl.getSignature());
         }
 
-        List<FunctionDecl> storedFunctionDecls = FunctionUtil.retrieveUsedStoredFunctions(metadataProvider,
-                topExpr.getBody(), funIds, null,
-                expr -> getFunctionCalls(expr), func -> functionParser.getFunctionDecl(func),
-                signature -> CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(signature));
-        declaredFunctions.addAll(storedFunctionDecls);
+        List<FunctionDecl> storedFunctionDecls = new ArrayList<>();
+        for (Expression topLevelExpr : topStatement.getDirectlyEnclosedExpressions()) {
+            storedFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr, funIds,
+                    null, expr -> getFunctionCalls(expr), func -> functionParser.getFunctionDecl(func),
+                    signature -> CommonFunctionMapUtil.normalizeBuiltinFunctionSignature(signature)));
+            declaredFunctions.addAll(storedFunctionDecls);
+        }
         if (!declaredFunctions.isEmpty()) {
-            AQLInlineUdfsVisitor visitor =
-                    new AQLInlineUdfsVisitor(context, new AQLRewriterFactory(), declaredFunctions, metadataProvider);
-            while (topExpr.accept(visitor, declaredFunctions)) {
+            AQLInlineUdfsVisitor visitor = new AQLInlineUdfsVisitor(context, new AQLRewriterFactory(),
+                    declaredFunctions, metadataProvider);
+            while (topStatement.accept(visitor, declaredFunctions)) {
                 // loop until no more changes
             }
         }
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/AQLFormatPrintUtil.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/AQLFormatPrintUtil.java
index bd5ea2d..fe6c5ac 100644
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/AQLFormatPrintUtil.java
+++ b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/AQLFormatPrintUtil.java
@@ -20,6 +20,7 @@
 
 import java.io.ByteArrayOutputStream;
 import java.io.PrintWriter;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.AsterixException;
@@ -42,11 +43,23 @@
         }
     }
 
-    public static String toString(List<Statement> exprs) throws AsterixException {
+    /**
+     * @param expr
+     *            a language expression.
+     * @return a formatted string of a language expression.
+     * @throws AsterixException
+     */
+    public static String toString(ILangExpression expr) throws AsterixException {
+        List<ILangExpression> exprs = new ArrayList<>();
+        exprs.add(expr);
+        return toString(exprs);
+    }
+
+    public static String toString(List<ILangExpression> exprs) throws AsterixException {
         ByteArrayOutputStream bos = new ByteArrayOutputStream();
         PrintWriter output = new PrintWriter(bos);
         AQLFormatPrintVisitor visitor = new AQLFormatPrintVisitor(output);
-        for (Statement expr : exprs) {
+        for (ILangExpression expr : exprs) {
             expr.accept(visitor, 0);
         }
         output.close();
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 463805b..c43ff66 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -50,8 +50,10 @@
 import org.apache.asterix.lang.aql.expression.FLWOGRExpression;
 import org.apache.asterix.lang.aql.expression.UnionExpr;
 import org.apache.asterix.lang.aql.util.RangeMapBuilder;
+import org.apache.asterix.lang.aql.util.AQLFormatPrintUtil;
 import org.apache.asterix.lang.common.base.Clause;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.Statement;
@@ -190,6 +192,18 @@
         return s.substring(1).trim();
     }
 
+    private static void checkBindingVariable(Expression returnExpression, VariableExpr var,
+                                             ILangExpression bodyExpression) throws ParseException {
+       if (returnExpression != null && var == null) {
+         try {
+           throw new ParseException("Need a binding variable for the enclosed expression: " +
+                                       AQLFormatPrintUtil.toString(bodyExpression));
+         } catch (AsterixException e){
+           throw new ParseException(e.getLocalizedMessage());
+         }
+       }
+    }
+
     private static IRecordFieldDataGen parseFieldDataGen(String hint) throws ParseException {
       IRecordFieldDataGen rfdg = null;
       String splits[] = hint.split(" +");
@@ -294,6 +308,7 @@
     | stmt = WriteStatement()
     | stmt = SetStatement()
     | stmt = InsertStatement()
+    | stmt = UpsertStatement()
     | stmt = DeleteStatement()
     | stmt = UpdateStatement()
     | stmt = FeedStatement()
@@ -926,28 +941,49 @@
 
 InsertStatement InsertStatement() throws ParseException:
 {
-  VariableExpr var = null;
   Pair<Identifier,Identifier> nameComponents = null;
+  VariableExpr var = null;
   Query query;
-  Query returnQuery = null;
-  boolean upsert = false;
+  Expression returnExpression = null;
 }
 {
-  (<INSERT>|<UPSERT>{ upsert = true; }) <INTO> <DATASET> nameComponents = QualifiedName()
+  <INSERT> <INTO> <DATASET> nameComponents = QualifiedName()
   (<AS> var = Variable())?
   {
     if(var != null){
       getCurrentScope().addNewVarSymbolToScope(var.getVar());
     }
   }
-  query = Query() ( <RETURNING> returnQuery = Query())?
+  query = Query() ( <RETURNING> returnExpression = Expression())?
     {
+      checkBindingVariable(returnExpression, var, query);
       query.setTopLevel(true);
-      if(upsert){
-        return new UpsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var, returnQuery);
-      } else{
-        return new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var, returnQuery);
-      }
+      return new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
+                                 returnExpression);
+    }
+}
+
+UpsertStatement UpsertStatement() throws ParseException:
+{
+  Pair<Identifier,Identifier> nameComponents = null;
+  VariableExpr var = null;
+  Query query;
+  Expression returnExpression = null;
+}
+{
+  <UPSERT> <INTO> <DATASET> nameComponents = QualifiedName()
+  (<AS> var = Variable())?
+  {
+    if(var != null){
+      getCurrentScope().addNewVarSymbolToScope(var.getVar());
+    }
+  }
+  query = Query() ( <RETURNING> returnExpression = Expression())?
+    {
+      checkBindingVariable(returnExpression, var, query);
+      query.setTopLevel(true);
+      return new UpsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
+                                 returnExpression);
     }
 }
 
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
index 4eacdb9..d77592a 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IQueryRewriter.java
@@ -23,7 +23,6 @@
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 
 public interface IQueryRewriter {
@@ -40,6 +39,6 @@
      * @param context,
      *            manages ids of variables and guarantees uniqueness of variables.
      */
-    public void rewrite(List<FunctionDecl> declaredFunctions, Query topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context) throws AsterixException;
+    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr,
+            MetadataProvider metadataProvider, LangRewritingContext context) throws AsterixException;
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IReturningStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IReturningStatement.java
new file mode 100644
index 0000000..02e5267
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/base/IReturningStatement.java
@@ -0,0 +1,67 @@
+/*
+ * 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.lang.common.base;
+
+import java.util.List;
+
+/**
+ * Interface for statements that can return tuples to users,
+ * e.g., QUERY, INSERT, and UPSERT.
+ */
+public interface IReturningStatement extends Statement {
+
+    /**
+     * @return the current variable counter, i.e., the largest used variable id.
+     */
+    int getVarCounter();
+
+    /**
+     * Sets the variable counter to the input value.
+     *
+     * @param varCounter,
+     *            the largest occupied variable id.
+     */
+    void setVarCounter(int varCounter);
+
+    /**
+     * Is the statement a top-level statement or a subquery?
+     *
+     * @return true if yes, false otherwise.
+     */
+    boolean isTopLevel();
+
+    /**
+     * @return directly enclosed top-level expressions within the statement.
+     */
+    List<Expression> getDirectlyEnclosedExpressions();
+
+    /**
+     * @return the main body expression of the statement.
+     */
+    Expression getBody();
+
+    /**
+     * Sets the main body expression of the statement.
+     *
+     * @param expr,
+     *            the main body expression.
+     */
+    void setBody(Expression expr);
+}
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
index 987c3d9..6ddf071 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/InsertStatement.java
@@ -18,30 +18,35 @@
  */
 package org.apache.asterix.lang.common.statement;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.commons.lang3.ObjectUtils;
 
-public class InsertStatement implements Statement {
+public class InsertStatement implements IReturningStatement {
 
     private final Identifier dataverseName;
     private final Identifier datasetName;
     private final Query query;
-    private int varCounter;
     private final VariableExpr var;
-    private Query returnQuery;
+    private Expression returnExpression;
+    private int varCounter;
 
     public InsertStatement(Identifier dataverseName, Identifier datasetName, Query query, int varCounter,
-            VariableExpr var, Query returnQuery) {
+            VariableExpr var, Expression returnExpression) {
         this.dataverseName = dataverseName;
         this.datasetName = datasetName;
         this.query = query;
         this.varCounter = varCounter;
         this.var = var;
-        this.returnQuery = returnQuery;
+        this.returnExpression = returnExpression;
     }
 
     @Override
@@ -61,24 +66,51 @@
         return query;
     }
 
-    public void addToVarCounter(int addition) {
-        varCounter += addition;
-    }
-
+    @Override
     public int getVarCounter() {
         return varCounter;
     }
 
+    @Override
+    public void setVarCounter(int varCounter) {
+        this.varCounter = varCounter;
+    }
+
+    @Override
+    public List<Expression> getDirectlyEnclosedExpressions() {
+        List<Expression> topLevelExpressions = new ArrayList<>();
+        topLevelExpressions.add(query.getBody());
+        if (returnExpression != null) {
+            topLevelExpressions.add(returnExpression);
+        }
+        return topLevelExpressions;
+    }
+
+    @Override
+    public boolean isTopLevel() {
+        return true;
+    }
+
+    @Override
+    public Expression getBody() {
+        return query.getBody();
+    }
+
+    @Override
+    public void setBody(Expression body) {
+        query.setBody(body);
+    }
+
     public VariableExpr getVar() {
         return var;
     }
 
-    public Query getReturnQuery() {
-        return returnQuery;
+    public Expression getReturnExpression() {
+        return returnExpression;
     }
 
-    public void setRewrittenReturnQuery(Query rewrittenReturnQuery) {
-        this.returnQuery = rewrittenReturnQuery;
+    public void setReturnExpression(Expression expr) {
+        this.returnExpression = expr;
     }
 
     @Override
@@ -88,7 +120,7 @@
 
     @Override
     public int hashCode() {
-        return ObjectUtils.hashCodeMulti(datasetName, dataverseName, query, varCounter, var, returnQuery);
+        return ObjectUtils.hashCodeMulti(datasetName, dataverseName, query, varCounter, var, returnExpression);
     }
 
     @Override
@@ -103,7 +135,7 @@
         return ObjectUtils.equals(datasetName, target.datasetName)
                 && ObjectUtils.equals(dataverseName, target.dataverseName) && ObjectUtils.equals(query, target.query)
                 && ObjectUtils.equals(varCounter, target.varCounter) && ObjectUtils.equals(var, target.var)
-                && ObjectUtils.equals(returnQuery, target.returnQuery);
+                && ObjectUtils.equals(returnExpression, target.returnExpression);
     }
 
     @Override
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
index fd6a088..f878998 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/Query.java
@@ -19,15 +19,17 @@
 package org.apache.asterix.lang.common.statement;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
 import org.apache.commons.lang3.ObjectUtils;
 
-public class Query implements Statement {
+public class Query implements IReturningStatement {
     private final boolean explain;
     private boolean topLevel = true;
     private Expression body;
@@ -49,26 +51,36 @@
         this.datasets.addAll(datasets);
     }
 
+    @Override
     public Expression getBody() {
         return body;
     }
 
+    @Override
     public void setBody(Expression body) {
         this.body = body;
     }
 
+    @Override
     public int getVarCounter() {
         return varCounter;
     }
 
+    @Override
     public void setVarCounter(int varCounter) {
         this.varCounter = varCounter;
     }
 
+    @Override
+    public List<Expression> getDirectlyEnclosedExpressions() {
+        return Collections.singletonList(body);
+    }
+
     public void setTopLevel(boolean topLevel) {
         this.topLevel = topLevel;
     }
 
+    @Override
     public boolean isTopLevel() {
         return topLevel;
     }
@@ -126,4 +138,9 @@
     public byte getCategory() {
         return Category.QUERY;
     }
+
+    @Override
+    public String toString() {
+        return body.toString();
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
index a82d948..bedaf43 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/statement/UpsertStatement.java
@@ -18,6 +18,7 @@
  */
 package org.apache.asterix.lang.common.statement;
 
+import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.struct.Identifier;
@@ -25,8 +26,8 @@
 public class UpsertStatement extends InsertStatement {
 
     public UpsertStatement(Identifier dataverseName, Identifier datasetName, Query query, int varCounter,
-            VariableExpr var, Query returnQuery) {
-        super(dataverseName, datasetName, query, varCounter, var, returnQuery);
+            VariableExpr var, Expression returnExpression) {
+        super(dataverseName, datasetName, query, varCounter, var, returnExpression);
     }
 
     @Override
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
index ea56a6b..725d7f4 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/AbstractInlineUdfsVisitor.java
@@ -50,6 +50,7 @@
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.rewrites.VariableSubstitutionEnvironment;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
@@ -245,6 +246,20 @@
         return false;
     }
 
+    @Override
+    public Boolean visit(InsertStatement insert, List<FunctionDecl> arg) throws AsterixException {
+        boolean changed = false;
+        Expression returnExpression = insert.getReturnExpression();
+        if (returnExpression != null) {
+            Pair<Boolean, Expression> rewrittenReturnExpr = inlineUdfsInExpr(returnExpression, arg);
+            insert.setReturnExpression(rewrittenReturnExpr.second);
+            changed |= rewrittenReturnExpr.first;
+        }
+        Pair<Boolean, Expression> rewrittenBodyExpression = inlineUdfsInExpr(insert.getBody(), arg);
+        insert.setBody(rewrittenBodyExpression.second);
+        return changed || rewrittenBodyExpression.first;
+    }
+
     protected Pair<Boolean, Expression> inlineUdfsInExpr(Expression expr, List<FunctionDecl> arg)
             throws AsterixException {
         if (expr.getKind() != Kind.CALL_EXPRESSION) {
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
index 745c989..12c95bb 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/visitor/GatherFunctionCallsVisitor.java
@@ -46,6 +46,7 @@
 import org.apache.asterix.lang.common.expression.UnaryExpr;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
@@ -191,6 +192,16 @@
         return null;
     }
 
+    @Override
+    public Void visit(InsertStatement wc, Void arg) throws AsterixException {
+        wc.getQuery().accept(this, arg);
+        Expression returnExpression = wc.getReturnExpression();
+        if (returnExpression != null) {
+            returnExpression.accept(this, arg);
+        }
+        return null;
+    }
+
     public Set<FunctionSignature> getCalls() {
         return calls;
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
index 6532a47..5fee28f 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppFunctionBodyRewriter.java
@@ -21,18 +21,19 @@
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 
 class SqlppFunctionBodyRewriter extends SqlppQueryRewriter {
 
     @Override
-    public void rewrite(List<FunctionDecl> declaredFunctions, Query topExpr, MetadataProvider metadataProvider,
+    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
+            MetadataProvider metadataProvider,
             LangRewritingContext context) throws AsterixException {
         // Sets up parameters.
-        setup(declaredFunctions, topExpr, metadataProvider, context);
+        setup(declaredFunctions, topStatement, metadataProvider, context);
 
         // Inlines column aliases.
         inlineColumnAlias();
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index d6b01f2..1662f85 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -26,10 +26,10 @@
 import org.apache.asterix.common.functions.FunctionSignature;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
+import org.apache.asterix.lang.common.base.IReturningStatement;
 import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
-import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.common.visitor.GatherFunctionCallsVisitor;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
@@ -71,13 +71,13 @@
     private static final String INLINE_WITH = "inline_with";
     private static final String NOT_INLINE_WITH = "false";
     private final FunctionParser functionRepository = new FunctionParser(new SqlppParserFactory());
-    private Query topExpr;
+    private IReturningStatement topExpr;
     private List<FunctionDecl> declaredFunctions;
     private LangRewritingContext context;
     private MetadataProvider metadataProvider;
 
-    protected void setup(List<FunctionDecl> declaredFunctions, Query topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context) {
+    protected void setup(List<FunctionDecl> declaredFunctions, IReturningStatement topExpr,
+            MetadataProvider metadataProvider, LangRewritingContext context) {
         this.topExpr = topExpr;
         this.context = context;
         this.declaredFunctions = declaredFunctions;
@@ -85,13 +85,16 @@
     }
 
     @Override
-    public void rewrite(List<FunctionDecl> declaredFunctions, Query topExpr, MetadataProvider metadataProvider,
-            LangRewritingContext context) throws AsterixException {
+    public void rewrite(List<FunctionDecl> declaredFunctions, IReturningStatement topStatement,
+            MetadataProvider metadataProvider, LangRewritingContext context) throws AsterixException {
+        if (topStatement == null) {
+            return;
+        }
         // Marks the current variable counter.
         context.markCounter();
 
         // Sets up parameters.
-        setup(declaredFunctions, topExpr, metadataProvider, context);
+        setup(declaredFunctions, topStatement, metadataProvider, context);
 
         // Inlines column aliases.
         inlineColumnAlias();
@@ -141,122 +144,88 @@
         inlineWithExpressions();
 
         // Sets the var counter of the query.
-        topExpr.setVarCounter(context.getVarCounter());
+        topStatement.setVarCounter(context.getVarCounter());
     }
 
     protected void rewriteGlobalAggregations() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         SqlppGlobalAggregationSugarVisitor globalAggregationVisitor = new SqlppGlobalAggregationSugarVisitor();
-        globalAggregationVisitor.visit(topExpr, null);
+        topExpr.accept(globalAggregationVisitor, null);
     }
 
     protected void rewriteListInputFunctions() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         SqlppListInputFunctionRewriteVisitor listInputFunctionVisitor = new SqlppListInputFunctionRewriteVisitor();
-        listInputFunctionVisitor.visit(topExpr, null);
+        topExpr.accept(listInputFunctionVisitor, null);
     }
 
     protected void rewriteFunctionNames() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         SqlppBuiltinFunctionRewriteVisitor functionNameMapVisitor = new SqlppBuiltinFunctionRewriteVisitor();
-        functionNameMapVisitor.visit(topExpr, null);
+        topExpr.accept(functionNameMapVisitor, null);
     }
 
     protected void inlineWithExpressions() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         String inlineWith = metadataProvider.getConfig().get(INLINE_WITH);
         if (inlineWith != null && inlineWith.equalsIgnoreCase(NOT_INLINE_WITH)) {
             return;
         }
         // Inlines with expressions.
         InlineWithExpressionVisitor inlineWithExpressionVisitor = new InlineWithExpressionVisitor(context);
-        inlineWithExpressionVisitor.visit(topExpr, null);
+        topExpr.accept(inlineWithExpressionVisitor, null);
     }
 
     protected void generateColumnNames() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         // Generate column names if they are missing in the user query.
         GenerateColumnNameVisitor generateColumnNameVisitor = new GenerateColumnNameVisitor(context);
-        generateColumnNameVisitor.visit(topExpr, null);
+        topExpr.accept(generateColumnNameVisitor, null);
     }
 
     protected void substituteGroupbyKeyExpression() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         // Substitute group-by key expressions that appear in the select clause.
         SubstituteGroupbyExpressionWithVariableVisitor substituteGbyExprVisitor =
                 new SubstituteGroupbyExpressionWithVariableVisitor(context);
-        substituteGbyExprVisitor.visit(topExpr, null);
+        topExpr.accept(substituteGbyExprVisitor, null);
     }
 
     protected void rewriteSetOperations() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         // Rewrites set operation queries that contain order-by and limit clauses.
         SetOperationVisitor setOperationVisitor = new SetOperationVisitor(context);
-        setOperationVisitor.visit(topExpr, null);
+        topExpr.accept(setOperationVisitor, null);
     }
 
     protected void rewriteOperatorExpression() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         // Rewrites like/not-like/in/not-in operators into function call expressions.
         OperatorExpressionVisitor operatorExpressionVisitor = new OperatorExpressionVisitor(context);
-        operatorExpressionVisitor.visit(topExpr, null);
+        topExpr.accept(operatorExpressionVisitor, null);
     }
 
     protected void inlineColumnAlias() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         // Inline column aliases.
         InlineColumnAliasVisitor inlineColumnAliasVisitor = new InlineColumnAliasVisitor(context);
-        inlineColumnAliasVisitor.visit(topExpr, null);
+        topExpr.accept(inlineColumnAliasVisitor, null);
     }
 
     protected void variableCheckAndRewrite(boolean overwrite) throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         VariableCheckAndRewriteVisitor variableCheckAndRewriteVisitor =
                 new VariableCheckAndRewriteVisitor(context, overwrite, metadataProvider);
-        variableCheckAndRewriteVisitor.visit(topExpr, null);
+        topExpr.accept(variableCheckAndRewriteVisitor, null);
     }
 
     protected void rewriteGroupBys() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         SqlppGroupByVisitor groupByVisitor = new SqlppGroupByVisitor(context);
-        groupByVisitor.visit(topExpr, null);
+        topExpr.accept(groupByVisitor, null);
     }
 
     protected void inlineDeclaredUdfs() throws AsterixException {
-        if (topExpr == null) {
-            return;
-        }
         List<FunctionSignature> funIds = new ArrayList<FunctionSignature>();
         for (FunctionDecl fdecl : declaredFunctions) {
             funIds.add(fdecl.getSignature());
         }
 
-        List<FunctionDecl> usedStoredFunctionDecls = FunctionUtil.retrieveUsedStoredFunctions(metadataProvider,
-                topExpr.getBody(), funIds, null,
-                expr -> getFunctionCalls(expr), func -> functionRepository.getFunctionDecl(func),
-                signature -> FunctionMapUtil.normalizeBuiltinFunctionSignature(signature, false));
+        List<FunctionDecl> usedStoredFunctionDecls = new ArrayList<>();
+        for (Expression topLevelExpr : topExpr.getDirectlyEnclosedExpressions()) {
+            usedStoredFunctionDecls.addAll(FunctionUtil.retrieveUsedStoredFunctions(metadataProvider, topLevelExpr,
+                    funIds, null, expr -> getFunctionCalls(expr), func -> functionRepository.getFunctionDecl(func),
+                    signature -> FunctionMapUtil.normalizeBuiltinFunctionSignature(signature, false)));
+        }
         declaredFunctions.addAll(usedStoredFunctionDecls);
         if (!declaredFunctions.isEmpty()) {
             SqlppInlineUdfsVisitor visitor = new SqlppInlineUdfsVisitor(context,
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
index 6ae55dd..c289ebe 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppExpressionScopingVisitor.java
@@ -42,6 +42,7 @@
 import org.apache.asterix.lang.common.parser.ScopeChecker;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
@@ -359,6 +360,27 @@
         return varExpr;
     }
 
+    @Override
+    public Expression visit(InsertStatement insertStatement, ILangExpression arg) throws AsterixException {
+        scopeChecker.createNewScope();
+
+        // Visits the body query.
+        insertStatement.getQuery().accept(this, insertStatement);
+
+        // Registers the (inserted) data item variable.
+        VariableExpr bindingVar = insertStatement.getVar();
+        if (bindingVar != null) {
+            addNewVarSymbolToScope(scopeChecker.getCurrentScope(), bindingVar.getVar());
+        }
+
+        // Visits the expression for the returning expression.
+        Expression returningExpr = insertStatement.getReturnExpression();
+        if (returningExpr != null) {
+            insertStatement.setReturnExpression(visit(returningExpr, insertStatement));
+        }
+        return null;
+    }
+
     // Rewrites for an undefined variable reference, which potentially could be a syntatic sugar.
     protected Expression wrapWithResolveFunction(VariableExpr expr, Set<VariableExpr> liveVars)
             throws AsterixException {
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
index 21883f7..90f812c 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/visitor/base/AbstractSqlppSimpleExpressionVisitor.java
@@ -44,6 +44,7 @@
 import org.apache.asterix.lang.common.expression.UnaryExpr;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
+import org.apache.asterix.lang.common.statement.InsertStatement;
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
@@ -348,6 +349,17 @@
         return caseExpr;
     }
 
+    @Override
+    public Expression visit(InsertStatement insertStatement, ILangExpression arg) throws AsterixException {
+        Expression returnExpr = insertStatement.getReturnExpression();
+        if (returnExpr != null) {
+            insertStatement.setReturnExpression(visit(returnExpr, arg));
+        }
+        Query bodyQuery = insertStatement.getQuery();
+        bodyQuery.accept(this, arg);
+        return null;
+    }
+
     protected Expression visit(Expression expr, ILangExpression arg) throws AsterixException{
         return postVisit(preVisit(expr).accept(this, arg));
     }
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index ac6083d..b6334c8 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -120,6 +120,7 @@
 import org.apache.asterix.lang.common.statement.TypeDecl;
 import org.apache.asterix.lang.common.statement.TypeDropStatement;
 import org.apache.asterix.lang.common.statement.UpdateStatement;
+import org.apache.asterix.lang.common.statement.UpsertStatement;
 import org.apache.asterix.lang.common.statement.WriteStatement;
 import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
@@ -356,6 +357,7 @@
     | stmt = InsertStatement()
     | stmt = DeleteStatement()
     | stmt = UpdateStatement()
+    | stmt = UpsertStatement()
     | stmt = ConnectionStatement()
     | stmt = CompactStatement()
     | stmt = ExplainStatement()
@@ -963,13 +965,42 @@
 InsertStatement InsertStatement() throws ParseException:
 {
   Pair<Identifier,Identifier> nameComponents = null;
-  Query query;
+  VariableExpr var = null;
+  Query query = null;
+  Expression returnExpression = null;
 }
 {
-  <INSERT> <INTO> nameComponents = QualifiedName() query = Query(false)
+  <INSERT> <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
+    query = Query(false)
+    ( <RETURNING> returnExpression = Expression())?
     {
+      if (returnExpression != null && var == null) {
+         var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
+      }
       query.setTopLevel(true);
-      return new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), null, null);
+      return new InsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
+                                 returnExpression);
+    }
+}
+
+UpsertStatement UpsertStatement() throws ParseException:
+{
+  Pair<Identifier,Identifier> nameComponents = null;
+  VariableExpr var = null;
+  Query query = null;
+  Expression returnExpression = null;
+}
+{
+  <UPSERT> <INTO> nameComponents = QualifiedName() (<AS> var = Variable())?
+    query = Query(false)
+    ( <RETURNING> returnExpression = Expression())?
+    {
+      if (returnExpression != null && var == null) {
+         var = ExpressionToVariableUtil.getGeneratedVariable(query.getBody(), true);
+      }
+      query.setTopLevel(true);
+      return new UpsertStatement(nameComponents.first, nameComponents.second, query, getVarCounter(), var,
+                                 returnExpression);
     }
 }
 
@@ -3116,6 +3147,7 @@
   | <RAW : "raw">
   | <REFRESH : "refresh">
   | <RETURN : "return">
+  | <RETURNING : "returning">
   | <RTREE : "rtree">
   | <RUN : "run">
   | <SATISFIES : "satisfies">
@@ -3131,6 +3163,7 @@
   | <UNKOWN : "unknown">
   | <UNNEST : "unnest">
   | <UPDATE : "update">
+  | <UPSERT : "upsert">
   | <USE : "use">
   | <USING : "using">
   | <VALUE : "value">
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
index baf141f..e684285 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/entities/Dataset.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.metadata.IDatasetDetails;
 import org.apache.asterix.metadata.MetadataCache;
 import org.apache.asterix.metadata.api.IMetadataEntity;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 
 /**
@@ -191,7 +192,7 @@
         return true;
     }
 
-    public boolean allow(ProjectOperator project, byte operation) {
+    public boolean allow(ILogicalOperator topOp, byte operation) {
         return !hasMetaPart();
     }