Merge branch 'neo' into master

Change-Id: I321be68b46a7fc71d2f119bc658cb4a8844d4343
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 794a0b7..daa1d2f 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
@@ -1716,8 +1716,7 @@
             default:
                 if (expressionNeedsNoNesting(expr)) {
                     Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, topOpRef);
-                    ILogicalExpression exp = ((AssignOperator) p.first).getExpressions().get(0).getValue();
-                    return new Pair<>(exp, p.first.getInputs().get(0));
+                    return inlineAssignIfPossible((AssignOperator) p.first);
                 } else {
                     Mutable<ILogicalOperator> srcRef = new MutableObject<>();
                     Pair<ILogicalOperator, LogicalVariable> p = expr.accept(this, srcRef);
@@ -1749,6 +1748,32 @@
         }
     }
 
+    /**
+     * TODO(wyk) I believe that inlining expressions should be done at the optimization level and not at the translation
+     *  level. By inlining at the translation level, we could possibly miss optimizing inlined expressions in rules
+     *  that do not inspect arguments of a function. I kept inlining all pure (a.k.a functional) functions for now to
+     *  match the previous behavior. For non-pure functions, the assign should be kept as we do not inline them at
+     *  first due to ASTERIXDB-3103
+     *
+     * @see org.apache.hyracks.algebricks.rewriter.rules.InlineVariablesRule
+     */
+    private Pair<ILogicalExpression, Mutable<ILogicalOperator>> inlineAssignIfPossible(AssignOperator assignOp) {
+        ILogicalExpression expr = assignOp.getExpressions().get(0).getValue();
+
+        if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+            return new Pair<>(expr, assignOp.getInputs().get(0));
+        }
+
+        AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
+        if (funcExpr.isFunctional()) {
+            return new Pair<>(expr, assignOp.getInputs().get(0));
+        }
+
+        //Do not inline non-functional expressions (e.g. uuid()) and keep the assign
+        return new Pair<>(new VariableReferenceExpression(assignOp.getVariables().get(0)),
+                new MutableObject<>(assignOp));
+    }
+
     protected Pair<ILogicalOperator, LogicalVariable> aggListifyForSubquery(LogicalVariable var,
             Mutable<ILogicalOperator> opRef, boolean bProject) {
         SourceLocation sourceLoc = opRef.getValue().getSourceLocation();
@@ -1902,8 +1927,7 @@
      * Eliminate shared operator references in a query plan. Deep copy a new query
      * plan subtree whenever there is a shared operator reference.
      *
-     * @param plan,
-     *            the query plan.
+     * @param plan, the query plan.
      * @throws CompilationException
      */
     protected void eliminateSharedOperatorReferenceForPlan(ILogicalPlan plan) throws CompilationException {
@@ -1918,12 +1942,10 @@
      * <code>currentOpRef.getValue()</code>. Deep copy a new query plan subtree
      * whenever there is a shared operator reference.
      *
-     * @param currentOpRef,
-     *            the operator reference to consider
-     * @param opRefSet,
-     *            the set storing seen operator references so far.
+     * @param currentOpRef, the operator reference to consider
+     * @param opRefSet,     the set storing seen operator references so far.
      * @return a mapping that maps old variables to new variables, for the ancestors
-     *         of <code>currentOpRef</code> to replace variables properly.
+     * of <code>currentOpRef</code> to replace variables properly.
      * @throws CompilationException
      */
     private LinkedHashMap<LogicalVariable, LogicalVariable> eliminateSharedOperatorReference(
@@ -2005,14 +2027,11 @@
     /**
      * Constructs a subplan operator for a branch in a if-else (or case) expression.
      *
-     * @param inputOp,
-     *            the input operator.
-     * @param selectExpr,
-     *            the expression to select tuples that are processed by this branch.
-     * @param branchExpression,
-     *            the expression to be evaluated in this branch.
+     * @param inputOp,          the input operator.
+     * @param selectExpr,       the expression to select tuples that are processed by this branch.
+     * @param branchExpression, the expression to be evaluated in this branch.
      * @return a pair of the constructed subplan operator and the output variable
-     *         for the branch.
+     * for the branch.
      * @throws CompilationException
      */
     protected Pair<ILogicalOperator, LogicalVariable> constructSubplanOperatorForBranch(ILogicalOperator inputOp,
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query1.sqlpp
index 114fc27..0412602 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query1.sqlpp
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-DROP   DATAVERSE Test IF EXISTS;
-CREATE DATAVERSE Test;
-USE    Test;
+DROP   DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE    test;
 
 CREATE TYPE GenericType AS { _id: uuid };
 CREATE DATASET KSI (GenericType)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query2.sqlpp
index 6316ab0..2f2bb8a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-indexes/query2.sqlpp
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-DROP   DATAVERSE Test IF EXISTS;
-CREATE DATAVERSE Test;
-USE    Test;
+DROP   DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE    test;
 
 CREATE TYPE GenericType AS { _id: uuid };
 CREATE DATASET KSI (GenericType)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query1.sqlpp
index 56b061a..687ac95 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query1.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query1.sqlpp
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-DROP   DATAVERSE Test IF EXISTS;
-CREATE DATAVERSE Test;
-USE    Test;
+DROP   DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE    test;
 
 CREATE TYPE GenericType AS { _id: uuid };
 CREATE DATASET KSI (GenericType)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query2.sqlpp
index b926f71..b1d7640 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query2.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/multiple-indexes/query2.sqlpp
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-DROP   DATAVERSE Test IF EXISTS;
-CREATE DATAVERSE Test;
-USE    Test;
+DROP   DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE    test;
 
 CREATE TYPE GenericType AS { _id: uuid };
 CREATE DATASET KSI (GenericType)
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
index 7c8d8e7..b596e4b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/ASTERIXDB-2402.plan
@@ -34,98 +34,48 @@
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ASSIGN  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
-                                -- ASSIGN  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$267]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- PRE_CLUSTERED_GROUP_BY[$$268]  |PARTITIONED|
-                                              {
-                                                -- AGGREGATE  |LOCAL|
-                                                  -- STREAM_SELECT  |LOCAL|
-                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                              }
-                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- STABLE_SORT [$$268(ASC)]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$268]  |PARTITIONED|
-                                              -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STABLE_SORT [$$267(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$267]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$266][$$237]  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- HYBRID_HASH_JOIN [$$323][$$237]  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$323]  |PARTITIONED|
-                                                      -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
-                                                          -- UNNEST  |PARTITIONED|
-                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- PRE_CLUSTERED_GROUP_BY[$$326]  |PARTITIONED|
-                                                                        {
-                                                                          -- AGGREGATE  |LOCAL|
-                                                                            -- MICRO_PRE_CLUSTERED_GROUP_BY[$$328, $$330]  |LOCAL|
-                                                                                    {
-                                                                                      -- AGGREGATE  |LOCAL|
-                                                                                        -- STREAM_SELECT  |LOCAL|
-                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                                    }
-                                                                              -- STREAM_SELECT  |LOCAL|
-                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                                                        }
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- STABLE_SORT [$$326(ASC), $$328(ASC), $$330(ASC)]  |PARTITIONED|
-                                                                      -- HASH_PARTITION_EXCHANGE [$$326]  |PARTITIONED|
-                                                                        -- UNION_ALL  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                      -- BTREE_SEARCH (channels.Shelters.Shelters)  |PARTITIONED|
-                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                              -- SPLIT  |PARTITIONED|
-                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                      -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
-                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                              -- NESTED_LOOP  |PARTITIONED|
-                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- NESTED_LOOP  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                                                                        -- ASSIGN  |PARTITIONED|
-                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
-                                                                                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                                -- ASSIGN  |UNPARTITIONED|
-                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-                                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- ASSIGN  |PARTITIONED|
-                                                                                                                        -- STREAM_SELECT  |PARTITIONED|
-                                                                                                                          -- ASSIGN  |PARTITIONED|
-                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                              -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
-                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                  -- STABLE_SORT [$$275(ASC)]  |PARTITIONED|
-                                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                      -- STREAM_PROJECT  |PARTITIONED|
-                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                          -- BTREE_SEARCH (channels.Reports.report_time)  |PARTITIONED|
-                                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                                              -- ASSIGN  |PARTITIONED|
-                                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
-                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                            -- STREAM_PROJECT  |PARTITIONED|
-                                                                              -- STREAM_SELECT  |PARTITIONED|
-                                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                                  -- ASSIGN  |PARTITIONED|
+                                                  -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- UNNEST  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- PRE_CLUSTERED_GROUP_BY[$$322]  |PARTITIONED|
+                                                                  {
+                                                                    -- AGGREGATE  |LOCAL|
+                                                                      -- MICRO_PRE_CLUSTERED_GROUP_BY[$$324, $$325]  |LOCAL|
+                                                                              {
+                                                                                -- AGGREGATE  |LOCAL|
+                                                                                  -- STREAM_SELECT  |LOCAL|
+                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                              }
+                                                                        -- STREAM_SELECT  |LOCAL|
+                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                  }
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- STABLE_SORT [$$322(ASC), $$324(ASC), $$325(ASC)]  |PARTITIONED|
+                                                                -- HASH_PARTITION_EXCHANGE [$$322]  |PARTITIONED|
+                                                                  -- UNION_ALL  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- BTREE_SEARCH (channels.Shelters.Shelters)  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                     -- STREAM_PROJECT  |PARTITIONED|
                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                         -- SPLIT  |PARTITIONED|
@@ -140,13 +90,11 @@
                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                             -- NESTED_LOOP  |PARTITIONED|
                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                -- RUNNING_AGGREGATE  |PARTITIONED|
-                                                                                                                  -- ASSIGN  |PARTITIONED|
-                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                      -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
-                                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                                                                                                          -- ASSIGN  |UNPARTITIONED|
-                                                                                                                            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
+                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                                                                                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                                                                                 -- ASSIGN  |PARTITIONED|
                                                                                                                   -- STREAM_SELECT  |PARTITIONED|
@@ -154,7 +102,7 @@
                                                                                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                         -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
                                                                                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                                                                            -- STABLE_SORT [$$275(ASC)]  |PARTITIONED|
+                                                                                                                            -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
                                                                                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -168,22 +116,67 @@
                                                                                                                 -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
                                                                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                    -- HASH_PARTITION_EXCHANGE [$$237]  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- STREAM_SELECT  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- SPLIT  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- RTREE_SEARCH (channels.Shelters.s_location)  |PARTITIONED|
+                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- NESTED_LOOP  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- NESTED_LOOP  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelChannelSubscriptions)  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- STREAM_SELECT  |PARTITIONED|
+                                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- BTREE_SEARCH (channels.Reports.Reports)  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- STABLE_SORT [$$274(ASC)]  |PARTITIONED|
+                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                              -- BTREE_SEARCH (channels.Reports.report_time)  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN (channels.UserLocations)  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$237]  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240]  |PARTITIONED|
+                                                          -- HASH_PARTITION_EXCHANGE [$$248, $$250]  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions)  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- HYBRID_HASH_JOIN [$$248, $$250][$$239, $$240]  |PARTITIONED|
-                                                              -- HASH_PARTITION_EXCHANGE [$$248, $$250]  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN (channels.Broker)  |PARTITIONED|
                                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (channels.EmergenciesNearMeChannelBrokerSubscriptions)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- STREAM_PROJECT  |PARTITIONED|
-                                                                  -- ASSIGN  |PARTITIONED|
-                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                      -- DATASOURCE_SCAN (channels.Broker)  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
index 060ee79..0d37f51 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query1.plan
@@ -16,7 +16,7 @@
                 -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (Test.KSI.KSI)  |PARTITIONED|
+                      -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- INTERSECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -26,7 +26,7 @@
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (Test.KSI.KS1_array_index1)  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
@@ -37,7 +37,7 @@
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (Test.KSI.KS1_array_index2)  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
index e3d1c5d..b793939 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-indexes/query2.plan
@@ -16,7 +16,7 @@
                 -- ASSIGN  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- BTREE_SEARCH (Test.KSI.KSI)  |PARTITIONED|
+                      -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- INTERSECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -26,7 +26,7 @@
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (Test.KSI.KS1_array_index1)  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
@@ -37,7 +37,7 @@
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (Test.KSI.KS1_array_index2)  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
@@ -48,7 +48,7 @@
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- BTREE_SEARCH (Test.KSI.KS1_array_index3)  |PARTITIONED|
+                                          -- BTREE_SEARCH (test.KSI.KS1_array_index3)  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
index def9040..4c76ea0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query1.plan
@@ -8,7 +8,7 @@
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (Test.KSI.KSI)  |PARTITIONED|
+                    -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- INTERSECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -18,7 +18,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (Test.KSI.KS1_array_index1)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
@@ -29,7 +29,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (Test.KSI.KS1_array_index2)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
index 1c73ebc..7a2f7ce 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/multiple-indexes/query2.plan
@@ -8,7 +8,7 @@
               -- ASSIGN  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    -- BTREE_SEARCH (Test.KSI.KSI)  |PARTITIONED|
+                    -- BTREE_SEARCH (test.KSI.KSI)  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                         -- INTERSECT  |PARTITIONED|
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -18,7 +18,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (Test.KSI.KS1_array_index1)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.KSI.KS1_array_index1)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
@@ -29,7 +29,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (Test.KSI.KS1_array_index2)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.KSI.KS1_array_index2)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
@@ -40,7 +40,7 @@
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- BTREE_SEARCH (Test.KSI.KS1_array_index3)  |PARTITIONED|
+                                        -- BTREE_SEARCH (test.KSI.KS1_array_index3)  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                             -- ASSIGN  |PARTITIONED|
                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
index 075f7af..15c163d 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/joins/nested_query_with_bcast.plan
@@ -9,31 +9,29 @@
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- ASSIGN  |PARTITIONED|
-                        -- STREAM_PROJECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
-                                    {
-                                      -- AGGREGATE  |LOCAL|
-                                        -- STREAM_SELECT  |LOCAL|
-                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                    }
+                            -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- STABLE_SORT [$$38(ASC)]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STREAM_PROJECT  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$41][$$39]  |PARTITIONED|
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- HYBRID_HASH_JOIN [$$41][$$39]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.tweetDataset)  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- DATASOURCE_SCAN (test.tweetDataset)  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
-                                            -- STREAM_PROJECT  |PARTITIONED|
-                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN (test.countryDataset)  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- DATASOURCE_SCAN (test.countryDataset)  |PARTITIONED|
-                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
index 792b79d..9a296a5 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/primary-key-index/insert-primary-key-index-with-auto-gen-pk.plan
@@ -6,7 +6,5 @@
           -- ASSIGN  |UNPARTITIONED|
             -- STREAM_PROJECT  |UNPARTITIONED|
               -- ASSIGN  |UNPARTITIONED|
-                -- STREAM_PROJECT  |UNPARTITIONED|
-                  -- ASSIGN  |UNPARTITIONED|
-                    -- ASSIGN  |UNPARTITIONED|
-                      -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                -- ASSIGN  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.1.ddl.sqlpp
index 73e442e..d589457 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.1.ddl.sqlpp
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-DROP   DATAVERSE Test IF EXISTS;
-CREATE DATAVERSE Test;
-USE    Test;
+DROP   DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE    test;
 
 CREATE TYPE GenericType AS { id: bigint };
 CREATE DATASET KSI (GenericType)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.2.update.sqlpp
index 1276c97..1d1f638 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.2.update.sqlpp
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-USE    Test;
+USE    test;
 
 INSERT INTO KSI [
   { "id": 1, "uarr_i": [ {"a": 1, "b": 1, "c": 1 } ] },
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.query.sqlpp
index 697cca1..ce7f053 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.3.query.sqlpp
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-USE    Test;
+USE    test;
 
 FROM     KSI k
 WHERE    ( SOME i IN k.uarr_i
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.query.sqlpp
index 948d367..1314ad8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-quantified-queries/multiple-indexes/multiple-indexes.4.query.sqlpp
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-USE    Test;
+USE    test;
 
 FROM     KSI k
 WHERE    ( SOME i IN k.uarr_i
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.1.ddl.sqlpp
index 73e442e..d589457 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.1.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.1.ddl.sqlpp
@@ -16,9 +16,9 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-DROP   DATAVERSE Test IF EXISTS;
-CREATE DATAVERSE Test;
-USE    Test;
+DROP   DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE    test;
 
 CREATE TYPE GenericType AS { id: bigint };
 CREATE DATASET KSI (GenericType)
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.2.update.sqlpp
index 1276c97..1d1f638 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.2.update.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.2.update.sqlpp
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-USE    Test;
+USE    test;
 
 INSERT INTO KSI [
   { "id": 1, "uarr_i": [ {"a": 1, "b": 1, "c": 1 } ] },
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.query.sqlpp
index 3880801..c9b3208 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.3.query.sqlpp
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-USE    Test;
+USE    test;
 
 FROM     KSI k
 UNNEST   k.uarr_i i
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.query.sqlpp
index 252e073..96aee1c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/array-index/select-unnest-queries/open/multiple-indexes/multiple-indexes.4.query.sqlpp
@@ -16,7 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-USE    Test;
+USE    test;
 
 FROM     KSI k
 UNNEST   k.uarr_i i
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.1.query.sqlpp
new file mode 100644
index 0000000..c23e98c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.1.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.
+ */
+
+EXPLAIN
+SELECT VALUE A
+FROM [1, 2, 3] AS A
+LIMIT random()
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.2.query.sqlpp
new file mode 100644
index 0000000..9c995e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit-non-pure-function/limit-non-pure-function.2.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: Make sure that the call to random() (rand) is not inlined
+ */
+
+EXPLAIN
+WITH rand AS random()
+SELECT A, rand
+FROM [1, 2, 3] AS A
+LIMIT rand
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
index 517a996..dfb47e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/constant_folding/constant_folding.7.query.sqlpp
@@ -19,6 +19,10 @@
 
 /*
  * Description: No constant folding of OR with a non functional argument
+ *
+ * Update/note: this should be folded as the non functional argument would short-circuited in runtime. Due to
+ * ASTERIXDB-3103, the constant folding rule doesn't see a non functional argument, but a variable. Hence,
+ * the expression 'get_year(current_date()) < x' is eliminated
  */
 
 explain select value true or get_year(current_date()) < x
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/serialized_size_fun/serialized_size_fun.01.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/serialized_size_fun/serialized_size_fun.01.query.sqlpp
new file mode 100644
index 0000000..d3c8f1a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/serialized_size_fun/serialized_size_fun.01.query.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.
+ */
+SET `import-private-functions` `true`;
+
+from [
+{"id": 1, "f": 1},
+{"id": 2, "f": true},
+{"id": 3, "f": "test"},
+{"id": 4, "f": [1,2]},
+{"id": 5, "f": {"f1": 1, "f2": [{"n": "str"}]}}
+] AS t
+select t.f AS val, serialized_size(t.f) AS serialized_size
+order by t.id;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp
index 278f5d2..1ed0c05 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.3.query.sqlpp
@@ -25,6 +25,6 @@
 
 SELECT VALUE md.name
 FROM MyDataset md
-LET currentData = {"myDate": current_date()}
-WHERE currentData.myDate = current_date()
+LET myObject = {"myUid": uuid()}
+WHERE myObject.myUid != uuid()
 ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp
index f44921f..96095b4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.4.query.sqlpp
@@ -25,6 +25,6 @@
 EXPLAIN
 SELECT VALUE md.name
 FROM MyDataset md
-LET currentData = {"myDate": current_date()}
-WHERE currentData.myDate = current_date()
+LET myObject = {"myUid": uuid()}
+WHERE myObject.myUid != uuid()
 ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.5.query.sqlpp
new file mode 100644
index 0000000..60f5d4b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.5.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.
+ */
+
+USE test;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+
+SELECT VALUE md.name
+FROM MyDataset md
+LET currentData = {"myDate": current_date()}
+WHERE currentData.myDate != date("1980-09-10")
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.6.query.sqlpp
new file mode 100644
index 0000000..5dae0bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/load-record-fields/load-record-fields.6.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.
+ */
+
+USE test;
+
+-- Disabled for a simpler plan
+SET `compiler.sort.parallel` "false";
+
+EXPLAIN
+SELECT VALUE md.name
+FROM MyDataset md
+LET currentData = {"myDate": current_date()}
+WHERE currentData.myDate != date("1980-09-10")
+ORDER BY md.id
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.1.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.1.plan
new file mode 100644
index 0000000..d5f6401
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.1.plan
@@ -0,0 +1,14 @@
+distribute result [$$A]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    project ([$$A])
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      limit switch-case(gt($$13, 0), true, $$13, 0)
+      -- STREAM_LIMIT  |UNPARTITIONED|
+        assign [$$13] <- [treat-as-integer(random())]
+        -- ASSIGN  |UNPARTITIONED|
+          unnest $$A <- scan-collection(array: [ 1, 2, 3 ])
+          -- UNNEST  |UNPARTITIONED|
+            empty-tuple-source
+            -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.2.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.2.plan
new file mode 100644
index 0000000..54efcb6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit-non-pure-function/limit-non-pure-function.2.plan
@@ -0,0 +1,18 @@
+distribute result [$$24]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    project ([$$24])
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      limit switch-case(gt($$25, 0), true, $$25, 0)
+      -- STREAM_LIMIT  |UNPARTITIONED|
+        project ([$$25, $$24])
+        -- STREAM_PROJECT  |UNPARTITIONED|
+          assign [$$25, $$24] <- [treat-as-integer($$21), {"A": $$A, "rand": $$21}]
+          -- ASSIGN  |UNPARTITIONED|
+            unnest $$A <- scan-collection(array: [ 1, 2, 3 ])
+            -- UNNEST  |UNPARTITIONED|
+              assign [$$21] <- [random()]
+              -- ASSIGN  |UNPARTITIONED|
+                empty-tuple-source
+                -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
index ef61aa3..c46d9ad 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/constant_folding/constant_folding.7.plan
@@ -4,10 +4,10 @@
 -- DISTRIBUTE_RESULT  |UNPARTITIONED|
   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
-    project ([$$17]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-    -- STREAM_PROJECT  |UNPARTITIONED|
-      assign [$$17] <- [or(true, lt(get-year(current-date()), $$x))] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-      -- ASSIGN  |UNPARTITIONED|
+    assign [$$17] <- [true] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+    -- ASSIGN  |UNPARTITIONED|
+      project ([]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+      -- STREAM_PROJECT  |UNPARTITIONED|
         exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
         -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
           order (ASC, $$x) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/serialized_size_fun/serialized_size_fun.01.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/serialized_size_fun/serialized_size_fun.01.adm
new file mode 100644
index 0000000..511674b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/serialized_size_fun/serialized_size_fun.01.adm
@@ -0,0 +1,5 @@
+{ "serialized_size": 9, "val": 1 }
+{ "serialized_size": 2, "val": true }
+{ "serialized_size": 6, "val": "test" }
+{ "serialized_size": 36, "val": [ 1, 2 ] }
+{ "serialized_size": 88, "val": { "f1": 1, "f2": [ { "n": "str" } ] } }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
index 79b7115..8d159b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.4.plan
@@ -6,23 +6,19 @@
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     project ([$$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
     -- STREAM_PROJECT  |PARTITIONED|
-      exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-      -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
-        project ([$$30, $$32]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+      assign [$$30] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+      -- ASSIGN  |PARTITIONED|
+        project ([$$md]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
         -- STREAM_PROJECT  |PARTITIONED|
-          select (eq($$33, current-date())) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
-          -- STREAM_SELECT  |PARTITIONED|
-            assign [$$33] <- [current-date()] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-            -- ASSIGN  |PARTITIONED|
-              project ([$$32, $$30]) [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-              -- STREAM_PROJECT  |PARTITIONED|
-                assign [$$30] <- [$$md.getField("name")] [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                -- ASSIGN  |PARTITIONED|
+          exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+          -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
+            select (neq(uuid(), uuid())) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- DATASOURCE_SCAN  |PARTITIONED|
                   exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                    data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                    -- DATASOURCE_SCAN  |PARTITIONED|
-                      exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
-                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.5.adm
new file mode 100644
index 0000000..ac2dc97
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.5.adm
@@ -0,0 +1,2 @@
+"Alice"
+"Bob"
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan
new file mode 100644
index 0000000..abd19fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/objects/load-record-fields/load-record-fields.6.plan
@@ -0,0 +1,24 @@
+cardinality: 1000000.0
+cost: 1000000.0
+distribute result [$$30] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    project ([$$30]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+    -- STREAM_PROJECT  |PARTITIONED|
+      assign [$$30] <- [$$md.getField("name")] [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+      -- ASSIGN  |PARTITIONED|
+        project ([$$md]) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+        -- STREAM_PROJECT  |PARTITIONED|
+          exchange [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+          -- SORT_MERGE_EXCHANGE [$$32(ASC) ]  |PARTITIONED|
+            select (neq(current-date(), date: { 1980-09-10 })) [cardinality: 1000000.0, op-cost: 0.0, total-cost: 1000000.0]
+            -- STREAM_SELECT  |PARTITIONED|
+              exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                data-scan []<-[$$32, $$md] <- test.MyDataset [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                -- DATASOURCE_SCAN  |PARTITIONED|
+                  exchange [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    empty-tuple-source [cardinality: 0.0, op-cost: 0.0, total-cost: 0.0]
+                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
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 17a5205..e124b56 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -7283,6 +7283,11 @@
         <output-dir compare="Text">query-ASTERIXDB-865</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="serialized_size_fun">
+        <output-dir compare="Text">serialized_size_fun</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="multipart-dataverse">
     <test-case FilePath="multipart-dataverse">
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java
index 83ebc98..c21ac58 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/FuzzyJoinFunctionRegistrant.java
@@ -37,7 +37,6 @@
 import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardPrefixDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedCheckDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SimilarityJaccardSortedDescriptor;
-import org.apache.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.WordTokensDescriptor;
 
 public class FuzzyJoinFunctionRegistrant implements IFunctionRegistrant {
@@ -48,9 +47,6 @@
         fc.add(SimilarityJaccardPrefixDescriptor.FACTORY);
         fc.add(SimilarityJaccardPrefixCheckDescriptor.FACTORY);
 
-        // Spatial
-        fc.add(SpatialIntersectDescriptor.FACTORY);
-
         // fuzzyjoin function
         fc.add(PrefixLenJaccardDescriptor.FACTORY);
         fc.add(WordTokensDescriptor.FACTORY);
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
index 95fd5bf..341dfe8 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
+++ b/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SimilarityDescriptor.java
@@ -26,7 +26,6 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.AOrderedListSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.AUnorderedListSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.fuzzyjoin.IntArray;
 import org.apache.asterix.fuzzyjoin.similarity.PartialIntersect;
 import org.apache.asterix.fuzzyjoin.similarity.SimilarityFilters;
 import org.apache.asterix.fuzzyjoin.similarity.SimilarityMetric;
@@ -36,6 +35,7 @@
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.IntArray;
 import org.apache.asterix.runtime.evaluators.common.SimilarityFiltersCache;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index a7066a2..cdc8111 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -1709,6 +1709,9 @@
     public static final FunctionIdentifier DECODE_DATAVERSE_NAME =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "decode-dataverse-name", 1);
 
+    public static final FunctionIdentifier SERIALIZED_SIZE =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "serialized-size", 1);
+
     static {
         // first, take care of Algebricks builtin functions
         addFunction(IS_MISSING, BooleanOnlyTypeComputer.INSTANCE, true);
@@ -2543,6 +2546,7 @@
         addFunction(DECODE_DATAVERSE_NAME, OrderedListOfAStringTypeComputer.INSTANCE_NULLABLE, true);
 
         addPrivateFunction(COLLECTION_TO_SEQUENCE, CollectionToSequenceTypeComputer.INSTANCE, true);
+        addPrivateFunction(SERIALIZED_SIZE, AInt64TypeComputer.INSTANCE, true);
 
         // external lookup
         addPrivateFunction(EXTERNAL_LOOKUP, AnyTypeComputer.INSTANCE, false);
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/fuzzyjoin/IntArray.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/IntArray.java
similarity index 96%
rename from asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/fuzzyjoin/IntArray.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/IntArray.java
index 91fb8b4..627ed89 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/fuzzyjoin/IntArray.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/common/IntArray.java
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package org.apache.asterix.fuzzyjoin;
+package org.apache.asterix.runtime.evaluators.common;
 
 import java.util.Arrays;
 
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SerializedSizeDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SerializedSizeDescriptor.java
new file mode 100644
index 0000000..2669fd7
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SerializedSizeDescriptor.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt64;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public final class SerializedSizeDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+
+    public static final IFunctionDescriptorFactory FACTORY = SerializedSizeDescriptor::new;
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(IEvaluatorContext ctx) throws HyracksDataException {
+                return new AbstractScalarEval(sourceLoc, getIdentifier()) {
+                    private final IScalarEvaluator eval0 = args[0].createScalarEvaluator(ctx);
+                    private final VoidPointable arg0 = VoidPointable.FACTORY.createPointable();
+                    private final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                    private final DataOutput resultOut = resultStorage.getDataOutput();
+
+                    @SuppressWarnings("unchecked")
+                    private final ISerializerDeserializer<AInt64> int64Serde =
+                            SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+                    private final AMutableInt64 int64 = new AMutableInt64(0);
+
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        resultStorage.reset();
+                        eval0.evaluate(tuple, arg0);
+                        int64.setValue(arg0.getLength());
+                        try {
+                            int64Serde.serialize(int64, resultOut);
+                            result.set(resultStorage);
+                        } catch (IOException e) {
+                            throw HyracksDataException.create(e);
+                        }
+                    }
+                };
+            }
+        };
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.SERIALIZED_SIZE;
+    }
+}
diff --git a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
similarity index 99%
rename from asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
rename to asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
index 5a01007..4e69b43 100644
--- a/asterixdb/asterix-fuzzyjoin/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/SpatialIntersectDescriptor.java
@@ -31,7 +31,6 @@
 import org.apache.asterix.dataflow.data.nontagged.serde.APolygonSerializerDeserializer;
 import org.apache.asterix.dataflow.data.nontagged.serde.ARectangleSerializerDeserializer;
 import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
-import org.apache.asterix.fuzzyjoin.IntArray;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
@@ -39,6 +38,7 @@
 import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
 import org.apache.asterix.runtime.evaluators.common.DoubleArray;
+import org.apache.asterix.runtime.evaluators.common.IntArray;
 import org.apache.asterix.runtime.evaluators.common.SpatialUtils;
 import org.apache.asterix.runtime.exceptions.InvalidDataFormatException;
 import org.apache.asterix.runtime.exceptions.TypeMismatchException;
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index e07d1b7..63dae91 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -449,10 +449,12 @@
 import org.apache.asterix.runtime.evaluators.functions.RandomDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.RandomWithSeedDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ReferenceTileDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SerializedSizeDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SleepDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialAreaDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialCellDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.SpatialDistanceDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.SpatialIntersectDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringConcatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringContainsDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.StringEndsWithDescriptor;
@@ -1175,6 +1177,7 @@
         fc.add(LineRectanglePolygonAccessor.FACTORY);
         fc.add(ReferenceTileDescriptor.FACTORY);
         fc.add(GetIntersectionDescriptor.FACTORY);
+        fc.add(SpatialIntersectDescriptor.FACTORY);
 
         // full-text function
         fc.add(FullTextContainsFunctionDescriptor.FACTORY);
@@ -1321,6 +1324,7 @@
         // Other functions
         fc.add(DecodeDataverseNameDescriptor.FACTORY);
         fc.add(RandomWithSeedDescriptor.FACTORY);
+        fc.add(SerializedSizeDescriptor.FACTORY);
 
         ServiceLoader.load(IFunctionRegistrant.class).iterator().forEachRemaining(c -> c.register(fc));
         return fc;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index ccd152f..b6ad341c 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -251,8 +251,8 @@
             return Boolean.FALSE;
         }
         OrderOperator orderOpArg = (OrderOperator) copyAndSubstituteVar(op, arg);
-        boolean isomorphic = compareIOrderAndExpressions(op.getOrderExpressions(), orderOpArg.getOrderExpressions());
-        return isomorphic;
+        return op.getTopK() == orderOpArg.getTopK()
+                && compareIOrderAndExpressions(op.getOrderExpressions(), orderOpArg.getOrderExpressions());
     }
 
     @Override
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
index cb66cd2..c0c5681 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
@@ -942,8 +942,9 @@
     }
 
     private void writeBuildSide(AbstractBinaryJoinOperator op) throws IOException {
-        int buildInputIndex = printInputsInReverse(op) ? 0 : 1;
-        jsonGenerator.writeNumberField("build-side", buildInputIndex);
+        if (isHashJoin(op)) {
+            jsonGenerator.writeNumberField("build-side", 0);
+        }
     }
 
     private static boolean printInputsInReverse(AbstractLogicalOperator op) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
index f072312..be169f4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineSingleReferenceVariablesRule.java
@@ -27,7 +27,6 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 
 /**
@@ -83,7 +82,7 @@
     }
 
     @Override
-    protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
+    protected boolean performBottomUpAction(ILogicalOperator op) throws AlgebricksException {
         usedVars.clear();
         VariableUtilities.getUsedVariables(op, usedVars);
         for (LogicalVariable var : usedVars) {
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
index cec830e..6ad50e6 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
@@ -26,6 +26,7 @@
 import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.commons.lang3.mutable.MutableInt;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -37,7 +38,6 @@
 import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WindowOperator;
@@ -80,6 +80,10 @@
     // set to prevent re-visiting a subtree from the other sides. Operators with multiple outputs are the ones that
     // could be re-visited twice or more (e.g. replicate and split operators)
     private final Map<ILogicalOperator, Boolean> subTreesDone = new HashMap<>();
+    // temporary set to get used variables
+    private final List<LogicalVariable> usedVars = new ArrayList<>();
+    // map of variables and the counts of how many times they were used
+    private final Map<LogicalVariable, MutableInt> usedVariableCounter = new HashMap<>();
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
@@ -108,9 +112,10 @@
         varAssignRhs.clear();
         inlineVisitor.setContext(context);
         subTreesDone.clear();
+        usedVariableCounter.clear();
     }
 
-    protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
+    protected boolean performBottomUpAction(ILogicalOperator op) throws AlgebricksException {
         // Only inline variables in operators that can deal with arbitrary expressions.
         if (!op.requiresVariableReferenceExpressions()) {
             inlineVisitor.setOperator(op);
@@ -125,27 +130,33 @@
 
     private boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+        ILogicalOperator op = opRef.getValue();
 
         // check if you have already visited the subtree rooted at this operator
         if (subTreesDone.containsKey(op)) {
             return subTreesDone.get(op);
         }
+
+        // compute how many times a variable was used
+        computeUsedVariableCount(op);
+
         // Update mapping from variables to expressions during top-down traversal.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AssignOperator assignOp = (AssignOperator) op;
             List<LogicalVariable> vars = assignOp.getVariables();
             List<Mutable<ILogicalExpression>> exprs = assignOp.getExpressions();
             for (int i = 0; i < vars.size(); i++) {
+                LogicalVariable variable = vars.get(i);
                 ILogicalExpression expr = exprs.get(i).getValue();
                 // Ignore functions that are either in the doNotInline set or are non-functional
                 if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
                     AbstractFunctionCallExpression funcExpr = (AbstractFunctionCallExpression) expr;
-                    if (doNotInlineFuncs.contains(funcExpr.getFunctionIdentifier()) || !funcExpr.isFunctional()) {
+                    if (doNotInlineFuncs.contains(funcExpr.getFunctionIdentifier())
+                            || skipNonFunctional(variable, funcExpr)) {
                         continue;
                     }
                 }
-                varAssignRhs.put(vars.get(i), exprs.get(i).getValue());
+                varAssignRhs.put(variable, expr);
             }
         }
 
@@ -200,6 +211,54 @@
         return modified;
     }
 
+    /**
+     * Skip inlining non-pure functions if they are referenced more than once.
+     *
+     * @param variable assigned to the function
+     * @param expr     of the assigned variable (potentially a non-pure one)
+     * @return true if inlining should be skipped, false if the non-pure functions can be inlined
+     */
+    private boolean skipNonFunctional(LogicalVariable variable, AbstractFunctionCallExpression expr) {
+        return !expr.isFunctional() && usedVariableCounter.containsKey(variable)
+                && usedVariableCounter.get(variable).getValue() > 1;
+    }
+
+    /**
+     * Computes how many times the variables in the plan were used. The variable counts
+     * help to determine if whether we can inline non-pure functions (e.g., current_date()) or not.
+     * Inlining non-pure functions can help unlocking other optimizations. For instance, we can pushdown
+     * limits and selects into data-scans or avoid create_query_uuid().
+     * <p>
+     * Non-pure functions can only be inlined if they were referenced once. For example,
+     * in the following plan, the function current_date() (or variable $$x) can be inlined as it was used
+     * once by select:
+     * <p>
+     * Before:
+     * select (get_year($$x) > 2000)
+     * * assign [$$x] <- [current_date()]
+     * After:
+     * select (get_year(current_date()) > 2000)
+     * <p>
+     * However, the following current_date() (or variable $$x) cannot be inlined as it referenced twice:
+     * select (get_year($$x) > 2000 && get_month($$x) == 11)
+     * * assign [$$x] <- [current_date()]
+     *
+     * @param op the logical operator that potentially uses one or more variables
+     */
+    private void computeUsedVariableCount(ILogicalOperator op) throws AlgebricksException {
+        if (op.getOperatorTag() == LogicalOperatorTag.SUBPLAN) {
+            // to avoid count a variable twice as this routine traverses the subplan
+            return;
+        }
+
+        usedVars.clear();
+        VariableUtilities.getUsedVariables(op, usedVars);
+        for (LogicalVariable variable : usedVars) {
+            MutableInt counter = usedVariableCounter.computeIfAbsent(variable, k -> new MutableInt(0));
+            counter.increment();
+        }
+    }
+
     public static class InlineVariablesVisitor extends LogicalExpressionReferenceTransformVisitor
             implements ILogicalExpressionReferenceTransform {
 
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
index 12985c0..613a396 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/buffermanager/PreferToSpillFullyOccupiedFramePolicy.java
@@ -98,4 +98,14 @@
             }
         };
     }
+
+    public String partitionsStatus() {
+        StringBuilder sb = new StringBuilder();
+        int numPartitions = bufferManager.getNumPartitions();
+        for (int p = 0; p < numPartitions; p++) {
+            sb.append("p:").append(p).append(",#t:").append(bufferManager.getNumTuples(p)).append(",s:")
+                    .append(spilledStatus.get(p)).append(",s:").append(bufferManager.getPhysicalSize(p)).append('\n');
+        }
+        return sb.toString();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
index b732cd4..b35c5ff 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/OptimizedHybridHashJoin.java
@@ -50,6 +50,8 @@
 import org.apache.hyracks.dataflow.std.structures.ISerializableTable;
 import org.apache.hyracks.dataflow.std.structures.SerializableHashTable;
 import org.apache.hyracks.dataflow.std.structures.TuplePointer;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 /**
  * This class mainly applies one level of HHJ on a pair of
@@ -57,6 +59,7 @@
  */
 public class OptimizedHybridHashJoin {
 
+    private static final Logger LOGGER = LogManager.getLogger();
     // Used for special probe BigObject which can not be held into the Join memory
     private FrameTupleAppender bigFrameAppender;
 
@@ -157,12 +160,19 @@
             int recordSize = VPartitionTupleBufferManager.calculateActualSize(null, accessorBuild.getTupleLength(tid));
             double numFrames = (double) recordSize / (double) jobletCtx.getInitialFrameSize();
             int victimPartition;
-            if (numFrames > bufferManager.getConstrain().frameLimit(pid)
-                    || (victimPartition = spillPolicy.selectVictimPartition(pid)) < 0) {
+            int partitionFrameLimit = bufferManager.getConstrain().frameLimit(pid);
+            if (numFrames > partitionFrameLimit || (victimPartition = spillPolicy.selectVictimPartition(pid)) < 0) {
                 // insert request can never be satisfied
                 if (numFrames > memSizeInFrames || recordSize < jobletCtx.getInitialFrameSize()) {
                     // the tuple is greater than the memory budget or although the record is small we could not find
                     // a frame for it (possibly due to a bug)
+                    String details = String.format(
+                            "partition %s, tuple size %s, needed # frames %s, partition frame limit %s, join "
+                                    + "memory in frames %s, initial frame size %s",
+                            pid, recordSize, numFrames, partitionFrameLimit, memSizeInFrames,
+                            jobletCtx.getInitialFrameSize());
+                    LOGGER.debug("can't insert tuple in join memory. {}", details);
+                    LOGGER.debug("partitions status:\n{}", spillPolicy.partitionsStatus());
                     throw HyracksDataException.create(ErrorCode.INSUFFICIENT_MEMORY);
                 }
                 // Record is large but insertion failed either 1) we could not satisfy the request because of the