Merge branch 'gerrit/cheshire-cat'

Change-Id: Ia23c75c9cc5852837276eed134876690ce3e3901
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 6ae1050..b6e287b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -185,6 +185,8 @@
         normalization.add(new CheckInsertUpsertReturningRule());
         normalization.add(new IntroduceUnnestForCollectionToSequenceRule());
         normalization.add(new EliminateSubplanRule());
+        // The following rule must run before PushAggregateIntoNestedSubplanRule
+        normalization.add(new EliminateIsomorphicSubplanRule());
         normalization.add(new EnforceOrderByAfterSubplan());
         normalization.add(new BreakSelectIntoConjunctsRule());
         normalization.add(new ExtractGbyExpressionsRule());
@@ -221,7 +223,7 @@
 
         condPushDownAndJoinInference.add(new PushSelectDownRule());
         condPushDownAndJoinInference.add(new PushSortDownRule());
-        condPushDownAndJoinInference.add(new RemoveRedundantListifyRule());
+        condPushDownAndJoinInference.add(new RemoveRedundantListifyRule(false));
         condPushDownAndJoinInference.add(new CancelUnnestWithNestedListifyRule());
         condPushDownAndJoinInference.add(new SimpleUnnestToProductRule());
         condPushDownAndJoinInference.add(new ComplexUnnestToProductRule());
@@ -236,9 +238,6 @@
         condPushDownAndJoinInference
                 .add(new AsterixPushMapOperatorThroughUnionRule(LogicalOperatorTag.ASSIGN, LogicalOperatorTag.SELECT));
         condPushDownAndJoinInference.add(new SubplanOutOfGroupRule());
-        // The following rule must run before PushAggregateIntoNestedSubplanRule
-        // (before common subplans diverge due to aggregate pushdown)
-        condPushDownAndJoinInference.add(new EliminateIsomorphicSubplanRule());
 
         condPushDownAndJoinInference.add(new AsterixExtractFunctionsFromJoinConditionRule());
 
@@ -298,7 +297,7 @@
         consolidation.add(new RemoveRedundantGroupByDecorVarsRule());
         //PushUnnestThroughUnion => RemoveRedundantListifyRule cause these rules are correlated
         consolidation.add(new AsterixPushMapOperatorThroughUnionRule(LogicalOperatorTag.UNNEST));
-        consolidation.add(new RemoveRedundantListifyRule());
+        consolidation.add(new RemoveRedundantListifyRule(true));
         // Window operator consolidation rules
         consolidation.add(new AsterixConsolidateWindowOperatorsRule());
         consolidation.add(new ReuseWindowAggregateRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index b7713c9..bf87fc0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -573,7 +573,8 @@
                         AbstractFunctionCallExpression createMBR = new ScalarFunctionCallExpression(
                                 FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
                         createMBR.setSourceLocation(sourceLoc);
-                        createMBR.getArguments().add(beforeOpSecondaryExpressions.get(0));
+                        createMBR.getArguments().add(
+                                new MutableObject<>(beforeOpSecondaryExpressions.get(0).getValue().cloneExpression()));
                         createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                                 new ConstantExpression(new AsterixConstantValue(new AInt32(dimension)))));
                         createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
index 1fff73a..1dfa0eb 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggFuncIntoStandaloneAggregateRule.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.optimizer.rules;
 
 import java.util.ArrayList;
-import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
@@ -44,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 /**
@@ -72,13 +72,13 @@
         if (op2.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
             AggregateOperator aggOp = (AggregateOperator) op2;
             // Make sure the agg expr is a listify.
-            return pushAggregateFunction(aggOp, assignOp, context);
+            return pushAggregateFunction(assignOp, aggOp, context);
         } else if (op2.getOperatorTag() == LogicalOperatorTag.INNERJOIN
                 || op2.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
             AbstractBinaryJoinOperator join = (AbstractBinaryJoinOperator) op2;
             // Tries to push aggregates through the join.
             if (containsAggregate(assignOp.getExpressions()) && pushableThroughJoin(join)) {
-                return pushAggregateFunctionThroughJoin(join, assignOp, context);
+                return pushAggregateFunctionThroughJoin(assignOp, join, context);
             }
         }
         return false;
@@ -87,7 +87,6 @@
     /**
      * Recursively check whether the list of expressions contains an aggregate function.
      *
-     * @param exprRefs
      * @return true if the list contains an aggregate function and false otherwise.
      */
     private boolean containsAggregate(List<Mutable<ILogicalExpression>> exprRefs) {
@@ -116,7 +115,6 @@
      * 1) the join condition is true;
      * 2) each join branch produces only one tuple.
      *
-     * @param join
      * @return true if pushable
      */
     private boolean pushableThroughJoin(AbstractBinaryJoinOperator join) {
@@ -144,100 +142,116 @@
     /**
      * Does the actual push of aggregates for qualified joins.
      *
-     * @param join
      * @param assignOp
      *            that contains aggregate function calls.
      * @param context
      * @throws AlgebricksException
      */
-    private boolean pushAggregateFunctionThroughJoin(AbstractBinaryJoinOperator join, AssignOperator assignOp,
+    private boolean pushAggregateFunctionThroughJoin(AssignOperator assignOp, AbstractBinaryJoinOperator join,
             IOptimizationContext context) throws AlgebricksException {
         boolean applied = false;
         for (Mutable<ILogicalOperator> branchRef : join.getInputs()) {
             AbstractLogicalOperator branch = (AbstractLogicalOperator) branchRef.getValue();
             if (branch.getOperatorTag() == LogicalOperatorTag.AGGREGATE) {
                 AggregateOperator aggOp = (AggregateOperator) branch;
-                applied |= pushAggregateFunction(aggOp, assignOp, context);
+                applied |= pushAggregateFunction(assignOp, aggOp, context);
             } else if (branch.getOperatorTag() == LogicalOperatorTag.INNERJOIN
                     || branch.getOperatorTag() == LogicalOperatorTag.LEFTOUTERJOIN) {
                 AbstractBinaryJoinOperator childJoin = (AbstractBinaryJoinOperator) branch;
-                applied |= pushAggregateFunctionThroughJoin(childJoin, assignOp, context);
+                applied |= pushAggregateFunctionThroughJoin(assignOp, childJoin, context);
             }
         }
         return applied;
     }
 
-    private boolean pushAggregateFunction(AggregateOperator aggOp, AssignOperator assignOp,
+    private boolean pushAggregateFunction(AssignOperator assignOp, AggregateOperator aggOp,
             IOptimizationContext context) throws AlgebricksException {
-        Mutable<ILogicalOperator> opRef3 = aggOp.getInputs().get(0);
-        AbstractLogicalOperator op3 = (AbstractLogicalOperator) opRef3.getValue();
+        Mutable<ILogicalOperator> aggChilldOpRef = aggOp.getInputs().get(0);
+        AbstractLogicalOperator aggChildOp = (AbstractLogicalOperator) aggChilldOpRef.getValue();
         // If there's a group by below the agg, then we want to have the agg pushed into the group by
-        if (op3.getOperatorTag() == LogicalOperatorTag.GROUP && !((GroupByOperator) op3).getNestedPlans().isEmpty()) {
-            return false;
-        }
-        if (aggOp.getVariables().size() != 1) {
-            return false;
-        }
-        ILogicalExpression aggExpr = aggOp.getExpressions().get(0).getValue();
-        if (aggExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-            return false;
-        }
-        AbstractFunctionCallExpression origAggFuncExpr = (AbstractFunctionCallExpression) aggExpr;
-        if (origAggFuncExpr.getFunctionIdentifier() != BuiltinFunctions.LISTIFY) {
+        if (aggChildOp.getOperatorTag() == LogicalOperatorTag.GROUP
+                && !((GroupByOperator) aggChildOp).getNestedPlans().isEmpty()) {
             return false;
         }
 
-        LogicalVariable aggVar = aggOp.getVariables().get(0);
-        List<LogicalVariable> used = new LinkedList<LogicalVariable>();
-        VariableUtilities.getUsedVariables(assignOp, used);
-        if (!used.contains(aggVar)) {
+        List<LogicalVariable> assignUsedVars = new ArrayList<>();
+        VariableUtilities.getUsedVariables(assignOp, assignUsedVars);
+
+        List<Mutable<ILogicalExpression>> assignScalarAggExprRefs = new ArrayList<>();
+        List<LogicalVariable> aggAddVars = null;
+        List<Mutable<ILogicalExpression>> aggAddExprs = null;
+
+        for (int i = 0, n = aggOp.getVariables().size(); i < n; i++) {
+            LogicalVariable aggVar = aggOp.getVariables().get(i);
+            Mutable<ILogicalExpression> aggExprRef = aggOp.getExpressions().get(i);
+            ILogicalExpression aggExpr = aggExprRef.getValue();
+            if (aggExpr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
+                continue;
+            }
+            AbstractFunctionCallExpression listifyCandidateExpr = (AbstractFunctionCallExpression) aggExpr;
+            if (listifyCandidateExpr.getFunctionIdentifier() != BuiltinFunctions.LISTIFY) {
+                continue;
+            }
+            if (!assignUsedVars.contains(aggVar)) {
+                continue;
+            }
+            assignScalarAggExprRefs.clear();
+            findScalarAggFuncExprRef(assignOp.getExpressions(), aggVar, assignScalarAggExprRefs);
+            if (assignScalarAggExprRefs.isEmpty()) {
+                continue;
+            }
+            // perform rewrite
+            if (aggAddVars == null) {
+                aggAddVars = new ArrayList<>();
+                aggAddExprs = new ArrayList<>();
+            }
+            for (Mutable<ILogicalExpression> assignScalarAggExprRef : assignScalarAggExprRefs) {
+                AbstractFunctionCallExpression assignScalarAggExpr =
+                        (AbstractFunctionCallExpression) assignScalarAggExprRef.getValue();
+                FunctionIdentifier aggFuncIdent =
+                        BuiltinFunctions.getAggregateFunction(assignScalarAggExpr.getFunctionIdentifier());
+
+                // Push the scalar aggregate function into the aggregate op.
+                int nArgs = assignScalarAggExpr.getArguments().size();
+                List<Mutable<ILogicalExpression>> aggArgs = new ArrayList<>(nArgs);
+                aggArgs.add(
+                        new MutableObject<>(listifyCandidateExpr.getArguments().get(0).getValue().cloneExpression()));
+                aggArgs.addAll(OperatorManipulationUtil
+                        .cloneExpressions(assignScalarAggExpr.getArguments().subList(1, nArgs)));
+                AggregateFunctionCallExpression aggFuncExpr =
+                        BuiltinFunctions.makeAggregateFunctionExpression(aggFuncIdent, aggArgs);
+                aggFuncExpr.setSourceLocation(assignScalarAggExpr.getSourceLocation());
+
+                LogicalVariable newVar = context.newVar();
+                aggAddVars.add(newVar);
+                aggAddExprs.add(new MutableObject<>(aggFuncExpr));
+                // The assign now just "renames" the variable to make sure the upstream plan still works.
+                VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
+                newVarRef.setSourceLocation(assignScalarAggExpr.getSourceLocation());
+                assignScalarAggExprRef.setValue(newVarRef);
+            }
+        }
+
+        if (aggAddVars == null) {
             return false;
         }
 
-        List<Mutable<ILogicalExpression>> srcAssignExprRefs = new LinkedList<Mutable<ILogicalExpression>>();
-        findAggFuncExprRef(assignOp.getExpressions(), aggVar, srcAssignExprRefs);
-        if (srcAssignExprRefs.isEmpty()) {
-            return false;
-        }
+        // add new variables and expressions to the aggregate operator.
+        aggOp.getVariables().addAll(aggAddVars);
+        aggOp.getExpressions().addAll(aggAddExprs);
 
-        AbstractFunctionCallExpression aggOpExpr =
-                (AbstractFunctionCallExpression) aggOp.getExpressions().get(0).getValue();
-        aggOp.getExpressions().clear();
-        aggOp.getVariables().clear();
-
-        for (Mutable<ILogicalExpression> srcAssignExprRef : srcAssignExprRefs) {
-            AbstractFunctionCallExpression assignFuncExpr =
-                    (AbstractFunctionCallExpression) srcAssignExprRef.getValue();
-            FunctionIdentifier aggFuncIdent =
-                    BuiltinFunctions.getAggregateFunction(assignFuncExpr.getFunctionIdentifier());
-
-            // Push the agg func into the agg op.
-
-            List<Mutable<ILogicalExpression>> aggArgs = new ArrayList<Mutable<ILogicalExpression>>();
-            aggArgs.add(aggOpExpr.getArguments().get(0));
-            int sz = assignFuncExpr.getArguments().size();
-            aggArgs.addAll(assignFuncExpr.getArguments().subList(1, sz));
-            AggregateFunctionCallExpression aggFuncExpr =
-                    BuiltinFunctions.makeAggregateFunctionExpression(aggFuncIdent, aggArgs);
-
-            aggFuncExpr.setSourceLocation(assignFuncExpr.getSourceLocation());
-            LogicalVariable newVar = context.newVar();
-            aggOp.getVariables().add(newVar);
-            aggOp.getExpressions().add(new MutableObject<ILogicalExpression>(aggFuncExpr));
-
-            // The assign now just "renames" the variable to make sure the upstream plan still works.
-            VariableReferenceExpression newVarRef = new VariableReferenceExpression(newVar);
-            newVarRef.setSourceLocation(assignFuncExpr.getSourceLocation());
-            srcAssignExprRef.setValue(newVarRef);
-        }
+        // Note: we retain the original listify() call in the aggregate operator because
+        // the variable it is assigned to might be used upstream by other operators.
+        // If the variable is not used upstream then it'll later be removed
+        // by {@code RemoveUnusedAssignAndAggregateRule}
 
         context.computeAndSetTypeEnvironmentForOperator(aggOp);
         context.computeAndSetTypeEnvironmentForOperator(assignOp);
         return true;
     }
 
-    private void findAggFuncExprRef(List<Mutable<ILogicalExpression>> exprRefs, LogicalVariable aggVar,
-            List<Mutable<ILogicalExpression>> srcAssignExprRefs) {
+    private void findScalarAggFuncExprRef(List<Mutable<ILogicalExpression>> exprRefs, LogicalVariable aggVar,
+            List<Mutable<ILogicalExpression>> outScalarAggExprRefs) {
         for (Mutable<ILogicalExpression> exprRef : exprRefs) {
             ILogicalExpression expr = exprRef.getValue();
             if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
@@ -245,10 +259,10 @@
                 FunctionIdentifier funcIdent = BuiltinFunctions.getAggregateFunction(funcExpr.getFunctionIdentifier());
                 if (funcIdent != null
                         && aggVar.equals(SqlppVariableUtil.getVariable(funcExpr.getArguments().get(0).getValue()))) {
-                    srcAssignExprRefs.add(exprRef);
+                    outScalarAggExprRefs.add(exprRef);
                 } else {
                     // Recursively look in func args.
-                    findAggFuncExprRef(funcExpr.getArguments(), aggVar, srcAssignExprRefs);
+                    findScalarAggFuncExprRef(funcExpr.getArguments(), aggVar, outScalarAggExprRefs);
                 }
             }
         }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
index f8457cc..d2ac8e5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushAggregateIntoNestedSubplanRule.java
@@ -19,7 +19,6 @@
 package org.apache.asterix.optimizer.rules;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -208,55 +207,38 @@
     private void collectAggregateVars(Map<LogicalVariable, Integer> nspListifyVarsCount,
             Map<LogicalVariable, AbstractOperatorWithNestedPlans> nspWithAgg,
             Map<LogicalVariable, Integer> nspAggVarToPlanIndex, AbstractOperatorWithNestedPlans op) {
-        List<LogicalVariable> vars = collectOneVarPerAggFromOpWithNestedPlans(op);
-        for (int i = 0; i < vars.size(); i++) {
-            LogicalVariable v = vars.get(i);
-            if (v != null) {
-                nspListifyVarsCount.put(v, 0);
-                nspAggVarToPlanIndex.put(v, i);
-                nspWithAgg.put(v, op);
-            }
-        }
-    }
-
-    private List<LogicalVariable> collectOneVarPerAggFromOpWithNestedPlans(AbstractOperatorWithNestedPlans op) {
         List<ILogicalPlan> nPlans = op.getNestedPlans();
-        if (nPlans == null || nPlans.isEmpty()) {
-            return Collections.emptyList();
+        for (int planIdx = 0, planCount = nPlans.size(); planIdx < planCount; planIdx++) {
+            ILogicalPlan nestedPlan = nPlans.get(planIdx);
+            List<Mutable<ILogicalOperator>> roots = nestedPlan.getRoots();
+            if (roots.size() != 1) {
+                continue;
+            }
+            AbstractLogicalOperator rootOp = (AbstractLogicalOperator) roots.get(0).getValue();
+            if (rootOp.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+                continue;
+            }
+            AggregateOperator agg = (AggregateOperator) rootOp;
+            // TODO: for now we only consider aggregate operators with every expression being listify()
+            // in the future we should check whether this can be relaxed (i.e. if some expressions are listify())
+            boolean everyExprIsListify = agg.getExpressions().stream().map(Mutable::getValue)
+                    .allMatch(expr -> expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL
+                            && ((AbstractFunctionCallExpression) expr).getFunctionIdentifier()
+                                    .equals(BuiltinFunctions.LISTIFY));
+            if (everyExprIsListify) {
+                for (LogicalVariable v : agg.getVariables()) {
+                    nspListifyVarsCount.put(v, 0);
+                    nspAggVarToPlanIndex.put(v, planIdx);
+                    nspWithAgg.put(v, op);
+                }
+            }
         }
-
-        List<LogicalVariable> aggVars = new ArrayList<>();
-        // test that the operator computes a "listify" aggregate
-        for (int i = 0; i < nPlans.size(); i++) {
-            AbstractLogicalOperator topOp = (AbstractLogicalOperator) nPlans.get(i).getRoots().get(0).getValue();
-            if (topOp.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
-                continue;
-            }
-            AggregateOperator agg = (AggregateOperator) topOp;
-            if (agg.getVariables().size() != 1) {
-                continue;
-            }
-            ILogicalExpression expr = agg.getExpressions().get(0).getValue();
-            if (expr.getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) {
-                continue;
-            }
-            AbstractFunctionCallExpression fceAgg = (AbstractFunctionCallExpression) expr;
-            if (fceAgg.getFunctionIdentifier() != BuiltinFunctions.LISTIFY) {
-                continue;
-            }
-            aggVars.add(agg.getVariables().get(0));
-        }
-        return aggVars;
     }
 
     /**
-     * @param exprRef
-     * @param nspWithAgg
-     * @param context
      * @return a pair whose first member is a boolean which is true iff
      *         something was changed in the expression tree rooted at expr. The
      *         second member is the result of transforming expr.
-     * @throws AlgebricksException
      */
     private Pair<Boolean, ILogicalExpression> extractAggFunctionsFromExpression(Mutable<ILogicalExpression> exprRef,
             Map<LogicalVariable, AbstractOperatorWithNestedPlans> nspWithAgg,
@@ -272,9 +254,9 @@
                     if (a1.getExpressionTag() == LogicalExpressionTag.VARIABLE) {
                         LogicalVariable argVar = ((VariableReferenceExpression) a1).getVariableReference();
                         AbstractOperatorWithNestedPlans nspOp = nspWithAgg.get(argVar);
-
                         if (nspOp != null) {
-                            if (!aggregateExprToVarExpr.containsKey(expr)) {
+                            ILogicalExpression varExpr = aggregateExprToVarExpr.get(expr);
+                            if (varExpr == null) {
                                 LogicalVariable newVar = context.newVar();
                                 AggregateFunctionCallExpression aggFun =
                                         BuiltinFunctions.makeAggregateFunctionExpression(fi, fce.getArguments());
@@ -285,8 +267,7 @@
                                 aggregateExprToVarExpr.put(expr, newVarExpr);
                                 return new Pair<>(Boolean.TRUE, newVarExpr);
                             } else {
-                                ILogicalExpression varExpr = aggregateExprToVarExpr.get(expr);
-                                return new Pair<>(Boolean.TRUE, varExpr);
+                                return new Pair<>(Boolean.TRUE, varExpr.cloneExpression());
                             }
                         }
                     }
@@ -296,7 +277,7 @@
                 for (Mutable<ILogicalExpression> a : fce.getArguments()) {
                     Pair<Boolean, ILogicalExpression> aggArg =
                             extractAggFunctionsFromExpression(a, nspWithAgg, aggregateExprToVarExpr, context);
-                    if (aggArg.first.booleanValue()) {
+                    if (aggArg.first) {
                         a.setValue(aggArg.second);
                         change = true;
                     }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
index 21990c9..f968b35 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/RemoveRedundantListifyRule.java
@@ -23,7 +23,6 @@
 import java.util.List;
 import java.util.Set;
 
-import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -45,6 +44,7 @@
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.CardinalityInferenceVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.properties.UnpartitionedPropertyComputer;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
@@ -52,36 +52,53 @@
 /**
  * The rule cancels out redundant pairs of operators unnest-listify aggregate
  * <p>
- * <ul>
- * <li>case 1 (direct):
+ * Case 1 (direct):
  * <p>
  * Before plan:
- * <ul>
- * <li>unnest $x [[ at $p ]] <- function-call:scan-collection($y)
- * <li>aggregate $y <- function-call:listify($z)
- * </ul>
- * <p>
+ * <pre>
+ * unnest $x [[ at $p ]] <- function-call:scan-collection($y)
+ * aggregate $y <- function-call:listify($z)
+ * </pre>
  * After plan:
- * <ul>
- * <li>[[ runningaggregate $p <- tid]]
- * <li>assign $x <- $z
- * </ul>
- * <li>case 2 (reverse):
+ * <pre>
+ * [[ runningaggregate $p <- tid]]
+ * assign $x <- $z
+ * </pre>
+ *
+ * Case 2 (reverse):
  * <p>
  * Before plan:
- * <ul>
- * <li>aggregate $x <- function-call:listify($y)
- * <li>unnest $y <- function-call:scan-collection($z)
- * </ul>
- * <p>
+ * <pre>
+ * aggregate $x <- function-call:listify($y)
+ * unnest $y <- function-call:scan-collection($z)
+ * </pre>
  * After plan:
- * <ul>
- * <li>assign $x <- $z
- * </ul>
- * </ul>
+ * <pre>
+ * assign $x <- $z
+ * </pre>
+ *
+ * Notes regarding Case 2(reverse):
+ * <ol>
+ * <li> Case 2 rewriting is only considered if unnest's input operator has cardinality "exactly 1".
+ *      If unnest's input operator produces 0 tuples then the aggregate operator would produce 1 tuple with $x = [],
+ *      while the assign rewriting would produce 0 tuples. Therefore it's not equivalent.
+ *      If unnest's input operator produces N tuples (where N > 1) then the aggregate operator would produce 1 tuple
+ *      with a concatenated list of all unnested values from all input tuples,
+ *      while the assign rewriting would produce N tuples. Therefore it's not equivalent.
+ *
+ * <li> It's configurable whether Case 2 rewriting is attempted or not if the 'aggregate' operator
+ *      is the root of a subplan's nested plan. The reason for allowing disabling Case 2 is because
+ *      aggregate-unnest elimination may prevent further subplan inlining rewritings.
+ * </ol>
  */
-
 public class RemoveRedundantListifyRule implements IAlgebraicRewriteRule {
+
+    private final boolean allowReverseCaseAtSubplanRoot;
+
+    public RemoveRedundantListifyRule(boolean allowReverseCaseAtSubplanRoot) {
+        this.allowReverseCaseAtSubplanRoot = allowReverseCaseAtSubplanRoot;
+    }
+
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
         return false;
@@ -95,25 +112,29 @@
         if (context.checkIfInDontApplySet(this, op)) {
             return false;
         }
-        Set<LogicalVariable> varSet = new HashSet<LogicalVariable>();
-        return applyRuleDown(opRef, varSet, context);
+        Set<LogicalVariable> varSet = new HashSet<>();
+        return applyRuleDown(opRef, false, varSet, context);
     }
 
-    private boolean applyRuleDown(Mutable<ILogicalOperator> opRef, Set<LogicalVariable> varSet,
+    private boolean applyRuleDown(Mutable<ILogicalOperator> opRef, boolean isSubplanRoot, Set<LogicalVariable> varSet,
             IOptimizationContext context) throws AlgebricksException {
         boolean changed = applies(opRef, varSet, context);
-        changed |= appliesForReverseCase(opRef, varSet, context);
+        boolean skipReverseCase = isSubplanRoot && !allowReverseCaseAtSubplanRoot;
+        if (!skipReverseCase) {
+            changed |= appliesForReverseCase(opRef, varSet, context);
+        }
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
         VariableUtilities.getUsedVariables(op, varSet);
         if (op.hasNestedPlans()) {
+            boolean isSubplanOp = op.getOperatorTag() == LogicalOperatorTag.SUBPLAN;
             // Variables used by the parent operators should be live at op.
-            Set<LogicalVariable> localLiveVars = new ListSet<LogicalVariable>();
+            Set<LogicalVariable> localLiveVars = new ListSet<>();
             VariableUtilities.getLiveVariables(op, localLiveVars);
             varSet.retainAll(localLiveVars);
             AbstractOperatorWithNestedPlans aonp = (AbstractOperatorWithNestedPlans) op;
             for (ILogicalPlan p : aonp.getNestedPlans()) {
                 for (Mutable<ILogicalOperator> r : p.getRoots()) {
-                    if (applyRuleDown(r, varSet, context)) {
+                    if (applyRuleDown(r, isSubplanOp, varSet, context)) {
                         changed = true;
                     }
                     context.addToDontApplySet(this, r.getValue());
@@ -121,7 +142,7 @@
             }
         }
         for (Mutable<ILogicalOperator> i : op.getInputs()) {
-            if (applyRuleDown(i, varSet, context)) {
+            if (applyRuleDown(i, false, varSet, context)) {
                 changed = true;
             }
             context.addToDontApplySet(this, i.getValue());
@@ -205,7 +226,7 @@
         List<Mutable<ILogicalExpression>> assgnExprs = new ArrayList<>(1);
         VariableReferenceExpression paramVarRef = new VariableReferenceExpression(paramVar);
         paramVarRef.setSourceLocation(arg0.getSourceLocation());
-        assgnExprs.add(new MutableObject<ILogicalExpression>(paramVarRef));
+        assgnExprs.add(new MutableObject<>(paramVarRef));
         AssignOperator assign = new AssignOperator(assgnVars, assgnExprs);
         assign.setSourceLocation(agg.getSourceLocation());
         assign.getInputs().add(agg.getInputs().get(0));
@@ -219,13 +240,14 @@
             List<LogicalVariable> raggVars = new ArrayList<>(1);
             raggVars.add(posVar);
             List<Mutable<ILogicalExpression>> rAggExprs = new ArrayList<>(1);
-            StatefulFunctionCallExpression tidFun = new StatefulFunctionCallExpression(
-                    FunctionUtil.getFunctionInfo(BuiltinFunctions.TID), UnpartitionedPropertyComputer.INSTANCE);
+            StatefulFunctionCallExpression tidFun =
+                    new StatefulFunctionCallExpression(BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.TID),
+                            UnpartitionedPropertyComputer.INSTANCE);
             tidFun.setSourceLocation(agg.getSourceLocation());
-            rAggExprs.add(new MutableObject<ILogicalExpression>(tidFun));
+            rAggExprs.add(new MutableObject<>(tidFun));
             RunningAggregateOperator rAgg = new RunningAggregateOperator(raggVars, rAggExprs);
             rAgg.setSourceLocation(agg.getSourceLocation());
-            rAgg.getInputs().add(new MutableObject<ILogicalOperator>(assign));
+            rAgg.getInputs().add(new MutableObject<>(assign));
             aggregateParentRef.setValue(rAgg);
             context.computeAndSetTypeEnvironmentForOperator(rAgg);
         }
@@ -241,7 +263,7 @@
             return false;
         }
         AggregateOperator agg = (AggregateOperator) op1;
-        if (agg.getVariables().size() > 1 || agg.getVariables().size() <= 0) {
+        if (agg.getVariables().size() != 1) {
             return false;
         }
         LogicalVariable aggVar = agg.getVariables().get(0);
@@ -288,12 +310,16 @@
         if (scanFunc.getArguments().size() != 1) {
             return false;
         }
+        ILogicalOperator unnestInputOp = unnest.getInputs().get(0).getValue();
+        if (!CardinalityInferenceVisitor.isCardinalityExactOne(unnestInputOp)) {
+            return false;
+        }
 
         List<LogicalVariable> assgnVars = new ArrayList<>(1);
         assgnVars.add(aggVar);
         AssignOperator assign = new AssignOperator(assgnVars, scanFunc.getArguments());
         assign.setSourceLocation(agg.getSourceLocation());
-        assign.getInputs().add(unnest.getInputs().get(0));
+        assign.getInputs().add(new MutableObject<>(unnestInputOp));
         context.computeAndSetTypeEnvironmentForOperator(assign);
         opRef.setValue(assign);
         return true;
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
index 076783f..d466446 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java
@@ -75,7 +75,7 @@
 
     // Disable ASSIGN_BATCH physical operator if this option is set to 'false'
     public static final String REWRITE_ATTEMPT_BATCH_ASSIGN = "rewrite_attempt_batch_assign";
-    static final boolean REWRITE_ATTEMPT_BATCH_ASSIGN_DEFAULT = false;
+    static final boolean REWRITE_ATTEMPT_BATCH_ASSIGN_DEFAULT = true;
 
     @Override
     protected ILogicalOperatorVisitor<IPhysicalOperator, Boolean> createPhysicalOperatorFactoryVisitor(
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
index b28c15e..09575e2 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/RTreeAccessMethod.java
@@ -266,7 +266,7 @@
                     new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CREATE_MBR));
             createMBR.setSourceLocation(optFuncExpr.getFuncExpr().getSourceLocation());
             // Spatial object is the constant from the func expr we are optimizing.
-            createMBR.getArguments().add(new MutableObject<>(returnedSearchKeyExpr));
+            createMBR.getArguments().add(new MutableObject<>(returnedSearchKeyExpr.cloneExpression()));
             // The number of dimensions
             createMBR.getArguments().add(new MutableObject<ILogicalExpression>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(numDimensions)))));
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
index e7de31d..81b4a3d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineSubplanInputForNestedTupleSourceRule.java
@@ -383,6 +383,11 @@
 
         Mutable<ILogicalOperator> lowestAggregateRefInSubplan =
                 SubplanFlatteningUtil.findLowestAggregate(subplanOp.getNestedPlans().get(0).getRoots().get(0));
+        if (lowestAggregateRefInSubplan == null) {
+            inputOpRef.setValue(inputOpBackup);
+            return new Pair<>(false, new LinkedHashMap<>());
+        }
+
         Mutable<ILogicalOperator> rightInputOpRef = lowestAggregateRefInSubplan.getValue().getInputs().get(0);
         ILogicalOperator rightInputOp = rightInputOpRef.getValue();
 
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
index 956ba6c..09377fa 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanFlatteningUtil.java
@@ -57,13 +57,17 @@
         // For nested subplan, we do not continue for the general inlining.
         if (OperatorManipulationUtil.ancestorOfOperators(subplanOp,
                 ImmutableSet.of(LogicalOperatorTag.NESTEDTUPLESOURCE))) {
-            return new Pair<Map<LogicalVariable, LogicalVariable>, List<Pair<IOrder, Mutable<ILogicalExpression>>>>(
-                    null, null);
+            return new Pair<>(null, null);
         }
-        InlineAllNtsInSubplanVisitor visitor = new InlineAllNtsInSubplanVisitor(context, subplanOp);
+
+        Mutable<ILogicalOperator> topOpRef = findLowestAggregate(subplanOp.getNestedPlans().get(0).getRoots().get(0));
+        if (topOpRef == null) {
+            return new Pair<>(null, null);
+        }
 
         // Rewrites the query plan.
-        ILogicalOperator topOp = findLowestAggregate(subplanOp.getNestedPlans().get(0).getRoots().get(0)).getValue();
+        InlineAllNtsInSubplanVisitor visitor = new InlineAllNtsInSubplanVisitor(context, subplanOp);
+        ILogicalOperator topOp = topOpRef.getValue();
         ILogicalOperator opToVisit = topOp.getInputs().get(0).getValue();
         ILogicalOperator result = opToVisit.accept(visitor, null);
         topOp.getInputs().get(0).setValue(result);
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
index 5b85cb6..d7dcb0c 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/temporal/TranslateIntervalExpressionRule.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.optimizer.rules.temporal;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -47,15 +48,10 @@
  */
 public class TranslateIntervalExpressionRule implements IAlgebraicRewriteRule {
 
-    private static final Set<FunctionIdentifier> TRANSLATABLE_INTERVALS = new HashSet<>();
-    {
-        TRANSLATABLE_INTERVALS.add(BuiltinFunctions.INTERVAL_MEETS);
-        TRANSLATABLE_INTERVALS.add(BuiltinFunctions.INTERVAL_MET_BY);
-        TRANSLATABLE_INTERVALS.add(BuiltinFunctions.INTERVAL_STARTS);
-        TRANSLATABLE_INTERVALS.add(BuiltinFunctions.INTERVAL_STARTED_BY);
-        TRANSLATABLE_INTERVALS.add(BuiltinFunctions.INTERVAL_ENDS);
-        TRANSLATABLE_INTERVALS.add(BuiltinFunctions.INTERVAL_ENDED_BY);
-    }
+    private static final Set<FunctionIdentifier> TRANSLATABLE_INTERVALS =
+            new HashSet<>(Arrays.asList(BuiltinFunctions.INTERVAL_MEETS, BuiltinFunctions.INTERVAL_MET_BY,
+                    BuiltinFunctions.INTERVAL_STARTS, BuiltinFunctions.INTERVAL_STARTED_BY,
+                    BuiltinFunctions.INTERVAL_ENDS, BuiltinFunctions.INTERVAL_ENDED_BY));
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -84,93 +80,94 @@
         if (!hasTranslatableInterval(funcExpr)) {
             return false;
         }
-
-        return translateIntervalExpression(exprRef, funcExpr);
+        ILogicalExpression newExpr = translateIntervalExpression(funcExpr);
+        if (newExpr == null) {
+            return false;
+        }
+        exprRef.setValue(newExpr);
+        return true;
     }
 
     private boolean hasTranslatableInterval(AbstractFunctionCallExpression funcExpr) {
-        if (TRANSLATABLE_INTERVALS.contains(funcExpr.getFunctionIdentifier())) {
-            return true;
-        }
-        return false;
+        return TRANSLATABLE_INTERVALS.contains(funcExpr.getFunctionIdentifier());
     }
 
-    private boolean translateIntervalExpression(Mutable<ILogicalExpression> exprRef,
-            AbstractFunctionCallExpression funcExpr) {
+    private ILogicalExpression translateIntervalExpression(AbstractFunctionCallExpression funcExpr) {
         // All interval relations are translated unless specified in a hint.
         // TODO A new strategy may be needed instead of just a simple translation.
         ILogicalExpression interval1 = funcExpr.getArguments().get(0).getValue();
         ILogicalExpression interval2 = funcExpr.getArguments().get(1).getValue();
         if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_MEETS)) {
-            exprRef.setValue(getEqualExpr(getIntervalEndExpr(interval1), getIntervalStartExpr(interval2)));
+            return getEqualExpr(getIntervalEndExpr(interval1), getIntervalStartExpr(interval2));
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_MET_BY)) {
-            exprRef.setValue(getEqualExpr(getIntervalStartExpr(interval1), getIntervalEndExpr(interval2)));
+            return getEqualExpr(getIntervalStartExpr(interval1), getIntervalEndExpr(interval2));
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_STARTS)) {
             ILogicalExpression startExpr =
                     getEqualExpr(getIntervalStartExpr(interval1), getIntervalStartExpr(interval2));
-            ILogicalExpression endExpr =
-                    getLessThanOrEqualExpr(getIntervalEndExpr(interval1), getIntervalEndExpr(interval2));
-            exprRef.setValue(getAndExpr(startExpr, endExpr));
+            ILogicalExpression endExpr = getLessThanOrEqualExpr(getIntervalEndExpr(interval1.cloneExpression()),
+                    getIntervalEndExpr(interval2.cloneExpression()));
+            return getAndExpr(startExpr, endExpr);
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_STARTED_BY)) {
             ILogicalExpression startExpr =
                     getEqualExpr(getIntervalStartExpr(interval1), getIntervalStartExpr(interval2));
-            ILogicalExpression endExpr =
-                    getLessThanOrEqualExpr(getIntervalEndExpr(interval2), getIntervalEndExpr(interval1));
-            exprRef.setValue(getAndExpr(startExpr, endExpr));
+            ILogicalExpression endExpr = getLessThanOrEqualExpr(getIntervalEndExpr(interval2.cloneExpression()),
+                    getIntervalEndExpr(interval1.cloneExpression()));
+            return getAndExpr(startExpr, endExpr);
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_ENDS)) {
             ILogicalExpression endExpr = getEqualExpr(getIntervalEndExpr(interval1), getIntervalEndExpr(interval2));
-            ILogicalExpression startExpr =
-                    getLessThanOrEqualExpr(getIntervalStartExpr(interval1), getIntervalStartExpr(interval2));
-            exprRef.setValue(getAndExpr(startExpr, endExpr));
+            ILogicalExpression startExpr = getLessThanOrEqualExpr(getIntervalStartExpr(interval1.cloneExpression()),
+                    getIntervalStartExpr(interval2.cloneExpression()));
+            return getAndExpr(startExpr, endExpr);
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_ENDED_BY)) {
             ILogicalExpression endExpr = getEqualExpr(getIntervalEndExpr(interval1), getIntervalEndExpr(interval2));
-            ILogicalExpression startExpr =
-                    getLessThanOrEqualExpr(getIntervalStartExpr(interval2), getIntervalStartExpr(interval1));
-            exprRef.setValue(getAndExpr(startExpr, endExpr));
+            ILogicalExpression startExpr = getLessThanOrEqualExpr(getIntervalStartExpr(interval2.cloneExpression()),
+                    getIntervalStartExpr(interval1.cloneExpression()));
+            return getAndExpr(startExpr, endExpr);
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_BEFORE)) {
-            exprRef.setValue(getLessThanExpr(getIntervalEndExpr(interval1), getIntervalStartExpr(interval2)));
+            return getLessThanExpr(getIntervalEndExpr(interval1), getIntervalStartExpr(interval2));
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_AFTER)) {
-            exprRef.setValue(getGreaterThanExpr(getIntervalStartExpr(interval1), getIntervalEndExpr(interval2)));
+            return getGreaterThanExpr(getIntervalStartExpr(interval1), getIntervalEndExpr(interval2));
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_OVERLAPS)) {
             ILogicalExpression expr1 =
                     getLessThanExpr(getIntervalStartExpr(interval1), getIntervalStartExpr(interval2));
-            ILogicalExpression expr2 = getGreaterThanExpr(getIntervalEndExpr(interval2), getIntervalEndExpr(interval1));
-            ILogicalExpression expr3 =
-                    getGreaterThanExpr(getIntervalEndExpr(interval1), getIntervalStartExpr(interval2));
-            exprRef.setValue(getAndExpr(getAndExpr(expr1, expr2), expr3));
+            ILogicalExpression expr2 = getGreaterThanExpr(getIntervalEndExpr(interval2.cloneExpression()),
+                    getIntervalEndExpr(interval1.cloneExpression()));
+            ILogicalExpression expr3 = getGreaterThanExpr(getIntervalEndExpr(interval1.cloneExpression()),
+                    getIntervalStartExpr(interval2.cloneExpression()));
+            return getAndExpr(getAndExpr(expr1, expr2), expr3);
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_OVERLAPPED_BY)) {
             ILogicalExpression expr1 =
                     getLessThanExpr(getIntervalStartExpr(interval2), getIntervalStartExpr(interval1));
-            ILogicalExpression expr2 = getGreaterThanExpr(getIntervalEndExpr(interval1), getIntervalEndExpr(interval2));
-            ILogicalExpression expr3 =
-                    getGreaterThanExpr(getIntervalEndExpr(interval2), getIntervalStartExpr(interval1));
-            exprRef.setValue(getAndExpr(getAndExpr(expr1, expr2), expr3));
+            ILogicalExpression expr2 = getGreaterThanExpr(getIntervalEndExpr(interval1.cloneExpression()),
+                    getIntervalEndExpr(interval2.cloneExpression()));
+            ILogicalExpression expr3 = getGreaterThanExpr(getIntervalEndExpr(interval2.cloneExpression()),
+                    getIntervalStartExpr(interval1.cloneExpression()));
+            return getAndExpr(getAndExpr(expr1, expr2), expr3);
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_OVERLAPPING)) {
             ILogicalExpression startExpr =
                     getLessThanOrEqualExpr(getIntervalStartExpr(interval1), getIntervalEndExpr(interval2));
-            ILogicalExpression endExpr =
-                    getGreaterThanOrEqualExpr(getIntervalEndExpr(interval1), getIntervalStartExpr(interval2));
-            ILogicalExpression startPointExpr =
-                    getNotEqualExpr(getIntervalEndExpr(interval1), getIntervalStartExpr(interval2));
-            ILogicalExpression endPointExpr =
-                    getNotEqualExpr(getIntervalStartExpr(interval1), getIntervalEndExpr(interval2));
-            exprRef.setValue(getAndExpr(getAndExpr(startExpr, endExpr), getAndExpr(startPointExpr, endPointExpr)));
+            ILogicalExpression endExpr = getGreaterThanOrEqualExpr(getIntervalEndExpr(interval1.cloneExpression()),
+                    getIntervalStartExpr(interval2.cloneExpression()));
+            ILogicalExpression startPointExpr = getNotEqualExpr(getIntervalEndExpr(interval1.cloneExpression()),
+                    getIntervalStartExpr(interval2.cloneExpression()));
+            ILogicalExpression endPointExpr = getNotEqualExpr(getIntervalStartExpr(interval1.cloneExpression()),
+                    getIntervalEndExpr(interval2.cloneExpression()));
+            return getAndExpr(getAndExpr(startExpr, endExpr), getAndExpr(startPointExpr, endPointExpr));
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_COVERS)) {
             ILogicalExpression startExpr =
                     getLessThanOrEqualExpr(getIntervalStartExpr(interval1), getIntervalStartExpr(interval2));
-            ILogicalExpression endExpr =
-                    getGreaterThanOrEqualExpr(getIntervalEndExpr(interval1), getIntervalEndExpr(interval2));
-            exprRef.setValue(getAndExpr(startExpr, endExpr));
+            ILogicalExpression endExpr = getGreaterThanOrEqualExpr(getIntervalEndExpr(interval1.cloneExpression()),
+                    getIntervalEndExpr(interval2.cloneExpression()));
+            return getAndExpr(startExpr, endExpr);
         } else if (funcExpr.getFunctionIdentifier().equals(BuiltinFunctions.INTERVAL_COVERED_BY)) {
             ILogicalExpression startExpr =
                     getLessThanOrEqualExpr(getIntervalStartExpr(interval2), getIntervalStartExpr(interval1));
-            ILogicalExpression endExpr =
-                    getGreaterThanOrEqualExpr(getIntervalEndExpr(interval2), getIntervalEndExpr(interval1));
-            exprRef.setValue(getAndExpr(startExpr, endExpr));
+            ILogicalExpression endExpr = getGreaterThanOrEqualExpr(getIntervalEndExpr(interval2.cloneExpression()),
+                    getIntervalEndExpr(interval1.cloneExpression()));
+            return getAndExpr(startExpr, endExpr);
         } else {
-            return false;
+            return null;
         }
-        return true;
     }
 
     private ILogicalExpression getAndExpr(ILogicalExpression arg1, ILogicalExpression arg2) {
@@ -211,7 +208,7 @@
 
     private ILogicalExpression getScalarExpr(FunctionIdentifier func, ILogicalExpression interval) {
         List<Mutable<ILogicalExpression>> intervalArg = new ArrayList<>();
-        intervalArg.add(new MutableObject<ILogicalExpression>(interval));
+        intervalArg.add(new MutableObject<>(interval));
         ScalarFunctionCallExpression fnExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
         fnExpr.setSourceLocation(interval.getSourceLocation());
@@ -221,8 +218,8 @@
     private ILogicalExpression getScalarExpr(FunctionIdentifier func, ILogicalExpression interval1,
             ILogicalExpression interval2) {
         List<Mutable<ILogicalExpression>> intervalArg = new ArrayList<>();
-        intervalArg.add(new MutableObject<ILogicalExpression>(interval1));
-        intervalArg.add(new MutableObject<ILogicalExpression>(interval2));
+        intervalArg.add(new MutableObject<>(interval1));
+        intervalArg.add(new MutableObject<>(interval2));
         ScalarFunctionCallExpression fnExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(func), intervalArg);
         fnExpr.setSourceLocation(interval1.getSourceLocation());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
index dbe1c9e..e10f9fb 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/IntervalJoinUtils.java
@@ -216,16 +216,22 @@
     private static void insertPartitionSortKey(AbstractBinaryJoinOperator op, int branch,
             List<LogicalVariable> partitionVars, LogicalVariable intervalVar, IOptimizationContext context)
             throws AlgebricksException {
-        Mutable<ILogicalExpression> intervalExp = new MutableObject<>(new VariableReferenceExpression(intervalVar));
-
         List<Mutable<ILogicalExpression>> assignExps = new ArrayList<>();
         // Start partition
+        VariableReferenceExpression intervalVarRef1 = new VariableReferenceExpression(intervalVar);
+        intervalVarRef1.setSourceLocation(op.getSourceLocation());
         IFunctionInfo startFi = FunctionUtil.getFunctionInfo(BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_START);
-        ScalarFunctionCallExpression startPartitionExp = new ScalarFunctionCallExpression(startFi, intervalExp);
+        ScalarFunctionCallExpression startPartitionExp =
+                new ScalarFunctionCallExpression(startFi, new MutableObject<>(intervalVarRef1));
+        startPartitionExp.setSourceLocation(op.getSourceLocation());
         assignExps.add(new MutableObject<>(startPartitionExp));
         // End partition
+        VariableReferenceExpression intervalVarRef2 = new VariableReferenceExpression(intervalVar);
+        intervalVarRef2.setSourceLocation(op.getSourceLocation());
         IFunctionInfo endFi = FunctionUtil.getFunctionInfo(BuiltinFunctions.ACCESSOR_TEMPORAL_INTERVAL_END);
-        ScalarFunctionCallExpression endPartitionExp = new ScalarFunctionCallExpression(endFi, intervalExp);
+        ScalarFunctionCallExpression endPartitionExp =
+                new ScalarFunctionCallExpression(endFi, new MutableObject<>(intervalVarRef2));
+        endPartitionExp.setSourceLocation(op.getSourceLocation());
         assignExps.add(new MutableObject<>(endPartitionExp));
 
         AssignOperator ao = new AssignOperator(partitionVars, assignExps);
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 24e1db8..fd689a5 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
@@ -1972,10 +1972,10 @@
     protected Mutable<ILogicalExpression> generateAndNotIsUnknownWrap(ILogicalExpression logicalExpr) {
         SourceLocation sourceLoc = logicalExpr.getSourceLocation();
         List<Mutable<ILogicalExpression>> arguments = new ArrayList<>();
-        arguments.add(new MutableObject<>(logicalExpr));
+        arguments.add(new MutableObject<>(logicalExpr.cloneExpression()));
         ScalarFunctionCallExpression isUnknownExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.IS_UNKNOWN),
-                        new ArrayList<>(Collections.singletonList(new MutableObject<>(logicalExpr))));
+                        new ArrayList<>(Collections.singletonList(new MutableObject<>(logicalExpr.cloneExpression()))));
         isUnknownExpr.setSourceLocation(sourceLoc);
         ScalarFunctionCallExpression notExpr =
                 new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.NOT),
diff --git a/asterixdb/asterix-app/pom.xml b/asterixdb/asterix-app/pom.xml
index 5d65cb7..34bab13 100644
--- a/asterixdb/asterix-app/pom.xml
+++ b/asterixdb/asterix-app/pom.xml
@@ -422,7 +422,7 @@
       <id>asterix-gerrit-asterix-app-sql-execution</id>
       <properties>
         <test.excludes>**/*.java</test.excludes>
-        <itest.includes>**/SqlppExecution*IT.java,**/ExternalPythonFunction*IT.java</itest.includes>
+        <itest.includes>**/SqlppExecution*IT.java,**/ExternalPythonFunctionIT.java</itest.includes>
         <failIfNoTests>false</failIfNoTests>
       </properties>
     </profile>
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
index 54e972e..a234b9f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractNCUdfServlet.java
@@ -22,40 +22,46 @@
 import static org.apache.asterix.common.functions.ExternalFunctionLanguage.JAVA;
 import static org.apache.asterix.common.functions.ExternalFunctionLanguage.PYTHON;
 
+import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
 import java.nio.file.Files;
 import java.nio.file.OpenOption;
 import java.nio.file.Path;
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
-import org.apache.asterix.common.library.LibraryDescriptor;
 import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.commons.io.IOUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.IFormattedException;
 import org.apache.hyracks.control.common.work.SynchronizableWork;
 import org.apache.hyracks.control.nc.NodeControllerService;
-import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.AbstractServlet;
 import org.apache.hyracks.http.server.utils.HttpUtil;
 
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.HttpScheme;
+import io.netty.handler.codec.http.multipart.FileUpload;
+import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
+import io.netty.handler.codec.http.multipart.InterfaceHttpData;
+import io.netty.handler.codec.http.multipart.MixedAttribute;
 
 public abstract class AbstractNCUdfServlet extends AbstractServlet {
 
+    private final IParserFactory parserFactory;
     INcApplicationContext appCtx;
     INCServiceContext srvCtx;
 
@@ -63,13 +69,81 @@
     private final HttpScheme httpServerProtocol;
     private final int httpServerPort;
 
+    public static final String GET_UDF_DIST_ENDPOINT = "/dist";
+    public static final String DATAVERSE_PARAMETER = "dataverse";
+    public static final String NAME_PARAMETER = "name";
+    public static final String TYPE_PARAMETER = "type";
+    public static final String DELETE_PARAMETER = "delete";
+    public static final String IFEXISTS_PARAMETER = "ifexists";
+    public static final String DATA_PARAMETER = "data";
+
+    protected enum LibraryOperation {
+        UPSERT,
+        DELETE
+    }
+
+    protected final static class LibraryUploadData {
+
+        final LibraryOperation op;
+        final DataverseName dataverse;
+        final String name;
+        final ExternalFunctionLanguage type;
+        final boolean replaceIfExists;
+        final FileUpload fileUpload;
+
+        private LibraryUploadData(LibraryOperation op, List<InterfaceHttpData> dataverse, MixedAttribute name,
+                MixedAttribute type, boolean replaceIfExists, InterfaceHttpData fileUpload) throws IOException {
+            this.op = op;
+            List<String> dataverseParts = new ArrayList<>(dataverse.size());
+            for (InterfaceHttpData attr : dataverse) {
+                dataverseParts.add(((MixedAttribute) attr).getValue());
+            }
+            this.dataverse = DataverseName.create(dataverseParts);
+            this.name = name.getValue();
+            this.type = type != null ? getLanguageByTypeParameter(type.getValue()) : null;
+            this.replaceIfExists = replaceIfExists;
+            this.fileUpload = (FileUpload) fileUpload;
+        }
+
+        private LibraryUploadData(LibraryOperation op, DataverseName dataverse, MixedAttribute name,
+                MixedAttribute type, boolean replaceIfExists, InterfaceHttpData fileUpload) throws IOException {
+            this.op = op;
+            this.dataverse = dataverse;
+            this.name = name.getValue();
+            this.type = type != null ? getLanguageByTypeParameter(type.getValue()) : null;
+            this.replaceIfExists = replaceIfExists;
+            this.fileUpload = (FileUpload) fileUpload;
+        }
+
+        public static LibraryUploadData libraryCreationUploadData(List<InterfaceHttpData> dataverse,
+                MixedAttribute name, MixedAttribute type, InterfaceHttpData fileUpload) throws IOException {
+            return new LibraryUploadData(LibraryOperation.UPSERT, dataverse, name, type, true, fileUpload);
+        }
+
+        public static LibraryUploadData libraryDeletionUploadData(List<InterfaceHttpData> dataverse,
+                MixedAttribute name, boolean replaceIfExists) throws IOException {
+            return new LibraryUploadData(LibraryOperation.DELETE, dataverse, name, null, replaceIfExists, null);
+        }
+
+        public static LibraryUploadData libraryCreationUploadData(DataverseName dataverse, MixedAttribute name,
+                MixedAttribute type, InterfaceHttpData fileUpload) throws IOException {
+            return new LibraryUploadData(LibraryOperation.UPSERT, dataverse, name, type, true, fileUpload);
+        }
+
+        public static LibraryUploadData libraryDeletionUploadData(DataverseName dataverse, MixedAttribute name,
+                boolean replaceIfExists) throws IOException {
+            return new LibraryUploadData(LibraryOperation.DELETE, dataverse, name, null, replaceIfExists, null);
+        }
+    }
+
     public AbstractNCUdfServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
-            HttpScheme httpServerProtocol, int httpServerPort) {
+            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
 
         super(ctx, paths);
         this.plainAppCtx = appCtx;
         this.httpServerProtocol = httpServerProtocol;
         this.httpServerPort = httpServerPort;
+        this.parserFactory = compilationProvider.getParserFactory();
     }
 
     void readFromFile(Path filePath, IServletResponse response, String contentType, OpenOption opt) throws Exception {
@@ -103,7 +177,7 @@
     }
 
     URI createDownloadURI(Path file) throws Exception {
-        String path = paths[0].substring(0, trims[0]) + '/' + file.getFileName();
+        String path = paths[0].substring(0, trims[0]) + GET_UDF_DIST_ENDPOINT + '/' + file.getFileName();
         String host = getHyracksClientConnection().getHost();
         return new URI(httpServerProtocol.toString(), null, host, httpServerPort, path, null, null);
     }
@@ -116,25 +190,105 @@
         return hcc;
     }
 
-    Pair<DataverseName, String> parseLibraryName(IServletRequest request) throws IllegalArgumentException {
-        String[] path = StringUtils.split(localPath(request), '/');
-        int ln = path.length;
-        if (ln < 2) {
-            return null;
-        }
-        String libraryName = path[ln - 1];
-        DataverseName dataverseName = DataverseName.create(Arrays.asList(path), 0, ln - 1);
-        return new Pair<>(dataverseName, libraryName);
+    protected String getDisplayFormDataverseParameter() {
+        return null;
     }
 
-    static ExternalFunctionLanguage getLanguageByFileExtension(String fileExtension) {
-        switch (fileExtension) {
-            case LibraryDescriptor.FILE_EXT_ZIP:
-                return JAVA;
-            case LibraryDescriptor.FILE_EXT_PYZ:
-                return PYTHON;
-            default:
-                return null;
+    protected String getDataverseParameter() {
+        return DATAVERSE_PARAMETER;
+    }
+
+    private boolean isNotAttribute(InterfaceHttpData field) {
+        return field == null || !field.getHttpDataType().equals(InterfaceHttpData.HttpDataType.Attribute);
+    }
+
+    private boolean areNotAttributes(List<InterfaceHttpData> fields) {
+        return fields == null || fields.stream().map(InterfaceHttpData::getHttpDataType)
+                .anyMatch(httpDataType -> !httpDataType.equals(InterfaceHttpData.HttpDataType.Attribute));
+    }
+
+    protected LibraryUploadData decodeMultiPartLibraryOptions(HttpPostRequestDecoder requestDecoder)
+            throws IOException, CompilationException {
+        List<InterfaceHttpData> dataverseAttributeParts = requestDecoder.getBodyHttpDatas(DATAVERSE_PARAMETER);
+        InterfaceHttpData displayFormDataverseAttribute = null;
+        if (getDisplayFormDataverseParameter() != null) {
+            displayFormDataverseAttribute = requestDecoder.getBodyHttpData(getDisplayFormDataverseParameter());
+        }
+        if (displayFormDataverseAttribute != null && dataverseAttributeParts != null) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME,
+                    getDisplayFormDataverseParameter(), getDataverseParameter());
+        }
+        InterfaceHttpData nameAtrribute = requestDecoder.getBodyHttpData(NAME_PARAMETER);
+        InterfaceHttpData typeAttribute = requestDecoder.getBodyHttpData(TYPE_PARAMETER);
+        InterfaceHttpData deleteAttribute = requestDecoder.getBodyHttpData(DELETE_PARAMETER);
+        InterfaceHttpData replaceIfExistsAttribute = requestDecoder.getBodyHttpData(IFEXISTS_PARAMETER);
+        if ((isNotAttribute(displayFormDataverseAttribute)) && (areNotAttributes(dataverseAttributeParts))) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, getDataverseParameter());
+        } else if (isNotAttribute(nameAtrribute)) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, NAME_PARAMETER);
+        } else if ((typeAttribute == null && deleteAttribute == null)) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED,
+                    TYPE_PARAMETER + " or " + DELETE_PARAMETER);
+        } else if (typeAttribute != null && deleteAttribute != null) {
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_NOT_ALLOWED_AT_SAME_TIME, TYPE_PARAMETER,
+                    DELETE_PARAMETER);
+        }
+
+        if (!isNotAttribute(deleteAttribute)) {
+            boolean replace = false;
+            if (replaceIfExistsAttribute != null) {
+                replace = Boolean.TRUE.toString()
+                        .equalsIgnoreCase(((MixedAttribute) replaceIfExistsAttribute).getValue());
+            }
+            if (displayFormDataverseAttribute == null) {
+                return LibraryUploadData.libraryDeletionUploadData(dataverseAttributeParts,
+                        (MixedAttribute) nameAtrribute, replace);
+            } else {
+                DataverseName dataverseName = DataverseName
+                        .create(parserFactory.createParser(((MixedAttribute) displayFormDataverseAttribute).getValue())
+                                .parseMultipartIdentifier());
+                return LibraryUploadData.libraryDeletionUploadData(dataverseName, (MixedAttribute) nameAtrribute,
+                        replace);
+            }
+        } else if (!isNotAttribute(typeAttribute)) {
+            InterfaceHttpData libraryData = requestDecoder.getBodyHttpData(DATA_PARAMETER);
+            if (libraryData == null) {
+                throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, DATA_PARAMETER);
+            } else if (!libraryData.getHttpDataType().equals(InterfaceHttpData.HttpDataType.FileUpload)) {
+                throw RuntimeDataException.create(ErrorCode.INVALID_REQ_PARAM_VAL, DATA_PARAMETER,
+                        libraryData.getHttpDataType());
+            }
+            LibraryUploadData uploadData;
+            if (displayFormDataverseAttribute == null) {
+                uploadData = LibraryUploadData.libraryCreationUploadData(dataverseAttributeParts,
+                        (MixedAttribute) nameAtrribute, (MixedAttribute) typeAttribute, libraryData);
+            } else {
+                DataverseName dataverseName = DataverseName
+                        .create(parserFactory.createParser(((MixedAttribute) displayFormDataverseAttribute).getValue())
+                                .parseMultipartIdentifier());
+                uploadData = LibraryUploadData.libraryCreationUploadData(dataverseName, (MixedAttribute) nameAtrribute,
+                        (MixedAttribute) typeAttribute, libraryData);
+            }
+            if (uploadData.type == null) {
+                throw RuntimeDataException.create(ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_KIND,
+                        ((MixedAttribute) typeAttribute).getValue());
+            }
+            return uploadData;
+        } else {
+            if (!typeAttribute.getHttpDataType().equals(InterfaceHttpData.HttpDataType.Attribute)) {
+                throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, TYPE_PARAMETER);
+            }
+            throw RuntimeDataException.create(ErrorCode.PARAMETERS_REQUIRED, DELETE_PARAMETER);
+        }
+    }
+
+    static ExternalFunctionLanguage getLanguageByTypeParameter(String lang) {
+        if (lang.equalsIgnoreCase(JAVA.name())) {
+            return JAVA;
+        } else if (lang.equalsIgnoreCase(PYTHON.name())) {
+            return PYTHON;
+        } else {
+            return null;
         }
     }
 
@@ -142,9 +296,15 @@
         if (IFormattedException.matchesAny(e, ErrorCode.UNKNOWN_DATAVERSE, ErrorCode.UNKNOWN_LIBRARY)) {
             return HttpResponseStatus.NOT_FOUND;
         }
+        if (IFormattedException.matchesAny(e, ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNKNOWN_KIND,
+                ErrorCode.LIBRARY_EXTERNAL_FUNCTION_UNSUPPORTED_KIND, ErrorCode.INVALID_REQ_PARAM_VAL,
+                ErrorCode.PARAMETERS_REQUIRED)) {
+            return HttpResponseStatus.BAD_REQUEST;
+        }
         if (e instanceof AlgebricksException) {
             return HttpResponseStatus.BAD_REQUEST;
         }
         return HttpResponseStatus.INTERNAL_SERVER_ERROR;
     }
+
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
index e062cdc..9dc971a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/BasicAuthServlet.java
@@ -130,7 +130,7 @@
             return false;
         }
         String providedUsername = providedCredentials[0];
-        String storedPw = getStoredCredentials(request).get(providedUsername);
+        String storedPw = getCredential(providedUsername, request);
         if (storedPw == null) {
             LOGGER.debug("Invalid username");
             return false;
@@ -144,8 +144,15 @@
         }
     }
 
-    protected Map<String, String> getStoredCredentials(IServletRequest request) {
-        return request.getHttpRequest().method().equals(HttpMethod.GET) ? ephemeralCredentials : storedCredentials;
+    private String getCredential(String username, IServletRequest request) {
+        String credential = storedCredentials.get(username);
+        if (credential != null) {
+            return credential;
+        } else if (request != null && request.getHttpRequest().method().equals(HttpMethod.GET)) {
+            return ephemeralCredentials.get(username);
+        } else {
+            return null;
+        }
     }
 
     public static String hashPassword(String password) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
index 0645870..f164938 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfApiServlet.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.api.http.server;
 
 import static org.apache.asterix.api.http.server.ServletConstants.SYS_AUTH_HEADER;
+import static org.apache.asterix.common.library.LibraryDescriptor.FIELD_HASH;
 
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -31,7 +32,10 @@
 import java.nio.file.Paths;
 import java.security.DigestOutputStream;
 import java.security.MessageDigest;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
@@ -44,6 +48,7 @@
 import org.apache.asterix.common.api.IReceptionist;
 import org.apache.asterix.common.api.IRequestReference;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
+import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.MessageFuture;
@@ -53,21 +58,22 @@
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.FilenameUtils;
 import org.apache.commons.io.IOUtils;
-import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.utils.HttpUtil;
+import org.apache.hyracks.util.JSONUtil;
 import org.apache.hyracks.util.file.FileUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+
 import io.netty.buffer.ByteBufInputStream;
 import io.netty.handler.codec.http.HttpRequest;
 import io.netty.handler.codec.http.HttpResponseStatus;
 import io.netty.handler.codec.http.HttpScheme;
-import io.netty.handler.codec.http.multipart.FileUpload;
 import io.netty.handler.codec.http.multipart.HttpPostRequestDecoder;
-import io.netty.handler.codec.http.multipart.InterfaceHttpData;
 
 public class NCUdfApiServlet extends AbstractNCUdfServlet {
 
@@ -76,12 +82,13 @@
 
     protected Path workingDir;
     protected String sysAuthHeader;
+    private ILibraryManager libraryManager;
 
     private static final Logger LOGGER = LogManager.getLogger();
 
     public NCUdfApiServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
             ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
-        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
+        super(ctx, paths, appCtx, compilationProvider, httpServerProtocol, httpServerPort);
         this.compilationProvider = compilationProvider;
         this.receptionist = appCtx.getReceptionist();
     }
@@ -89,6 +96,7 @@
     @Override
     public void init() throws IOException {
         appCtx = (INcApplicationContext) plainAppCtx;
+        this.libraryManager = appCtx.getLibraryManager();
         srvCtx = this.appCtx.getServiceContext();
         workingDir = Paths.get(appCtx.getLibraryManager().getDistributionDir().getAbsolutePath()).normalize();
         initAuth();
@@ -119,27 +127,26 @@
 
     private void doCreate(DataverseName dataverseName, String libraryName, ExternalFunctionLanguage language,
             String hash, URI downloadURI, boolean replaceIfExists, String sysAuthHeader,
-            IRequestReference requestReference, IServletRequest request, IServletResponse response) throws Exception {
+            IRequestReference requestReference, IServletRequest request) throws Exception {
         INCMessageBroker ncMb = (INCMessageBroker) srvCtx.getMessageBroker();
         MessageFuture responseFuture = ncMb.registerMessageFuture();
         CreateLibraryRequestMessage req = new CreateLibraryRequestMessage(srvCtx.getNodeId(),
                 responseFuture.getFutureId(), dataverseName, libraryName, language, hash, downloadURI, replaceIfExists,
                 sysAuthHeader, requestReference, additionalHttpHeadersFromRequest(request));
-        sendMessage(req, responseFuture, requestReference, request, response);
+        sendMessage(req, responseFuture);
     }
 
     private void doDrop(DataverseName dataverseName, String libraryName, boolean replaceIfExists,
-            IRequestReference requestReference, IServletRequest request, IServletResponse response) throws Exception {
+            IRequestReference requestReference, IServletRequest request) throws Exception {
         INCMessageBroker ncMb = (INCMessageBroker) srvCtx.getMessageBroker();
         MessageFuture responseFuture = ncMb.registerMessageFuture();
         DropLibraryRequestMessage req =
                 new DropLibraryRequestMessage(srvCtx.getNodeId(), responseFuture.getFutureId(), dataverseName,
                         libraryName, replaceIfExists, requestReference, additionalHttpHeadersFromRequest(request));
-        sendMessage(req, responseFuture, requestReference, request, response);
+        sendMessage(req, responseFuture);
     }
 
-    private void sendMessage(ICcAddressedMessage requestMessage, MessageFuture responseFuture,
-            IRequestReference requestReference, IServletRequest request, IServletResponse response) throws Exception {
+    private void sendMessage(ICcAddressedMessage requestMessage, MessageFuture responseFuture) throws Exception {
         // Running on NC -> send 'execute' message to CC
         INCMessageBroker ncMb = (INCMessageBroker) srvCtx.getMessageBroker();
         InternalRequestResponse responseMsg;
@@ -165,107 +172,125 @@
     @Override
     protected void get(IServletRequest request, IServletResponse response) throws Exception {
         String localPath = localPath(request);
-        while (localPath.startsWith("/")) {
-            localPath = localPath.substring(1);
+        try {
+            if (localPath.equals("/") || localPath.equals("")) {
+                //TODO: nicer way to get this into display form?
+                Map<DataverseName, Map<String, String>> dvToLibHashes =
+                        ExternalLibraryUtils.produceLibraryListing(libraryManager);
+                List<Map<String, Object>> libraryList = new ArrayList<>();
+                for (Map.Entry<DataverseName, Map<String, String>> dvAndLibs : dvToLibHashes.entrySet()) {
+                    for (Map.Entry<String, String> libsInDv : dvAndLibs.getValue().entrySet()) {
+                        Map<String, Object> libraryEntry = new HashMap<>();
+                        List<String> dvParts = dvAndLibs.getKey().getParts();
+                        String dvKey = getDisplayFormDataverseParameter() == null ? getDataverseParameter()
+                                : getDisplayFormDataverseParameter();
+                        libraryEntry.put(dvKey, dvParts.size() > 1 ? dvParts : dvAndLibs.getKey().toString());
+                        libraryEntry.put(NAME_PARAMETER, libsInDv.getKey());
+                        libraryEntry.put(FIELD_HASH, libsInDv.getValue());
+                        libraryList.add(libraryEntry);
+                    }
+                }
+                JsonNode libraryListing = OBJECT_MAPPER.valueToTree(libraryList);
+                response.setStatus(HttpResponseStatus.OK);
+                HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
+                PrintWriter responseWriter = response.writer();
+                JSONUtil.writeNode(responseWriter, libraryListing);
+                responseWriter.flush();
+            } else if (localPath(request).startsWith(GET_UDF_DIST_ENDPOINT)) {
+                localPath = localPath(request).substring(GET_UDF_DIST_ENDPOINT.length());
+                while (localPath.startsWith("/")) {
+                    localPath = localPath.substring(1);
+                }
+                if (localPath.isEmpty()) {
+                    response.setStatus(HttpResponseStatus.BAD_REQUEST);
+                    return;
+                }
+                Path filePath = workingDir.resolve(localPath).normalize();
+                if (!filePath.startsWith(workingDir)) {
+                    response.setStatus(HttpResponseStatus.BAD_REQUEST);
+                    return;
+                }
+                readFromFile(filePath, response, HttpUtil.ContentType.APPLICATION_OCTET_STREAM, null);
+            } else {
+                response.setStatus(HttpResponseStatus.NOT_FOUND);
+            }
+        } catch (Exception e) {
+            response.setStatus(toHttpErrorStatus(e));
+            PrintWriter responseWriter = response.writer();
+            Map<String, String> error = Collections.singletonMap("error", e.getMessage());
+            String errorJson = "";
+            try {
+                errorJson = OBJECT_MAPPER.writeValueAsString(error);
+            } catch (JsonProcessingException ex) {
+                responseWriter.write("{ \"error\": \"Unable to process error message!\" }");
+            }
+            responseWriter.write(errorJson);
+            responseWriter.flush();
+            LOGGER.error("Error reading library", e);
         }
-        if (localPath.isEmpty()) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
-        Path filePath = workingDir.resolve(localPath).normalize();
-        if (!filePath.startsWith(workingDir)) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
-        readFromFile(filePath, response, HttpUtil.ContentType.APPLICATION_OCTET_STREAM, null);
     }
 
     @Override
     protected void post(IServletRequest request, IServletResponse response) {
         HttpRequest httpRequest = request.getHttpRequest();
-        Pair<DataverseName, String> libraryName = parseLibraryName(request);
-        if (libraryName == null) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
         Path libraryTempFile = null;
         FileOutputStream libTmpOut = null;
         HttpPostRequestDecoder requestDecoder = new HttpPostRequestDecoder(httpRequest);
         try {
-            if (!requestDecoder.hasNext() || requestDecoder.getBodyHttpDatas().size() != 1) {
-                response.setStatus(HttpResponseStatus.BAD_REQUEST);
-                return;
-            }
-            InterfaceHttpData httpData = requestDecoder.getBodyHttpDatas().get(0);
-            if (!httpData.getHttpDataType().equals(InterfaceHttpData.HttpDataType.FileUpload)) {
-                response.setStatus(HttpResponseStatus.BAD_REQUEST);
-                return;
-            }
-            FileUpload fileUpload = (FileUpload) httpData;
-            String fileExt = FilenameUtils.getExtension(fileUpload.getFilename());
-            ExternalFunctionLanguage language = getLanguageByFileExtension(fileExt);
-            if (language == null) {
-                response.setStatus(HttpResponseStatus.BAD_REQUEST);
-                return;
-            }
-            try {
-                IRequestReference requestReference = receptionist.welcome(request);
+            LibraryUploadData uploadData = decodeMultiPartLibraryOptions(requestDecoder);
+            IRequestReference requestReference = receptionist.welcome(request);
+            if (uploadData.op == LibraryOperation.UPSERT) {
+                ExternalFunctionLanguage language = uploadData.type;
+                String fileExt = FilenameUtils.getExtension(uploadData.fileUpload.getFilename());
                 libraryTempFile = Files.createTempFile(workingDir, "lib_", '.' + fileExt);
                 if (LOGGER.isDebugEnabled()) {
-                    LOGGER.debug("Created temporary file " + libraryTempFile + " for library " + libraryName.first + "."
-                            + libraryName.second);
+                    LOGGER.debug("Created temporary file " + libraryTempFile + " for library " + uploadData.dataverse
+                            + "." + uploadData.name);
                 }
                 MessageDigest digest = MessageDigest.getInstance("MD5");
                 libTmpOut = new FileOutputStream(libraryTempFile.toFile());
-                OutputStream outStream = new DigestOutputStream(libTmpOut, digest);
-                InputStream uploadInput = new ByteBufInputStream(((FileUpload) httpData).getByteBuf());
-                IOUtils.copyLarge(uploadInput, outStream);
-                outStream.close();
+                try (OutputStream os = new DigestOutputStream(libTmpOut, digest);
+                        InputStream ui = new ByteBufInputStream((uploadData.fileUpload).getByteBuf())) {
+                    IOUtils.copyLarge(ui, os);
+                }
                 URI downloadURI = createDownloadURI(libraryTempFile);
-                doCreate(libraryName.first, libraryName.second, language,
+                doCreate(uploadData.dataverse, uploadData.name, language,
                         ExternalLibraryUtils.digestToHexString(digest), downloadURI, true, sysAuthHeader,
-                        requestReference, request, response);
-                response.setStatus(HttpResponseStatus.OK);
-            } catch (Exception e) {
-                response.setStatus(toHttpErrorStatus(e));
-                PrintWriter responseWriter = response.writer();
-                responseWriter.write(e.getMessage());
-                responseWriter.flush();
-                LOGGER.error("Error creating/updating library " + libraryName.first + "." + libraryName.second, e);
+                        requestReference, request);
+            } else if (uploadData.op == LibraryOperation.DELETE) {
+                doDrop(uploadData.dataverse, uploadData.name, uploadData.replaceIfExists, requestReference, request);
             }
+            response.setStatus(HttpResponseStatus.OK);
+            PrintWriter responseWriter = response.writer();
+            String emptyJson = "{}";
+            responseWriter.write(emptyJson);
+            responseWriter.flush();
+        } catch (Exception e) {
+            response.setStatus(toHttpErrorStatus(e));
+            PrintWriter responseWriter = response.writer();
+            Map<String, String> error = Collections.singletonMap("error", e.getMessage());
+            String errorJson = "";
+            try {
+                errorJson = OBJECT_MAPPER.writeValueAsString(error);
+            } catch (JsonProcessingException ex) {
+                responseWriter.write("{ \"error\": \"Unable to process error message!\" }");
+            }
+            responseWriter.write(errorJson);
+            responseWriter.flush();
+            LOGGER.error("Error modifying library", e);
         } finally {
             requestDecoder.destroy();
-            if (libraryTempFile != null) {
-                try {
+            try {
+                if (libraryTempFile != null) {
                     if (libTmpOut != null) {
                         libTmpOut.close();
                     }
                     Files.deleteIfExists(libraryTempFile);
-                } catch (IOException e) {
-                    LOGGER.warn("Could not delete temporary file " + libraryTempFile, e);
                 }
+            } catch (IOException e) {
+                LOGGER.warn("Could not delete temporary file " + libraryTempFile, e);
             }
         }
     }
 
-    @Override
-    protected void delete(IServletRequest request, IServletResponse response) {
-        Pair<DataverseName, String> libraryName = parseLibraryName(request);
-        if (libraryName == null) {
-            response.setStatus(HttpResponseStatus.BAD_REQUEST);
-            return;
-        }
-        try {
-            IRequestReference requestReference = receptionist.welcome(request);
-            doDrop(libraryName.first, libraryName.second, false, requestReference, request, response);
-            response.setStatus(HttpResponseStatus.OK);
-        } catch (Exception e) {
-            response.setStatus(toHttpErrorStatus(e));
-            PrintWriter responseWriter = response.writer();
-            responseWriter.write(e.getMessage());
-            responseWriter.flush();
-            LOGGER.error("Error deleting library " + libraryName.first + "." + libraryName.second, e);
-        }
-    }
-
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
index 2c29d14..1563833 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCUdfRecoveryServlet.java
@@ -24,6 +24,7 @@
 
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.external.library.ExternalLibraryManager;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
@@ -38,8 +39,8 @@
     public static final String GET_ALL_UDF_ENDPOINT = "/all";
 
     public NCUdfRecoveryServlet(ConcurrentMap<String, Object> ctx, String[] paths, IApplicationContext appCtx,
-            HttpScheme httpServerProtocol, int httpServerPort) {
-        super(ctx, paths, appCtx, httpServerProtocol, httpServerPort);
+            ILangCompilationProvider compilationProvider, HttpScheme httpServerProtocol, int httpServerPort) {
+        super(ctx, paths, appCtx, compilationProvider, httpServerProtocol, httpServerPort);
     }
 
     @Override
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index fcc6993..5153ebb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -18,13 +18,7 @@
  */
 package org.apache.asterix.api.http.server;
 
-import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_JSON_VAL;
-import static org.apache.asterix.common.exceptions.ErrorCode.INVALID_REQ_PARAM_VAL;
 import static org.apache.asterix.common.exceptions.ErrorCode.NO_STATEMENT_PROVIDED;
-import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_BAD_CLUSTER_STATE;
-import static org.apache.asterix.common.exceptions.ErrorCode.REJECT_NODE_UNREGISTERED;
-import static org.apache.asterix.common.exceptions.ErrorCode.REQUEST_TIMEOUT;
-import static org.apache.hyracks.api.exceptions.ErrorCode.JOB_REQUIREMENTS_EXCEED_CAPACITY;
 
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -35,6 +29,7 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.function.Function;
@@ -65,6 +60,7 @@
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
 import org.apache.asterix.lang.common.base.IParser;
@@ -86,7 +82,8 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.application.IServiceContext;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.exceptions.IError;
+import org.apache.hyracks.api.exceptions.IFormattedException;
 import org.apache.hyracks.api.exceptions.Warning;
 import org.apache.hyracks.api.result.IResultSet;
 import org.apache.hyracks.control.common.controllers.CCConfig;
@@ -419,6 +416,38 @@
         buildResponseResults(responsePrinter, sessionOutput, translator.getExecutionPlans(), warnings);
     }
 
+    protected boolean handleIFormattedException(IError error, IFormattedException ex,
+            RequestExecutionState executionState, QueryServiceRequestParameters param) {
+        if (error instanceof ErrorCode) {
+            switch ((ErrorCode) error) {
+                case INVALID_REQ_PARAM_VAL:
+                case INVALID_REQ_JSON_VAL:
+                case NO_STATEMENT_PROVIDED:
+                    executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
+                    return true;
+                case REQUEST_TIMEOUT:
+                    LOGGER.info(() -> "handleException: request execution timed out: "
+                            + LogRedactionUtil.userData(param.toString()));
+                    executionState.setStatus(ResultStatus.TIMEOUT, HttpResponseStatus.OK);
+                    return true;
+                case REJECT_NODE_UNREGISTERED:
+                case REJECT_BAD_CLUSTER_STATE:
+                    LOGGER.warn(() -> "handleException: " + ex.getMessage() + ": "
+                            + LogRedactionUtil.userData(param.toString()));
+                    executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.SERVICE_UNAVAILABLE);
+                default:
+                    // fall-through
+            }
+        } else if (error instanceof org.apache.hyracks.api.exceptions.ErrorCode) {
+            switch ((org.apache.hyracks.api.exceptions.ErrorCode) error) {
+                case JOB_REQUIREMENTS_EXCEED_CAPACITY:
+                    executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
+                    return true;
+            }
+        }
+        return false;
+    }
+
     protected void handleExecuteStatementException(Throwable t, RequestExecutionState executionState,
             QueryServiceRequestParameters param) {
         if (t instanceof org.apache.asterix.lang.sqlpp.parser.TokenMgrError || t instanceof AlgebricksException) {
@@ -430,30 +459,17 @@
                         + LogRedactionUtil.statement(param.toString()));
             }
             executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
-        } else if (t instanceof HyracksException) {
-            HyracksException he = (HyracksException) t;
-            // TODO(mblow): reconsolidate
-            if (he.matchesAny(INVALID_REQ_PARAM_VAL, INVALID_REQ_JSON_VAL, NO_STATEMENT_PROVIDED,
-                    JOB_REQUIREMENTS_EXCEED_CAPACITY)) {
-                executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.BAD_REQUEST);
-            } else if (he.matches(REQUEST_TIMEOUT)) {
-                LOGGER.info(() -> "handleException: request execution timed out: "
-                        + LogRedactionUtil.userData(param.toString()));
-                executionState.setStatus(ResultStatus.TIMEOUT, HttpResponseStatus.OK);
-            } else if (he.matchesAny(REJECT_BAD_CLUSTER_STATE, REJECT_NODE_UNREGISTERED)) {
-                LOGGER.warn(() -> "handleException: " + he.getMessage() + ": "
-                        + LogRedactionUtil.userData(param.toString()));
-                executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.SERVICE_UNAVAILABLE);
-            } else {
-                LOGGER.warn(() -> "handleException: unexpected exception " + he.getMessage() + ": "
-                        + LogRedactionUtil.userData(param.toString()), he);
-                executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
+            return;
+        } else if (t instanceof IFormattedException) {
+            IFormattedException formattedEx = (IFormattedException) t;
+            Optional<IError> maybeError = formattedEx.getError();
+            if (maybeError.isPresent()
+                    && handleIFormattedException(maybeError.get(), (IFormattedException) t, executionState, param)) {
+                return;
             }
-        } else {
-            LOGGER.warn(() -> "handleException: unexpected exception: " + LogRedactionUtil.userData(param.toString()),
-                    t);
-            executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
         }
+        LOGGER.warn(() -> "handleException: unexpected exception: " + LogRedactionUtil.userData(param.toString()), t);
+        executionState.setStatus(ResultStatus.FATAL, HttpResponseStatus.INTERNAL_SERVER_ERROR);
     }
 
     private void setSessionConfig(SessionOutput sessionOutput, QueryServiceRequestParameters param,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
index a7b91c6..8b04cc3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RebalanceApiServlet.java
@@ -24,11 +24,11 @@
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collections;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Queue;
+import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
@@ -106,21 +106,15 @@
             // Gets dataverse, dataset, and target nodes for rebalance.
             DataverseName dataverseName = ServletUtil.getDataverseName(request, "dataverseName");
             String datasetName = request.getParameter("datasetName");
-            String nodes = request.getParameter("nodes");
+            Set<String> targetNodes = new LinkedHashSet<>(request.getParameterValues("targetNode"));
             boolean forceRebalance = true;
             String force = request.getParameter("force");
             if (force != null) {
                 forceRebalance = Boolean.parseBoolean(force);
             }
             // Parses and check target nodes.
-            if (nodes == null) {
-                sendResponse(response, HttpResponseStatus.BAD_REQUEST, "nodes are not given");
-                return;
-            }
-            String nodesString = nodes.trim();
-            String[] targetNodes = nodesString.split(",");
-            if ("".equals(nodesString)) {
-                sendResponse(response, HttpResponseStatus.BAD_REQUEST, "target nodes should not be empty");
+            if (targetNodes.isEmpty()) {
+                sendResponse(response, HttpResponseStatus.BAD_REQUEST, "at least one targetNode must be specified");
                 return;
             }
 
@@ -161,7 +155,7 @@
 
     // Schedules a rebalance task.
     private synchronized CountDownLatch scheduleRebalance(DataverseName dataverseName, String datasetName,
-            String[] targetNodes, IServletResponse response, boolean force) {
+            Set<String> targetNodes, IServletResponse response, boolean force) {
         CountDownLatch terminated = new CountDownLatch(1);
         Future<Void> task = executor
                 .submit(() -> doRebalance(dataverseName, datasetName, targetNodes, response, terminated, force));
@@ -171,7 +165,7 @@
     }
 
     // Performs the actual rebalance.
-    private Void doRebalance(DataverseName dataverseName, String datasetName, String[] targetNodes,
+    private Void doRebalance(DataverseName dataverseName, String datasetName, Set<String> targetNodes,
             IServletResponse response, CountDownLatch terminated, boolean force) {
         try {
             // Sets the content type.
@@ -248,8 +242,8 @@
     }
 
     // Rebalances a given dataset.
-    private void rebalanceDataset(DataverseName dataverseName, String datasetName, String[] targetNodes, boolean force)
-            throws Exception {
+    private void rebalanceDataset(DataverseName dataverseName, String datasetName, Set<String> targetNodes,
+            boolean force) throws Exception {
         IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(HYRACKS_CONNECTION_ATTR);
         MetadataProvider metadataProvider = MetadataProvider.create(appCtx, null);
         try {
@@ -260,8 +254,8 @@
                 IMetadataLockManager lockManager = appCtx.getMetadataLockManager();
                 lockManager.acquireDatasetExclusiveModificationLock(metadataProvider.getLocks(), dataverseName,
                         datasetName);
-                RebalanceUtil.rebalance(dataverseName, datasetName, new LinkedHashSet<>(Arrays.asList(targetNodes)),
-                        metadataProvider, hcc, NoOpDatasetRebalanceCallback.INSTANCE, force);
+                RebalanceUtil.rebalance(dataverseName, datasetName, targetNodes, metadataProvider, hcc,
+                        NoOpDatasetRebalanceCallback.INSTANCE, force);
             } finally {
                 activeNotificationHandler.resume(metadataProvider);
             }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index e058d39..a9a3a3e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -582,20 +582,26 @@
 
     private ILSMIOOperationScheduler createIoScheduler(StorageProperties properties) {
         String schedulerName = storageProperties.getIoScheduler();
+        int numPartitions = ioManager.getIODevices().size();
+
+        int maxRunningFlushes = storageProperties.getMaxRunningFlushes(numPartitions);
+        int maxScheduledMerges = storageProperties.getMaxScheduledMerges(numPartitions);
+        int maxRunningMerges = storageProperties.getMaxRunningMerges(numPartitions);
+
         ILSMIOOperationScheduler ioScheduler = null;
         if (AsynchronousScheduler.FACTORY.getName().equalsIgnoreCase(schedulerName)) {
             ioScheduler = AsynchronousScheduler.FACTORY.createIoScheduler(getServiceContext().getThreadFactory(),
-                    HaltCallback.INSTANCE);
+                    HaltCallback.INSTANCE, maxRunningFlushes, maxScheduledMerges, maxRunningMerges);
         } else if (GreedyScheduler.FACTORY.getName().equalsIgnoreCase(schedulerName)) {
             ioScheduler = GreedyScheduler.FACTORY.createIoScheduler(getServiceContext().getThreadFactory(),
-                    HaltCallback.INSTANCE);
+                    HaltCallback.INSTANCE, maxRunningFlushes, maxScheduledMerges, maxRunningMerges);
         } else {
             if (LOGGER.isWarnEnabled()) {
                 LOGGER.log(Level.WARN,
                         "Unknown storage I/O scheduler: " + schedulerName + "; defaulting to greedy I/O scheduler.");
             }
             ioScheduler = GreedyScheduler.FACTORY.createIoScheduler(getServiceContext().getThreadFactory(),
-                    HaltCallback.INSTANCE);
+                    HaltCallback.INSTANCE, maxRunningFlushes, maxScheduledMerges, maxRunningMerges);
         }
         return ioScheduler;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
index b0cb4de..c27b3b1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplication.java
@@ -228,9 +228,9 @@
                 new NCUdfApiServlet(apiServer.ctx(), new String[] { UDF }, getApplicationContext(),
                         sqlppCompilationProvider, apiServer.getScheme(), apiServer.getAddress().getPort()),
                 auth.getFirst(), auth.getSecond()));
-        apiServer.addServlet(new BasicAuthServlet(
-                apiServer.ctx(), new NCUdfRecoveryServlet(apiServer.ctx(), new String[] { UDF_RECOVERY },
-                        getApplicationContext(), apiServer.getScheme(), apiServer.getAddress().getPort()),
+        apiServer.addServlet(new BasicAuthServlet(apiServer.ctx(),
+                new NCUdfRecoveryServlet(apiServer.ctx(), new String[] { UDF_RECOVERY }, getApplicationContext(),
+                        sqlppCompilationProvider, apiServer.getScheme(), apiServer.getAddress().getPort()),
                 auth.getFirst(), auth.getSecond()));
         apiServer.addServlet(new QueryStatusApiServlet(apiServer.ctx(), getApplicationContext(), QUERY_STATUS));
         apiServer.addServlet(new QueryResultApiServlet(apiServer.ctx(), getApplicationContext(), QUERY_RESULT));
diff --git a/asterixdb/asterix-app/src/main/resources/entrypoint.py b/asterixdb/asterix-app/src/main/resources/entrypoint.py
index 0917f49..aba4f29 100755
--- a/asterixdb/asterix-app/src/main/resources/entrypoint.py
+++ b/asterixdb/asterix-app/src/main/resources/entrypoint.py
@@ -26,14 +26,16 @@
 import signal
 import msgpack
 import socket
+import traceback
 from importlib import import_module
 from pathlib import Path
 from enum import IntEnum
 from io import BytesIO
 
 PROTO_VERSION = 1
-HEADER_SZ = 8+8+1
-REAL_HEADER_SZ = 4+8+8+1
+HEADER_SZ = 8 + 8 + 1
+REAL_HEADER_SZ = 4 + 8 + 8 + 1
+FRAMESZ = 32768
 
 
 class MessageType(IntEnum):
@@ -57,19 +59,29 @@
     wrapped_module = None
     wrapped_class = None
     wrapped_fn = None
+    sz = None
+    mid = None
+    rmid = None
+    flag = None
+    resp = None
+    unpacked_msg = None
+    msg_type = None
     packer = msgpack.Packer(autoreset=False)
     unpacker = msgpack.Unpacker()
     response_buf = BytesIO()
     stdin_buf = BytesIO()
     wrapped_fns = {}
     alive = True
+    readbuf = bytearray(FRAMESZ)
+    readview = memoryview(readbuf)
+
 
     def init(self, module_name, class_name, fn_name):
         self.wrapped_module = import_module(module_name)
         # do not allow modules to be called that are not part of the uploaded module
         wrapped_fn = None
         if not self.check_module_path(self.wrapped_module):
-            wrapped_module = None
+            self.wrapped_module = None
             raise ImportError("Module was not found in library")
         if class_name is not None:
             self.wrapped_class = getattr(
@@ -77,12 +89,13 @@
         if self.wrapped_class is not None:
             wrapped_fn = getattr(self.wrapped_class, fn_name)
         else:
-            wrapped_fn = locals()[fn_name]
+            wrapped_fn = getattr(import_module(module_name), fn_name)
         if wrapped_fn is None:
-            raise ImportError("Could not find class or function in specified module")
-        self.wrapped_fns[self.rmid] = wrapped_fn
+            raise ImportError(
+                "Could not find class or function in specified module")
+        self.wrapped_fns[self.mid] = wrapped_fn
 
-    def nextTuple(self, *args, key=None):
+    def next_tuple(self, *args, key=None):
         return self.wrapped_fns[key](*args)
 
     def check_module_path(self, module):
@@ -92,14 +105,14 @@
 
     def read_header(self, readbuf):
         self.sz, self.mid, self.rmid, self.flag = unpack(
-            "!iqqb", readbuf[0:21])
+            "!iqqb", readbuf[0:REAL_HEADER_SZ])
         return True
 
     def write_header(self, response_buf, dlen):
         total_len = dlen + HEADER_SZ
         header = pack("!iqqb", total_len, int(-1), int(self.rmid), self.flag)
         self.response_buf.write(header)
-        return total_len+4
+        return total_len + 4
 
     def get_ver_hlen(self, hlen):
         return hlen + (PROTO_VERSION << 4)
@@ -118,14 +131,13 @@
         self.packer.reset()
 
     def helo(self):
-        #need to ack the connection back before sending actual HELO
+        # need to ack the connection back before sending actual HELO
         self.init_remote_ipc()
-
         self.flag = MessageFlags.NORMAL
         self.response_buf.seek(0)
         self.packer.pack(int(MessageType.HELO))
         self.packer.pack("HELO")
-        dlen = 5 #tag(1) + body(4)
+        dlen = 5  # tag(1) + body(4)
         resp_len = self.write_header(self.response_buf, dlen)
         self.response_buf.write(self.packer.bytes())
         self.resp = self.response_buf.getbuffer()[0:resp_len]
@@ -160,16 +172,19 @@
 
     def handle_call(self):
         self.flag = MessageFlags.NORMAL
-        args = self.unpacked_msg[1]
-        result = None
-        if args is None:
-            result = self.nextTuple(key=self.rmid)
-        else:
-            result = self.nextTuple(args, key=self.rmid)
+        result = ([], [])
+        if len(self.unpacked_msg) > 1:
+            args = self.unpacked_msg[1]
+            if args is not None:
+                for arg in args:
+                    try:
+                        result[0].append(self.next_tuple(*arg, key=self.mid))
+                    except BaseException as e:
+                        result[1].append(traceback.format_exc())
         self.packer.reset()
         self.response_buf.seek(0)
         body = msgpack.packb(result)
-        dlen = len(body)+1  # 1 for tag
+        dlen = len(body) + 1  # 1 for tag
         resp_len = self.write_header(self.response_buf, dlen)
         self.packer.pack(int(MessageType.CALL_RSP))
         self.response_buf.write(self.packer.bytes())
@@ -179,13 +194,12 @@
         self.packer.reset()
         return True
 
-    def handle_error(self,e):
+    def handle_error(self, e):
         self.flag = MessageFlags.NORMAL
-        result = type(e).__name__ + ": " + str(e)
         self.packer.reset()
         self.response_buf.seek(0)
-        body = msgpack.packb(result)
-        dlen = len(body)+1  # 1 for tag
+        body = msgpack.packb(e)
+        dlen = len(body) + 1  # 1 for tag
         resp_len = self.write_header(self.response_buf, dlen)
         self.packer.pack(int(MessageType.ERROR))
         self.response_buf.write(self.packer.bytes())
@@ -193,6 +207,7 @@
         self.resp = self.response_buf.getbuffer()[0:resp_len]
         self.send_msg()
         self.packer.reset()
+        self.alive = False
         return True
 
     type_handler = {
@@ -204,33 +219,47 @@
 
     def connect_sock(self, addr, port):
         self.sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
-        try:
-            self.sock.connect((addr, int(port)))
-        except socket.error as msg:
-            print(sys.stderr, msg)
+        self.sock.connect((addr, int(port)))
 
     def disconnect_sock(self, *args):
         self.sock.shutdown(socket.SHUT_RDWR)
         self.sock.close()
 
     def recv_msg(self):
-        completed = False
-        while not completed and self.alive:
-            readbuf = sys.stdin.buffer.read1(4096)
+        while self.alive:
+            pos = sys.stdin.buffer.readinto1(self.readbuf)
+            if pos <= 0:
+                self.alive = False
+                return
             try:
-                if(len(readbuf) < REAL_HEADER_SZ):
-                    while(len(readbuf) < REAL_HEADER_SZ):
-                        readbuf += sys.stdin.buffer.read1(4096)
-                self.read_header(readbuf)
-                if(self.sz > len(readbuf)):
-                    while(len(readbuf) < self.sz):
-                        readbuf += sys.stdin.buffer.read1(4096)
-                self.unpacker.feed(readbuf[21:])
+                while pos < REAL_HEADER_SZ:
+                    read = sys.stdin.buffer.readinto1(self.readview[pos:])
+                    if read <= 0:
+                        self.alive = False
+                        return
+                    pos += read
+                self.read_header(self.readview)
+                while pos < self.sz and len(self.readbuf) - pos > 0:
+                    read = sys.stdin.buffer.readinto1(self.readview[pos:])
+                    if read <= 0:
+                        self.alive = False
+                        return
+                    pos += read
+                while pos < self.sz:
+                    vszchunk = sys.stdin.buffer.read1()
+                    if len(vszchunk) == 0:
+                        self.alive = False
+                        return
+                    self.readview = None
+                    self.readbuf.extend(vszchunk)
+                    self.readview = memoryview(self.readbuf)
+                    pos += len(vszchunk)
+                self.unpacker.feed(self.readview[REAL_HEADER_SZ:self.sz])
                 self.unpacked_msg = list(self.unpacker)
-                self.type = MessageType(self.unpacked_msg[0])
-                completed = self.type_handler[self.type](self)
+                self.msg_type = MessageType(self.unpacked_msg[0])
+                self.type_handler[self.msg_type](self)
             except BaseException as e:
-                completed = self.handle_error(e)
+                self.handle_error(traceback.format_exc())
 
     def send_msg(self):
         self.sock.sendall(self.resp)
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
index 81e35a0..88277e4 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/ExternalUDFLibrarian.java
@@ -23,8 +23,7 @@
 import java.net.URI;
 
 import org.apache.asterix.common.exceptions.AsterixException;
-import org.apache.commons.io.IOUtils;
-import org.apache.http.HttpEntity;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
 import org.apache.http.auth.AuthScope;
@@ -32,7 +31,6 @@
 import org.apache.http.client.AuthCache;
 import org.apache.http.client.CredentialsProvider;
 import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpDelete;
 import org.apache.http.client.methods.HttpPost;
 import org.apache.http.client.protocol.HttpClientContext;
 import org.apache.http.entity.ContentType;
@@ -44,32 +42,58 @@
 import org.apache.http.impl.client.DefaultHttpClient;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+
 @SuppressWarnings("squid:S134")
 public class ExternalUDFLibrarian implements IExternalUDFLibrarian {
 
     private HttpClient hc;
 
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
     public ExternalUDFLibrarian() {
         hc = new DefaultHttpClient();
     }
 
     @Override
-    public void install(URI path, String libPath, Pair<String, String> credentials) throws Exception {
+    public void install(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            String type, String libPath, Pair<String, String> credentials) throws Exception {
         HttpClientContext hcCtx = createHttpClientContext(path, credentials);
         HttpPost post = new HttpPost(path);
         File lib = new File(libPath);
-        HttpEntity file = MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT)
-                .addBinaryBody("lib", lib, ContentType.DEFAULT_BINARY, lib.getName()).build();
-        post.setEntity(file);
+        MultipartEntityBuilder entity = MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT);
+        if (!useDisplayForm) {
+            for (String dvPart : dataverse.getParts()) {
+                entity.addTextBody(dataverseKey, dvPart);
+            }
+        } else {
+            entity.addTextBody(dataverseKey, dataverse.toString());
+        }
+        entity.addTextBody("name", name);
+        entity.addTextBody("type", type);
+        entity.addBinaryBody("data", lib, ContentType.DEFAULT_BINARY, lib.getName()).build();
+        post.setEntity(entity.build());
         HttpResponse response = hc.execute(post, hcCtx);
         handleResponse(response);
     }
 
     @Override
-    public void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException {
+    public void uninstall(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            Pair<String, String> credentials) throws IOException, AsterixException {
         HttpClientContext hcCtx = createHttpClientContext(path, credentials);
-        HttpDelete del = new HttpDelete(path);
-        HttpResponse response = hc.execute(del, hcCtx);
+        HttpPost post = new HttpPost(path);
+        MultipartEntityBuilder entity = MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT);
+        if (!useDisplayForm) {
+            for (String dvPart : dataverse.getParts()) {
+                entity.addTextBody(dataverseKey, dvPart);
+            }
+        } else {
+            entity.addTextBody(dataverseKey, dataverse.toString());
+        }
+        entity.addTextBody("name", name);
+        entity.addTextBody("delete", "true");
+        post.setEntity(entity.build());
+        HttpResponse response = hc.execute(post, hcCtx);
         handleResponse(response);
     }
 
@@ -89,7 +113,7 @@
         String resp = null;
         int respCode = response.getStatusLine().getStatusCode();
         if (respCode == 500 || respCode == 400) {
-            resp = IOUtils.toString(response.getEntity().getContent());
+            resp = OBJECT_MAPPER.readTree(response.getEntity().getContent()).get("error").asText();
         }
         response.getEntity().consumeContent();
         if (resp == null && respCode != 200) {
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
index 2315bec..639475b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/external/IExternalUDFLibrarian.java
@@ -22,11 +22,14 @@
 import java.net.URI;
 
 import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 
 public interface IExternalUDFLibrarian {
 
-    void install(URI path, String libPath, Pair<String, String> credentials) throws Exception;
+    void install(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            String type, String libPath, Pair<String, String> credentials) throws Exception;
 
-    void uninstall(URI path, Pair<String, String> credentials) throws IOException, AsterixException;
+    void uninstall(URI path, String dataverseKey, DataverseName dataverse, boolean useDisplayForm, String name,
+            Pair<String, String> credentials) throws IOException, AsterixException;
 }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index 3895ec8..5ab5b01 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -28,9 +28,11 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.commons.io.IOUtils;
+import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -203,7 +205,15 @@
 
         LOGGER.debug("+++++++\n" + resultStr + "\n+++++++\n");
 
-        final ObjectNode result = OBJECT_READER.readValue(resultStr);
+        final ObjectNode result;
+        try {
+            result = OBJECT_READER.readValue(resultStr);
+        } catch (Exception e) {
+            // whoops, not JSON (e.g. 404) - just include the body
+            GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, resultStr);
+            throw new Exception(resultStr);
+        }
+
         final boolean isJsonFormat = isJsonFormat(fmt);
 
         // if we have errors field in the results, we will always return it
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 3cc6353..453ed59 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -82,6 +82,8 @@
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.common.utils.Servlets;
+import org.apache.asterix.lang.common.base.IParserFactory;
+import org.apache.asterix.lang.sqlpp.parser.SqlppParserFactory;
 import org.apache.asterix.lang.sqlpp.util.SqlppStatementUtil;
 import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
 import org.apache.asterix.metadata.utils.MetadataConstants;
@@ -104,14 +106,27 @@
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.mutable.MutableInt;
+import org.apache.http.HttpHost;
 import org.apache.http.HttpResponse;
 import org.apache.http.HttpStatus;
+import org.apache.http.NameValuePair;
+import org.apache.http.auth.AuthScope;
+import org.apache.http.auth.UsernamePasswordCredentials;
+import org.apache.http.client.AuthCache;
+import org.apache.http.client.CredentialsProvider;
 import org.apache.http.client.HttpClient;
 import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpUriRequest;
 import org.apache.http.client.methods.RequestBuilder;
+import org.apache.http.client.protocol.HttpClientContext;
+import org.apache.http.client.utils.URLEncodedUtils;
 import org.apache.http.entity.ContentType;
 import org.apache.http.entity.StringEntity;
+import org.apache.http.entity.mime.HttpMultipartMode;
+import org.apache.http.entity.mime.MultipartEntityBuilder;
+import org.apache.http.impl.auth.BasicScheme;
+import org.apache.http.impl.client.BasicAuthCache;
+import org.apache.http.impl.client.BasicCredentialsProvider;
 import org.apache.http.impl.client.CloseableHttpClient;
 import org.apache.http.impl.client.HttpClients;
 import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
@@ -173,6 +188,11 @@
     private static final Pattern VARIABLE_REF_PATTERN = Pattern.compile("\\$(\\w+)");
     private static final Pattern HTTP_PARAM_PATTERN =
             Pattern.compile("param (?<name>[\\w-$]+)(?::(?<type>\\w+))?=(?<value>.*)", Pattern.MULTILINE);
+    private static final Pattern HTTP_PARAMS_FROM_QUERY_PATTERN =
+            Pattern.compile("paramsfromquery (?<value>.*)", Pattern.MULTILINE);
+
+    private static final Pattern HTTP_AUTH_PATTERN =
+            Pattern.compile("auth (?<username>.*):(?<password>.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_BODY_PATTERN = Pattern.compile("body=(.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_STATUSCODE_PATTERN = Pattern.compile("statuscode (.*)", Pattern.MULTILINE);
     private static final Pattern MAX_RESULT_READS_PATTERN =
@@ -619,6 +639,11 @@
         return checkResponse(executeHttpRequest(method), responseCodeValidator);
     }
 
+    protected HttpResponse executeAndCheckHttpRequest(HttpUriRequest method, Predicate<Integer> responseCodeValidator,
+            Pair<String, String> credentials) throws Exception {
+        return checkResponse(executeBasicAuthHttpRequest(method, credentials), responseCodeValidator);
+    }
+
     protected HttpResponse executeHttpRequest(HttpUriRequest method) throws Exception {
         // https://issues.apache.org/jira/browse/ASTERIXDB-2315
         ExecutorService executor = Executors.newSingleThreadExecutor();
@@ -642,6 +667,36 @@
         }
     }
 
+    private HttpResponse executeBasicAuthHttpRequest(HttpUriRequest method, Pair<String, String> credentials)
+            throws Exception {
+        // https://issues.apache.org/jira/browse/ASTERIXDB-2315
+        ExecutorService executor = Executors.newSingleThreadExecutor();
+        CredentialsProvider cp = new BasicCredentialsProvider();
+        cp.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(credentials.first, credentials.second));
+        HttpClientContext hcCtx = HttpClientContext.create();
+        AuthCache ac = new BasicAuthCache();
+        ac.put(new HttpHost(method.getURI().getHost(), method.getURI().getPort(), "http"), new BasicScheme());
+        hcCtx.setAuthCache(ac);
+        CloseableHttpClient client = HttpClients.custom().setRetryHandler(StandardHttpRequestRetryHandler.INSTANCE)
+                .setDefaultCredentialsProvider(cp).build();
+        Future<HttpResponse> future = executor.submit(() -> {
+            try {
+                return client.execute(method, hcCtx);
+            } catch (Exception e) {
+                GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR, "Failure executing {}", method, e);
+                throw e;
+            }
+        });
+        try {
+            return future.get();
+        } catch (Exception e) {
+            client.close();
+            throw e;
+        } finally {
+            executor.shutdownNow();
+        }
+    }
+
     protected HttpContext getHttpContext() {
         return null;
     }
@@ -867,14 +922,38 @@
         return builder.build();
     }
 
+    private boolean isMultipart(Parameter p) {
+        return p != null && (ParameterTypeEnum.MULTIPART_TEXT == p.getType()
+                || ParameterTypeEnum.MULTIPART_BINARY == p.getType());
+    }
+
+    private void addMultipart(MultipartEntityBuilder multipartEntityBuilder, Parameter p) {
+        if (ParameterTypeEnum.MULTIPART_TEXT == p.getType()) {
+            multipartEntityBuilder.addTextBody(p.getName(), p.getValue());
+        } else if (ParameterTypeEnum.MULTIPART_BINARY == p.getType()) {
+            File binary = new File(p.getValue());
+            multipartEntityBuilder.addBinaryBody(p.getName(), binary, ContentType.DEFAULT_BINARY, binary.getName());
+        }
+    }
+
     private HttpUriRequest buildRequest(String method, URI uri, List<Parameter> params, Optional<String> body,
             ContentType contentType) {
         RequestBuilder builder = RequestBuilder.create(method);
         builder.setUri(uri);
+        Optional<MultipartEntityBuilder> mPartBuilder = params.stream()
+                .anyMatch(p -> p.getType() == ParameterTypeEnum.MULTIPART_BINARY
+                        || p.getType() == ParameterTypeEnum.MULTIPART_TEXT)
+                                ? Optional.of(MultipartEntityBuilder.create().setMode(HttpMultipartMode.STRICT))
+                                : Optional.empty();
         for (Parameter param : params) {
-            builder.addParameter(param.getName(), param.getValue());
+            if (isMultipart(param)) {
+                addMultipart(mPartBuilder.get(), param);
+            } else {
+                builder.addParameter(param.getName(), param.getValue());
+            }
         }
         builder.setCharset(UTF_8);
+        mPartBuilder.ifPresent(mpb -> builder.setEntity(mpb.build()));
         body.ifPresent(s -> builder.setEntity(new StringEntity(s, contentType)));
         return builder.build();
     }
@@ -978,6 +1057,14 @@
         return response.getEntity().getContent();
     }
 
+    private InputStream executeJSON(OutputFormat fmt, String method, URI uri, List<Parameter> params,
+            Predicate<Integer> responseCodeValidator, Optional<String> body, ContentType contentType,
+            Pair<String, String> credentials) throws Exception {
+        HttpUriRequest request = buildRequest(method, uri, fmt, params, body, contentType);
+        HttpResponse response = executeAndCheckHttpRequest(request, responseCodeValidator, credentials);
+        return response.getEntity().getContent();
+    }
+
     // Method that reads a DDL/Update/Query File
     // and returns the contents as a string
     // This string is later passed to REST API for execution.
@@ -1239,30 +1326,37 @@
                 // TODO: make this case work well with entity names containing spaces by
                 // looking for \"
                 lines = stripAllComments(statement).trim().split("\n");
+                IParserFactory parserFactory = new SqlppParserFactory();
                 for (String line : lines) {
                     String[] command = line.trim().split(" ");
+                    URI path = createEndpointURI("/admin/udf/");
                     if (command.length < 2) {
                         throw new Exception("invalid library command: " + line);
                     }
-                    String dataverse = command[1];
-                    String library = command[2];
-                    String username = command[3];
-                    String pw = command[4];
                     switch (command[0]) {
                         case "install":
-                            if (command.length != 6) {
+                            if (command.length != 7) {
                                 throw new Exception("invalid library format");
                             }
-                            String libPath = command[5];
-                            URI create = createEndpointURI("/admin/udf/" + dataverse + "/" + library);
-                            librarian.install(create, libPath, new Pair<>(username, pw));
+                            List<String> dataverse = parserFactory.createParser(command[1]).parseMultipartIdentifier();
+                            String library = command[2];
+                            String type = command[3];
+                            String username = command[4];
+                            String pw = command[5];
+                            String libPath = command[6];
+                            librarian.install(path, "dataverse", DataverseName.create(dataverse), false, library, type,
+                                    libPath, new Pair<>(username, pw));
                             break;
                         case "uninstall":
                             if (command.length != 5) {
                                 throw new Exception("invalid library format");
                             }
-                            URI delete = createEndpointURI("/admin/udf/" + dataverse + "/" + library);
-                            librarian.uninstall(delete, new Pair<>(username, pw));
+                            dataverse = parserFactory.createParser(command[1]).parseMultipartIdentifier();
+                            library = command[2];
+                            username = command[3];
+                            pw = command[4];
+                            librarian.uninstall(path, "dataverse", DataverseName.create(dataverse), false, library,
+                                    new Pair<>(username, pw));
                             break;
                         default:
                             throw new Exception("invalid library format");
@@ -1382,9 +1476,16 @@
         if (!body.isPresent()) {
             body = getBodyFromReference(statement, variableCtx);
         }
+        final Pair<String, String> credentials = extractCredentials(statement);
         InputStream resultStream;
         if ("http".equals(extension)) {
-            resultStream = executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body, contentType);
+            if (credentials != null) {
+                resultStream = executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body,
+                        contentType, credentials);
+            } else {
+                resultStream =
+                        executeHttp(reqType, variablesReplaced, fmt, params, statusCodePredicate, body, contentType);
+            }
         } else if ("uri".equals(extension)) {
             resultStream = executeURI(reqType, URI.create(variablesReplaced), fmt, params, statusCodePredicate, body,
                     contentType);
@@ -1834,7 +1935,7 @@
 
     public static List<Parameter> extractParameters(String statement) {
         List<Parameter> params = new ArrayList<>();
-        final Matcher m = HTTP_PARAM_PATTERN.matcher(statement);
+        Matcher m = HTTP_PARAM_PATTERN.matcher(statement);
         while (m.find()) {
             final Parameter param = new Parameter();
             String name = m.group("name");
@@ -1852,9 +1953,30 @@
             }
             params.add(param);
         }
+        m = HTTP_PARAMS_FROM_QUERY_PATTERN.matcher(statement);
+        while (m.find()) {
+            String queryString = m.group("value");
+            for (NameValuePair queryParam : URLEncodedUtils.parse(queryString, UTF_8)) {
+                Parameter param = new Parameter();
+                param.setName(queryParam.getName());
+                param.setValue(queryParam.getValue());
+                params.add(param);
+            }
+        }
         return params;
     }
 
+    public static Pair<String, String> extractCredentials(String statement) {
+        List<Parameter> params = new ArrayList<>();
+        final Matcher m = HTTP_AUTH_PATTERN.matcher(statement);
+        while (m.find()) {
+            String username = m.group("username");
+            String password = m.group("password");
+            return new Pair<>(username, password);
+        }
+        return null;
+    }
+
     private static String extractHttpRequestType(String statement) {
         Matcher m = HTTP_REQUEST_TYPE.matcher(statement);
         return m.find() ? m.group(1) : null;
@@ -1904,6 +2026,13 @@
         return executeURI(ctxType, uri, fmt, params, statusCodePredicate, body, contentType);
     }
 
+    private InputStream executeHttp(String ctxType, String endpoint, OutputFormat fmt, List<Parameter> params,
+            Predicate<Integer> statusCodePredicate, Optional<String> body, ContentType contentType,
+            Pair<String, String> credentials) throws Exception {
+        URI uri = createEndpointURI(endpoint);
+        return executeURI(ctxType, uri, fmt, params, statusCodePredicate, body, contentType, credentials);
+    }
+
     private InputStream executeURI(String ctxType, URI uri, OutputFormat fmt, List<Parameter> params) throws Exception {
         return executeJSON(fmt, ctxType.toUpperCase(), uri, params);
     }
@@ -1913,6 +2042,13 @@
         return executeJSON(fmt, ctxType.toUpperCase(), uri, params, responseCodeValidator, body, contentType);
     }
 
+    private InputStream executeURI(String ctxType, URI uri, OutputFormat fmt, List<Parameter> params,
+            Predicate<Integer> responseCodeValidator, Optional<String> body, ContentType contentType,
+            Pair<String, String> credentials) throws Exception {
+        return executeJSON(fmt, ctxType.toUpperCase(), uri, params, responseCodeValidator, body, contentType,
+                credentials);
+    }
+
     public void killNC(String nodeId, CompilationUnit cUnit) throws Exception {
         //get node process id
         OutputFormat fmt = OutputFormat.CLEAN_JSON;
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
index fe73baf..c3a6839 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/LSMFlushRecoveryTest.java
@@ -173,7 +173,7 @@
                     public void operationFailed(ILSMIOOperation operation, Throwable t) {
                         LOGGER.warn("IO Operation failed", t);
                     }
-                }));
+                }, Integer.MAX_VALUE, Integer.MAX_VALUE));
         dsLifecycleMgr = ncAppCtx.getDatasetLifecycleManager();
     }
 
diff --git a/asterixdb/asterix-app/src/test/resources/TweetSent/crashy.py b/asterixdb/asterix-app/src/test/resources/TweetSent/crashy.py
new file mode 100644
index 0000000..c3d9a74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/TweetSent/crashy.py
@@ -0,0 +1,37 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+import pickle
+import sklearn
+import sys
+import os
+import ctypes
+class TweetSent(object):
+
+    def __init__(self):
+        pickle_path = os.path.join(os.path.dirname(__file__), 'sentiment_pipeline3')
+        f = open(pickle_path,'rb')
+        self.pipeline = pickle.load(f)
+        f.close()
+
+    def sentiment(self, args):
+        if args is None:
+            return 2
+        return self.pipeline.predict([args])[0].item()
+
+    def crash(self):
+        os._exit(1)
diff --git a/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py b/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py
index 37350be..8b8fced 100644
--- a/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py
+++ b/asterixdb/asterix-app/src/test/resources/TweetSent/roundtrip.py
@@ -14,6 +14,10 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
+import math
+
+def sqrt(num):
+    return math.sqrt(num)
 
 class Tests(object):
 
diff --git a/asterixdb/asterix-app/src/test/resources/TweetSent/sentiment.py b/asterixdb/asterix-app/src/test/resources/TweetSent/sentiment.py
index 29d371f..66545ae 100644
--- a/asterixdb/asterix-app/src/test/resources/TweetSent/sentiment.py
+++ b/asterixdb/asterix-app/src/test/resources/TweetSent/sentiment.py
@@ -28,5 +28,7 @@
         self.pipeline = pickle.load(f)
         f.close()
 
-    def sentiment(self, *args):
-        return self.pipeline.predict(args[0])[0].item()
+    def sentiment(self, args):
+        if args is None:
+            return 2
+        return self.pipeline.predict([args])[0].item()
diff --git a/asterixdb/asterix-app/src/test/resources/cc.conf b/asterixdb/asterix-app/src/test/resources/cc.conf
index 1899122..e2cd5b9 100644
--- a/asterixdb/asterix-app/src/test/resources/cc.conf
+++ b/asterixdb/asterix-app/src/test/resources/cc.conf
@@ -33,6 +33,7 @@
 
 [nc]
 credential.file=src/test/resources/security/passwd
+python.cmd.autolocate=true
 address=127.0.0.1
 command=asterixnc
 app.class=org.apache.asterix.hyracks.bootstrap.NCApplication
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate-subclause/agg_filter_01/agg_filter_01.10.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate-subclause/agg_filter_01/agg_filter_01.10.sqlpp
new file mode 100644
index 0000000..dca81bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate-subclause/agg_filter_01/agg_filter_01.10.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test various combinations of grouping sets
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type tenkType as closed {
+  unique1         : integer,
+  unique2         : integer,
+  two             : integer,
+  four            : integer,
+  ten             : integer,
+  twenty          : integer,
+  hundred         : integer,
+  thousand        : integer,
+  twothous        : integer,
+  fivethous       : integer,
+  tenthous        : integer,
+  odd100          : integer,
+  even100         : integer,
+  stringu1        : string,
+  stringu2        : string,
+  string4         : string
+};
+
+create dataset tenk(tenkType) primary key unique2;
+
+select
+  ten,
+  count(*) filter(where four > 0) as cnt,
+  min(two) filter(where four > 0) as min2,
+  max(two) filter(where four > 0) as max2,
+  sum(twenty) filter(where four > 0) as sum20
+from tenk
+group by ten
+order by ten;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate-subclause/agg_filter_01/agg_filter_01.9.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate-subclause/agg_filter_01/agg_filter_01.9.sqlpp
new file mode 100644
index 0000000..686d83f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/aggregate-subclause/agg_filter_01/agg_filter_01.9.sqlpp
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test various combinations of grouping sets
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+create type tenkType as closed {
+  unique1         : integer,
+  unique2         : integer,
+  two             : integer,
+  four            : integer,
+  ten             : integer,
+  twenty          : integer,
+  hundred         : integer,
+  thousand        : integer,
+  twothous        : integer,
+  fivethous       : integer,
+  tenthous        : integer,
+  odd100          : integer,
+  even100         : integer,
+  stringu1        : string,
+  stringu2        : string,
+  string4         : string
+};
+
+create dataset tenk(tenkType) primary key unique2;
+
+select
+  count(*) filter(where four > 0) as cnt,
+  min(two) filter(where four > 0) as min2,
+  max(two) filter(where four > 0) as max2,
+  sum(twenty) filter(where four > 0) as sum20
+from tenk;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2845.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2845.sqlpp
new file mode 100644
index 0000000..4494cfa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-2845.sqlpp
@@ -0,0 +1,58 @@
+/*
+ * 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: This test case is to verify the fix for ASTERIXDB-2845
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset jds(jid integer not unknown) open type primary key jid;
+
+create dataset mds(mid integer not unknown) open type primary key mid;
+
+SET `compiler.sort.parallel` "false";
+
+WITH
+j AS (
+  SELECT jid, a
+  FROM jds
+), ---> 3 rows (jid=1, 2, 3)
+
+m1 AS (
+  SELECT jid, x, COUNT(1) c1
+  FROM mds
+  GROUP BY jid, x
+),
+
+m2 AS (
+  SELECT jid, y, COUNT(1) c2
+  FROM mds
+  GROUP BY jid, y
+)
+
+SELECT j.jid AS j_jid, j.a AS j_a,
+  m1.jid AS m1_jid, m1.x AS m1_x, m1.c1 AS m1_c1,
+  m2.jid AS m2_jid, m2.y AS m2_y, m2.c2 AS m2_c2
+FROM j
+LEFT OUTER JOIN m1 ON j.jid=m1.jid
+LEFT OUTER JOIN m2 ON j.jid=m1.jid
+ORDER BY j_jid, m1_x, m2_y, m2_jid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.plan
new file mode 100644
index 0000000..63d3f7c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.plan
@@ -0,0 +1,27 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$ten(ASC) ]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$95]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- HASH_PARTITION_EXCHANGE [$$95]  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$82]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$82(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan
index bc5d5aa..574df58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.3.plan
@@ -2,29 +2,11 @@
   -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
     -- STREAM_PROJECT  |UNPARTITIONED|
       -- ASSIGN  |UNPARTITIONED|
-        -- STREAM_PROJECT  |UNPARTITIONED|
-          -- SUBPLAN  |UNPARTITIONED|
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- AGGREGATE  |LOCAL|
-                        -- STREAM_SELECT  |UNPARTITIONED|
-                          -- ASSIGN  |UNPARTITIONED|
-                            -- ASSIGN  |UNPARTITIONED|
-                              -- UNNEST  |UNPARTITIONED|
-                                -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
-                  }
-            -- SUBPLAN  |UNPARTITIONED|
-                    {
-                      -- AGGREGATE  |LOCAL|
-                        -- AGGREGATE  |LOCAL|
-                          -- STREAM_SELECT  |UNPARTITIONED|
-                            -- ASSIGN  |UNPARTITIONED|
-                              -- ASSIGN  |UNPARTITIONED|
-                                -- UNNEST  |UNPARTITIONED|
-                                  -- NESTED_TUPLE_SOURCE  |UNPARTITIONED|
-                    }
-              -- AGGREGATE  |UNPARTITIONED|
-                -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_SELECT  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ASSIGN  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan
index 9071921..c94ef11 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.4.plan
@@ -3,36 +3,25 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$two(ASC) ]  |PARTITIONED|
-          -- PRE_CLUSTERED_GROUP_BY[$$69]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$67]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-                  {
-                    -- AGGREGATE  |LOCAL|
-                      -- NESTED_TUPLE_SOURCE  |LOCAL|
-                  }
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- STABLE_SORT [$$69(ASC)]  |PARTITIONED|
-                -- HASH_PARTITION_EXCHANGE [$$69]  |PARTITIONED|
-                  -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
-                          {
-                            -- AGGREGATE  |LOCAL|
-                              -- STREAM_SELECT  |LOCAL|
-                                -- NESTED_TUPLE_SOURCE  |LOCAL|
-                          }
+            -- HASH_PARTITION_EXCHANGE [$$67]  |PARTITIONED|
+              -- PRE_CLUSTERED_GROUP_BY[$$58]  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- STABLE_SORT [$$58(ASC)]  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
-                            -- ASSIGN  |PARTITIONED|
-                              -- STREAM_PROJECT  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- DATASOURCE_SCAN (test.tenk)  |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/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan
index 91e2624..9462514 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.5.plan
@@ -1,90 +1,68 @@
 -- DISTRIBUTE_RESULT  |PARTITIONED|
   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
     -- STREAM_PROJECT  |PARTITIONED|
-      -- SORT_MERGE_EXCHANGE [$$197(ASC) ]  |PARTITIONED|
-        -- STABLE_SORT [$$197(ASC)]  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$189(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$189(ASC)]  |PARTITIONED|
           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
             -- UNION_ALL  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- PRE_CLUSTERED_GROUP_BY[$$244]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$236]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                              {
-                                -- AGGREGATE  |LOCAL|
-                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
-                              }
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- STABLE_SORT [$$244(ASC)]  |PARTITIONED|
-                            -- HASH_PARTITION_EXCHANGE [$$244]  |PARTITIONED|
-                              -- PRE_CLUSTERED_GROUP_BY[$$115]  |PARTITIONED|
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
-                                      {
-                                        -- AGGREGATE  |LOCAL|
-                                          -- STREAM_SELECT  |LOCAL|
-                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                      }
+                        -- HASH_PARTITION_EXCHANGE [$$236]  |PARTITIONED|
+                          -- PRE_CLUSTERED_GROUP_BY[$$115]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- STREAM_SELECT  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$115(ASC)]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- STABLE_SORT [$$115(ASC)]  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ASSIGN  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- REPLICATE  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                -- REPLICATE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                        -- PRE_CLUSTERED_GROUP_BY[$$247]  |PARTITIONED|
+                        -- SORT_GROUP_BY[$$239]  |PARTITIONED|
                                 {
                                   -- AGGREGATE  |LOCAL|
                                     -- NESTED_TUPLE_SOURCE  |LOCAL|
                                 }
-                                {
-                                  -- AGGREGATE  |LOCAL|
-                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                }
-                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                            -- STABLE_SORT [$$247(ASC)]  |PARTITIONED|
-                              -- HASH_PARTITION_EXCHANGE [$$247]  |PARTITIONED|
-                                -- PRE_CLUSTERED_GROUP_BY[$$116]  |PARTITIONED|
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
-                                        {
-                                          -- AGGREGATE  |LOCAL|
-                                            -- STREAM_SELECT  |LOCAL|
-                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                        }
+                          -- HASH_PARTITION_EXCHANGE [$$239]  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$116]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$116(ASC)]  |PARTITIONED|
                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                    -- STABLE_SORT [$$116(ASC)]  |PARTITIONED|
-                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- REPLICATE  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- REPLICATE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                      -- DATASOURCE_SCAN (test.tenk)  |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/aggregate-subclause/agg_filter_01/agg_filter_01.9.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.plan
new file mode 100644
index 0000000..02d8027
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.plan
@@ -0,0 +1,16 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- AGGREGATE  |UNPARTITIONED|
+          -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
+            -- AGGREGATE  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- STREAM_SELECT  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- DATASOURCE_SCAN (test.tenk)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
index 0df14c7..0128265 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
@@ -22,12 +22,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- EXTERNAL_GROUP_BY[$$212, $$213]  |PARTITIONED|
+                      -- EXTERNAL_GROUP_BY[$$206, $$207]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$212, $$213]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$206, $$207]  |PARTITIONED|
                           -- EXTERNAL_GROUP_BY[$$181, $$182]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
@@ -51,12 +51,12 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- EXTERNAL_GROUP_BY[$$212, $$213]  |PARTITIONED|
+                                -- EXTERNAL_GROUP_BY[$$206, $$207]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$212, $$213]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$206, $$207]  |PARTITIONED|
                                     -- EXTERNAL_GROUP_BY[$$181, $$182]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
index 5d874f3..1feedcd 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806.plan
@@ -3,12 +3,12 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_returnflag(ASC), $$l_linestatus(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$165, $$166]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$159, $$160]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$165, $$166]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$159, $$160]  |PARTITIONED|
               -- SORT_GROUP_BY[$$133, $$134]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
index d918b1b..0d669e0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
@@ -9,12 +9,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$165, $$166]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$159, $$160]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$165, $$166]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$159, $$160]  |PARTITIONED|
                           -- SORT_GROUP_BY[$$133, $$134]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
@@ -37,12 +37,12 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$165, $$166]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$159, $$160]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$165, $$166]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$159, $$160]  |PARTITIONED|
                                     -- SORT_GROUP_BY[$$133, $$134]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
index 8985321..cb471a9 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists.plan
@@ -3,12 +3,12 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$188]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$187]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$187]  |PARTITIONED|
               -- SORT_GROUP_BY[$$162]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
@@ -21,20 +21,20 @@
                         -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$185]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$184]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
-                                -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$178]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- STREAM_SELECT  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
index e98c53a..52d6e25 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
@@ -9,12 +9,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$188]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$187]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$187]  |PARTITIONED|
                           -- SORT_GROUP_BY[$$162]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
@@ -27,20 +27,20 @@
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$185]  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$184]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                   }
-                                            -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
-                                              -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$178]  |PARTITIONED|
                                                       {
                                                         -- AGGREGATE  |LOCAL|
                                                           -- STREAM_SELECT  |LOCAL|
                                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                       }
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
+                                                  -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -86,12 +86,12 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$188]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$187]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$187]  |PARTITIONED|
                                     -- SORT_GROUP_BY[$$162]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
@@ -104,20 +104,20 @@
                                               -- STREAM_SELECT  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SORT_GROUP_BY[$$185]  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$184]  |PARTITIONED|
                                                             {
                                                               -- AGGREGATE  |LOCAL|
                                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                             }
-                                                      -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
-                                                        -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$184]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$178]  |PARTITIONED|
                                                                 {
                                                                   -- AGGREGATE  |LOCAL|
                                                                     -- STREAM_SELECT  |LOCAL|
                                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                 }
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
+                                                            -- STABLE_SORT [$$178(ASC)]  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
index 0120576..d631085 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists.plan
@@ -3,12 +3,12 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$cntrycode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$189]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$188]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$189]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
               -- SORT_GROUP_BY[$$163]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
@@ -21,20 +21,20 @@
                         -- STREAM_SELECT  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- SORT_GROUP_BY[$$186]  |PARTITIONED|
+                              -- SORT_GROUP_BY[$$185]  |PARTITIONED|
                                       {
                                         -- AGGREGATE  |LOCAL|
                                           -- NESTED_TUPLE_SOURCE  |LOCAL|
                                       }
-                                -- HASH_PARTITION_EXCHANGE [$$186]  |PARTITIONED|
-                                  -- PRE_CLUSTERED_GROUP_BY[$$180]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
+                                  -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
                                           {
                                             -- AGGREGATE  |LOCAL|
                                               -- STREAM_SELECT  |LOCAL|
                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                           }
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- STABLE_SORT [$$180(ASC)]  |PARTITIONED|
+                                      -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                           -- STREAM_PROJECT  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
index d1f0a82..5ab7d30 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
@@ -9,12 +9,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$189]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$188]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$189]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
                           -- SORT_GROUP_BY[$$163]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
@@ -27,20 +27,20 @@
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- SORT_GROUP_BY[$$186]  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$185]  |PARTITIONED|
                                                   {
                                                     -- AGGREGATE  |LOCAL|
                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                   }
-                                            -- HASH_PARTITION_EXCHANGE [$$186]  |PARTITIONED|
-                                              -- PRE_CLUSTERED_GROUP_BY[$$180]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
                                                       {
                                                         -- AGGREGATE  |LOCAL|
                                                           -- STREAM_SELECT  |LOCAL|
                                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                       }
                                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                  -- STABLE_SORT [$$180(ASC)]  |PARTITIONED|
+                                                  -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -86,12 +86,12 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$189]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$188]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$189]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$188]  |PARTITIONED|
                                     -- SORT_GROUP_BY[$$163]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
@@ -104,20 +104,20 @@
                                               -- STREAM_SELECT  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- SORT_GROUP_BY[$$186]  |PARTITIONED|
+                                                    -- SORT_GROUP_BY[$$185]  |PARTITIONED|
                                                             {
                                                               -- AGGREGATE  |LOCAL|
                                                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                             }
-                                                      -- HASH_PARTITION_EXCHANGE [$$186]  |PARTITIONED|
-                                                        -- PRE_CLUSTERED_GROUP_BY[$$180]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$185]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$179]  |PARTITIONED|
                                                                 {
                                                                   -- AGGREGATE  |LOCAL|
                                                                     -- STREAM_SELECT  |LOCAL|
                                                                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                                                                 }
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- STABLE_SORT [$$180(ASC)]  |PARTITIONED|
+                                                            -- STABLE_SORT [$$179(ASC)]  |PARTITIONED|
                                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
new file mode 100644
index 0000000..7cbb67f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-2845.plan
@@ -0,0 +1,123 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- SORT_MERGE_EXCHANGE [$$277(ASC), $#4(ASC), $#5(ASC), $#6(ASC) ]  |PARTITIONED|
+          -- STABLE_SORT [$$277(ASC), $#4(ASC), $#5(ASC), $#6(ASC)]  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- HYBRID_HASH_JOIN [$$295][$$296]  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$295]  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$277][$$jid]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN (test.jds)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$jid]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- SORT_GROUP_BY[$$319, $$320]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$319, $$320]  |PARTITIONED|
+                                              -- SORT_GROUP_BY[$$273, $$274]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                    -- HASH_PARTITION_EXCHANGE [$$296]  |PARTITIONED|
+                      -- NESTED_LOOP  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- STREAM_SELECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$303][$$306]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN (test.jds)  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$306]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- SORT_GROUP_BY[$$322, $$323]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- HASH_PARTITION_EXCHANGE [$$322, $$323]  |PARTITIONED|
+                                                  -- SORT_GROUP_BY[$$311, $$312]  |PARTITIONED|
+                                                          {
+                                                            -- AGGREGATE  |LOCAL|
+                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                          }
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- REPLICATE  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$325, $$326]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- HASH_PARTITION_EXCHANGE [$$325, $$326]  |PARTITIONED|
+                                        -- SORT_GROUP_BY[$$275, $$276]  |PARTITIONED|
+                                                {
+                                                  -- AGGREGATE  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                }
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN (test.mds)  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
index 9d91445..0be7ab2 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping.plan
@@ -3,12 +3,12 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$132]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$131]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
               -- SORT_GROUP_BY[$$114]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
@@ -17,12 +17,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$114][$$121]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$123][$$118]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
                                       -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
index 1fe6eb3..9908238 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast.plan
@@ -3,12 +3,12 @@
     -- STREAM_PROJECT  |PARTITIONED|
       -- ASSIGN  |PARTITIONED|
         -- SORT_MERGE_EXCHANGE [$$l_shipmode(ASC) ]  |PARTITIONED|
-          -- SORT_GROUP_BY[$$132]  |PARTITIONED|
+          -- SORT_GROUP_BY[$$131]  |PARTITIONED|
                   {
                     -- AGGREGATE  |LOCAL|
                       -- NESTED_TUPLE_SOURCE  |LOCAL|
                   }
-            -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+            -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
               -- SORT_GROUP_BY[$$114]  |PARTITIONED|
                       {
                         -- AGGREGATE  |LOCAL|
@@ -17,11 +17,11 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- HYBRID_HASH_JOIN [$$114][$$121]  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                           -- STREAM_PROJECT  |PARTITIONED|
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                              -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
                                     -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
index d86b6b0..baeda7a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
@@ -9,12 +9,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$132]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$131]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
                           -- SORT_GROUP_BY[$$114]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
@@ -23,11 +23,11 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$114][$$121]  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
                                                 -- STREAM_SELECT  |PARTITIONED|
@@ -55,12 +55,12 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$132]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$131]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
                                     -- SORT_GROUP_BY[$$114]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
@@ -69,11 +69,11 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$114][$$121]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$121][$$118]  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- STREAM_SELECT  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
index 9615a18..bbd6cf0 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
@@ -9,12 +9,12 @@
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- REPLICATE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- SORT_GROUP_BY[$$132]  |PARTITIONED|
+                      -- SORT_GROUP_BY[$$131]  |PARTITIONED|
                               {
                                 -- AGGREGATE  |LOCAL|
                                   -- NESTED_TUPLE_SOURCE  |LOCAL|
                               }
-                        -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
                           -- SORT_GROUP_BY[$$114]  |PARTITIONED|
                                   {
                                     -- AGGREGATE  |LOCAL|
@@ -23,12 +23,12 @@
                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                  -- HYBRID_HASH_JOIN [$$114][$$121]  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                       -- STREAM_PROJECT  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- HYBRID_HASH_JOIN [$$123][$$118]  |PARTITIONED|
-                                            -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
                                               -- STREAM_PROJECT  |PARTITIONED|
                                                 -- STREAM_SELECT  |PARTITIONED|
                                                   -- ASSIGN  |PARTITIONED|
@@ -55,12 +55,12 @@
                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                             -- REPLICATE  |PARTITIONED|
                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                -- SORT_GROUP_BY[$$132]  |PARTITIONED|
+                                -- SORT_GROUP_BY[$$131]  |PARTITIONED|
                                         {
                                           -- AGGREGATE  |LOCAL|
                                             -- NESTED_TUPLE_SOURCE  |LOCAL|
                                         }
-                                  -- HASH_PARTITION_EXCHANGE [$$132]  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$131]  |PARTITIONED|
                                     -- SORT_GROUP_BY[$$114]  |PARTITIONED|
                                             {
                                               -- AGGREGATE  |LOCAL|
@@ -69,12 +69,12 @@
                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                         -- STREAM_PROJECT  |PARTITIONED|
                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                            -- HYBRID_HASH_JOIN [$$114][$$121]  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$114][$$120]  |PARTITIONED|
                                               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- HYBRID_HASH_JOIN [$$123][$$118]  |PARTITIONED|
-                                                      -- HASH_PARTITION_EXCHANGE [$$123]  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$122][$$118]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
                                                         -- STREAM_PROJECT  |PARTITIONED|
                                                           -- STREAM_SELECT  |PARTITIONED|
                                                             -- ASSIGN  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_01.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_01.10.query.sqlpp
new file mode 100644
index 0000000..0fd64d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_01.10.query.sqlpp
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test several various aggregates with filter with GROUP BY clause
+ */
+
+use test;
+
+select
+  ten,
+  count(*) filter(where four > 0) as cnt,
+  min(two) filter(where four > 0) as min2,
+  max(two) filter(where four > 0) as max2,
+  sum(twenty) filter(where four > 0) as sum20
+from tenk
+group by ten
+order by ten;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_01.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_01.9.query.sqlpp
new file mode 100644
index 0000000..c9566c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_01.9.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Test several various aggregates with filter without GROUP BY clause
+ */
+
+use test;
+
+select
+  count(*) filter(where four > 0) as cnt,
+  min(two) filter(where four > 0) as min2,
+  max(two) filter(where four > 0) as max2,
+  sum(twenty) filter(where four > 0) as sum20
+from tenk;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/bad-ext-function-ddl-1/bad-ext-function-ddl-1.2.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.0.ddl.sqlpp
new file mode 100644
index 0000000..76cc70d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.0.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.1.lib.sqlpp
new file mode 100644
index 0000000..699e565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.1.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.2.ddl.sqlpp
new file mode 100644
index 0000000..8a2746c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.2.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallibtest;
+
+create function crash()
+  as "crashy", "TweetSent.crash" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.3.query.sqlpp
new file mode 100644
index 0000000..f1858f3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/crash/crash.3.query.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+// param max-warnings:json=2
+
+use externallibtest;
+
+crash();
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/create-or-replace-function-1/create-or-replace-function-1.2.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/deterministic/deterministic.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
index ac3d3d0..1707699 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/exception_create_system_library/exception_create_system_library.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install Metadata testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install Metadata testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital/getCapital.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/getCapital_open/getCapital_open.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
new file mode 100644
index 0000000..185d282
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.1.post.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=badType
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
new file mode 100644
index 0000000..01b4982
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.2.post.http
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param delete:multipart_text=true
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
new file mode 100644
index 0000000..01c05d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.3.post.http
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
new file mode 100644
index 0000000..0b6d882
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.4.post.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_text=bogus
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
new file mode 100644
index 0000000..e8de108
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/invalid_library_requests/library_list_api_multipart.5.post.http
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=Default
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp
index 63efff4..b1dba3d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/keyword_detector/keyword_detector.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install test testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install test testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.0.ddl.sqlpp
new file mode 100644
index 0000000..76cc70d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.0.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
new file mode 100644
index 0000000..de72c49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.1.post.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.2.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.2.get.http
new file mode 100644
index 0000000..5851101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api/library_list_api.2.get.http
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+# auth admin:admin
+ /admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.0.ddl.sqlpp
new file mode 100644
index 0000000..0c9da1e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.0.ddl.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE externallibtest.foo if exists;
+CREATE DATAVERSE  externallibtest.foo;
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
+DROP DATAVERSE `external`.lib.test if exists;
+CREATE DATAVERSE  `external`.lib.test;
+DROP DATAVERSE externallibtest.foo.bar if exists;
+CREATE DATAVERSE  externallibtest.foo.bar;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
new file mode 100644
index 0000000..c097ccc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.1.post.http
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param dataverse:multipart_text=foo
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
new file mode 100644
index 0000000..de72c49
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.2.post.http
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
new file mode 100644
index 0000000..16b3596
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.3.post.http
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=external
+# param dataverse:multipart_text=lib
+# param dataverse:multipart_text=test
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
new file mode 100644
index 0000000..97a3c27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.4.post.http
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+# auth admin:admin
+# param dataverse:multipart_text=externallibtest
+# param dataverse:multipart_text=foo
+# param dataverse:multipart_text=bar
+# param name:multipart_text=testlib
+# param type:multipart_text=java
+# param data:multipart_binary=target/data/externallib/asterix-external-data-testlib.zip
+
+/admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.5.get.http
new file mode 100644
index 0000000..5851101
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/library_list_api_multipart/library_list_api_multipart.5.get.http
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+# auth admin:admin
+ /admin/udf
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp
index 9e53153..f2ddc74 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/my_array_sum/my_array_sum.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest2 testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest2 testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
index 1650910..699e565 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.query.sqlpp
new file mode 100644
index 0000000..1ec6766
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+select sentiment("great") as peachy, sentiment("okay") as phlegmatic,
+       sentiment("meh") as indifferent, sentiment("ugh") as choleric;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.7.ddl.sqlpp
similarity index 100%
rename from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.ddl.sqlpp
rename to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.7.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.0.ddl.sqlpp
new file mode 100644
index 0000000..fe2ba4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.0.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE externallib.test if exists;
+CREATE DATAVERSE  externallib.test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
new file mode 100644
index 0000000..67e22cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.1.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install externallib.test testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.2.ddl.sqlpp
new file mode 100644
index 0000000..eff35da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.2.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallib.test;
+
+create function sentiment(s)
+  as "sentiment", "TweetSent.sentiment" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.3.query.sqlpp
new file mode 100644
index 0000000..dbfef4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallib.test;
+
+sentiment("bad");
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.4.query.sqlpp
new file mode 100644
index 0000000..d89c46d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.4.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallib.test;
+
+sentiment("great");
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.5.query.sqlpp
new file mode 100644
index 0000000..f46bde4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.5.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallib.test;
+
+sentiment("meh");
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.6.ddl.sqlpp
new file mode 100644
index 0000000..ef74f58
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_multipart/mysentiment_multipart.6.ddl.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE externallib.test;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.0.ddl.sqlpp
new file mode 100644
index 0000000..18ad48b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.0.ddl.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.
+ */
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
+USE  externallibtest;
+create type typeTweet if not exists as open{
+    create_at : datetime,
+    id: bigint
+};
+
+create dataset Tweet(typeTweet) primary key id;
+
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.1.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.1.update.sqlpp
new file mode 100644
index 0000000..5a8dae2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.1.update.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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  externallibtest;
+
+load dataset Tweet using localfs(("path"="asterix_nc1://data/twitter/real.adm"),("format"="adm"));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.10.query.sqlpp
new file mode 100644
index 0000000..f7a29fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.10.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+select value count(sentiment(t.text))
+from Tweet t;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.11.query.sqlpp
new file mode 100644
index 0000000..f1ef8e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.11.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+set `rewrite_attempt_batch_assign` "false";
+
+select value count(sentiment(t.text))
+from Tweet t;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.12.query.sqlpp
new file mode 100644
index 0000000..4d4d179
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.12.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+select count(sentiment(t.text)), count(t.text)
+from Tweet t;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.13.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.13.ddl.sqlpp
new file mode 100644
index 0000000..3438521
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.13.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ USE externallibtest;
+
+create function sentiment_nullcall(s)
+  as "sentiment", "TweetSent.sentiment" at testlib with {"null-call": "true"};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.14.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.14.query.sqlpp
new file mode 100644
index 0000000..0a219ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.14.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use externallibtest;
+
+select value count(sentiment_nullcall(t.text))
+from Tweet t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.15.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.15.query.sqlpp
new file mode 100644
index 0000000..a259660
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.15.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use externallibtest;
+
+set `rewrite_attempt_batch_assign` "false";
+
+select value count(sentiment_nullcall(t.text))
+from Tweet t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.16.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.16.ddl.sqlpp
new file mode 100644
index 0000000..b3adc32
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.16.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+ USE externallibtest;
+
+create function sentiment_nullcall_bool(s)
+  as "sentiment", "TweetSent.sentiment" at testlib with {"null-call": true};
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.17.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.17.query.sqlpp
new file mode 100644
index 0000000..a336979
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.17.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use externallibtest;
+
+select value count(sentiment_nullcall_bool(t.text))
+from Tweet t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.18.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.18.query.sqlpp
new file mode 100644
index 0000000..14650a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.18.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use externallibtest;
+
+set `rewrite_attempt_batch_assign` "false";
+
+select value count(sentiment_nullcall_bool(t.text))
+from Tweet t;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.19.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.19.ddl.sqlpp
new file mode 100644
index 0000000..d05af3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.19.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use externallibtest;
+
+create function roundtrip(s)
+  as "roundtrip", "Tests.roundtrip" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.2.lib.sqlpp
new file mode 100644
index 0000000..699e565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.2.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.20.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.20.query.sqlpp
new file mode 100644
index 0000000..7d6550c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.20.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+USE externallibtest;
+
+select t.id as id, length(roundtrip(t.text)[0]) as len, sentiment(t.text) as sent
+from Tweet t
+order by id DESC
+limit 100;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.21.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.21.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.3.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.3.ddl.sqlpp
new file mode 100644
index 0000000..00838de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.3.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallibtest;
+
+create function sentiment(s)
+  as "sentiment", "TweetSent.sentiment" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.4.query.sqlpp
new file mode 100644
index 0000000..f7a29fd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.4.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+select value count(sentiment(t.text))
+from Tweet t;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.5.query.sqlpp
new file mode 100644
index 0000000..f1ef8e9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+set `rewrite_attempt_batch_assign` "false";
+
+select value count(sentiment(t.text))
+from Tweet t;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.6.query.sqlpp
new file mode 100644
index 0000000..4a85ab7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.6.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+select sentiment(t.text) as sent, length(t.text) as text
+from Tweet t
+order by t.id
+limit 100;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.7.query.sqlpp
new file mode 100644
index 0000000..69174bf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.7.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+set `rewrite_attempt_batch_assign` "false";
+
+select sentiment(t.text) as sent, length(t.text) as text
+from Tweet t
+order by t.id
+limit 100;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.8.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.8.update.sqlpp
new file mode 100644
index 0000000..891efc0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.8.update.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+insert into Tweet (
+ select t.create_at, t.id+1000000 as id, t.in_reply_to_status, t.in_reply_to_user, t.favorite_count, t.coordinate, t.retweet_count, t.lang,
+        t.is_retweet, t.user, t.place
+        from Tweet t
+        limit 50
+);
+
+insert into Tweet (
+ select t.create_at, t.id+2000000 as id, t.in_reply_to_status, t.in_reply_to_user, t.favorite_count, t.coordinate, t.retweet_count, t.lang,
+        t.is_retweet, t.user, t.place, null as text
+        from Tweet t
+        limit 50
+);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.9.query.sqlpp
new file mode 100644
index 0000000..e77b11a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment_twitter/mysentiment_twitter.9.query.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+select value count(t.text)
+from Tweet t;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum/mysum.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp
index c825a93..4f2f3b6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin bad target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin bad target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp
index 35d6ef8..8f2af66 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.2.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib root admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java root admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_bad_credential/mysum_bad_credential.3.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysum_dropinuse/mysum_dropinuse.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp
index 0f0a05b..ef2c4e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.1.lib.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-install test testlib admin admin target/TweetSent.pyz
+install test testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.2.ddl.sqlpp
index 6bbeaa1..0ad9fb3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.2.ddl.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.2.ddl.sqlpp
@@ -21,3 +21,6 @@
 
 create function warning()
   as "roundtrip", "Tests.warning" at testlib;
+
+create function roundtrip(s)
+  as "roundtrip", "Tests.roundtrip" at testlib;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.4.query.sqlpp
new file mode 100644
index 0000000..1ef35a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_function_error/py_function_error.4.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Access a records nested records at each level.
+* Expected Res : Success
+* Date         : 04 Jun 2015
+*/
+// param max-warnings:json=0
+
+use test;
+
+set `rewrite_attempt_batch_assign` "false";
+
+select warning(), roundtrip(d)
+from [ "a", "b" , "c", "d" ] d;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
index 0f0a05b..ef2c4e9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.1.lib.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-install test testlib admin admin target/TweetSent.pyz
+install test testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.10.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.10.query.sqlpp
index c48dda5..30d9426 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.10.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.10.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification.lower
+with  result as roundtrip(test)[0].class.fullClassification.lower
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.11.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.11.query.sqlpp
index 30ec1da..9c3e0cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.11.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.11.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification
+with  result as roundtrip(test)[0].class.fullClassification
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.12.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.12.query.sqlpp
index 1f7925f..2aed607 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.12.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.12.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class
+with  result as roundtrip(test)[0].class
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.13.query.sqlpp
index 13b4c28..52705db 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.13.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.13.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0]
+with  result as roundtrip(test)[0]
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp
index 1e9a088..e17f03f 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.4.query.sqlpp
@@ -27,6 +27,5 @@
 
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification.lower.lower.lower.lower.lower.lower.Species
-order by result
-;
+with  result as roundtrip(test)[0].class.fullClassification.lower.lower.lower.lower.lower.lower.Species
+order by result;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.5.query.sqlpp
index eedf56b..621b4d6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.5.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.5.query.sqlpp
@@ -23,9 +23,9 @@
 */
 use test;
 
-select value [(
+select value (
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification.lower.lower.lower.lower.lower.lower
-order by result.id )][0]
+with  result as roundtrip(test)[0].class.fullClassification.lower.lower.lower.lower.lower.lower
+order by result.id )
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.6.query.sqlpp
index 4912ad6..8e2ec95 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.6.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.6.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification.lower.lower.lower.lower.lower
+with  result as roundtrip(test)[0].class.fullClassification.lower.lower.lower.lower.lower
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.7.query.sqlpp
index 546c174..c39d933 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.7.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.7.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification.lower.lower.lower.lower
+with  result as roundtrip(test)[0].class.fullClassification.lower.lower.lower.lower
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.8.query.sqlpp
index 62af850..9a20dc2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.8.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.8.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification.lower.lower.lower
+with  result as roundtrip(test)[0].class.fullClassification.lower.lower.lower
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.9.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.9.query.sqlpp
index 6c594f0..0314f22 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.9.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/py_nested_access/py_nested_access.9.query.sqlpp
@@ -28,6 +28,6 @@
 select value [(
 select element result
 from  Animals as test
-with  result as roundtrip(test)[0][0].class.fullClassification.lower.lower
+with  result as roundtrip(test)[0].class.fullClassification.lower.lower
 order by result.id)][0]
 ;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp
index 1650910..699e565 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python-fn-escape/python-fn-escape.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/TweetSent.pyz
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.0.ddl.sqlpp
new file mode 100644
index 0000000..76cc70d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.0.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.1.lib.sqlpp
new file mode 100644
index 0000000..3250a90
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.1.lib.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.2.ddl.sqlpp
new file mode 100644
index 0000000..a8ba8a1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.2.ddl.sqlpp
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+use externallibtest;
+
+create function typeValidation(a, b, c, d, e, f, g)
+  as "roundtrip", "Tests.roundtrip" at testlib;
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.3.query.sqlpp
new file mode 100644
index 0000000..0ae7d0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallibtest;
+typeValidation(907, 9.07, "907", 9.07, true, unix_time_from_date_in_days(date("2013-01-01")),
+               unix_time_from_datetime_in_secs(datetime("1989-09-07T12:13:14.039Z")));
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.4.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.4.ddl.sqlpp
new file mode 100644
index 0000000..2b27030
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/python_open_type_validation/type_validation.4.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE externallibtest;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/return_invalid_type/return_invalid_type.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.0.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.0.ddl.sqlpp
new file mode 100644
index 0000000..76cc70d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.0.ddl.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+DROP DATAVERSE externallibtest if exists;
+CREATE DATAVERSE  externallibtest;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.1.lib.sqlpp
new file mode 100644
index 0000000..699e565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.1.lib.sqlpp
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+install externallibtest testlib python admin admin target/TweetSent.pyz
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.2.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.2.ddl.sqlpp
new file mode 100644
index 0000000..d6a4ea7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.2.ddl.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallibtest;
+
+create function sqrt(s)
+  as "roundtrip", "sqrt" at testlib;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.3.query.sqlpp
new file mode 100644
index 0000000..755d980
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.3.query.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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 externallibtest;
+
+sqrt(4);
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.4.ddl.sqlpp
similarity index 100%
copy from asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/mysentiment/mysentiment.6.ddl.sqlpp
copy to asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/toplevel_fn/mysentiment.4.ddl.sqlpp
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp
index 6bbdd0e..b2bf929 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/type_validation/type_validation.1.lib.sqlpp
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp
index b4197f1..009f699 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/udf_metadata/udf_metadata.1.lib.sqlpp
@@ -16,5 +16,5 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
-install externallibtest2 testlib2 admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest2 testlib2 java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp
index 3dc6eb6..cc6c855 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/external-library/upperCase/upperCase.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp
index 45cdfd1..fdb4118 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/exception_create_system_adapter/exception_create_system_adapter.1.lib.sqlpp
@@ -21,4 +21,4 @@
  * Expected Res : Success
  */
 
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp
index 806ad5e..76cd853 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter-cross-dv/feed-with-external-adapter-cross-dv.1.lib.sqlpp
@@ -21,4 +21,4 @@
  * Expected Res : Success
  */
 
-install externallibtest2 testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
+install externallibtest2 testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp
index 253c657..2370125 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-adapter/feed-with-external-adapter.1.lib.sqlpp
@@ -21,4 +21,4 @@
  * Expected Res : Success
  */
 
-install externallibtest testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
+install externallibtest testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp
index f3ea94f..2b25300 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/feeds/feed-with-external-function/feed-with-external-function.1.lib.sqlpp
@@ -16,4 +16,4 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-install udfs testlib admin admin target/data/externallib/asterix-external-data-testlib.zip
+install udfs testlib java admin admin target/data/externallib/asterix-external-data-testlib.zip
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http
index 1b14773..1606124 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.10.post.http
@@ -17,4 +17,7 @@
  * under the License.
  */
 
-/admin/rebalance?nodes=asterix_nc1%2Casterix_nc2
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http
index af6b1cf..81c5e88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets/all_datasets.4.post.http
@@ -17,4 +17,6 @@
  * under the License.
  */
 
-/admin/rebalance?nodes=asterix_nc1
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
index 1b14773..1606124 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.10.post.http
@@ -17,4 +17,7 @@
  * under the License.
  */
 
-/admin/rebalance?nodes=asterix_nc1%2Casterix_nc2
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
index af6b1cf..81c5e88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/all_datasets_compressed/all_datasets_compressed.4.post.http
@@ -17,4 +17,6 @@
  * under the License.
  */
 
-/admin/rebalance?nodes=asterix_nc1
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http
index 3828950..bef5fe4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/duplicate_location/duplicate_location.3.post.http
@@ -17,4 +17,10 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1%2Casterix_nc1%2Casterix_nc1
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http
index a488876..4409856 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/empty_location/empty_location.3.post.http
@@ -17,4 +17,7 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=
+# param dataverseName=tpch
+# param datasetName=LineItem
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http
index d5427da..98b5ed1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/identical_location/identical_location.3.post.http
@@ -17,4 +17,10 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc2%2Casterix_nc1&force=false
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc2
+# param targetNode=asterix_nc1
+# param force=false
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http
index 53a6b6a..6f36577 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/metadata/metadata.1.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=Metadata&datasetName=Dataset&nodes=asterix_nc1
+# param dataverseName=Metadata
+# param datasetName=Dataset
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http
index 81060ca..56e4d6b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/miss_dataverse/miss_dataverse.3.post.http
@@ -17,4 +17,7 @@
  * under the License.
  */
 
-/admin/rebalance?datasetName=LineItem&nodes=asterix_nc1
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http
index 0065c1f..8de754d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/nonexist_dataset/nonexist_dataset.1.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http
index 0065c1f..8de754d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.4.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http
index 44d7f33..59760a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset/single_dataset.8.post.http
@@ -17,4 +17,9 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1%2Casterix_nc2
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
index 0065c1f..8de754d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.4.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
index 44d7f33..59760a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_compressed/single_dataset_compressed.8.post.http
@@ -17,4 +17,9 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1%2Casterix_nc2
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http
index 0065c1f..8de754d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.4.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http
index 44d7f33..59760a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.9.post.http
@@ -17,4 +17,9 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1%2Casterix_nc2
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
index 0065c1f..8de754d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.4.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
index 44d7f33..59760a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index_compressed/single_dataset_with_index_compressed.9.post.http
@@ -17,4 +17,9 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&datasetName=LineItem&nodes=asterix_nc1%2Casterix_nc2
+# param dataverseName=tpch
+# param datasetName=LineItem
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http
index 6b713bc..16db13b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.10.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&nodes=asterix_nc1%2Casterix_nc2
+# param dataverseName=tpch
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http
index b3897e5..3b14ca7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse/single_dataverse.4.post.http
@@ -17,4 +17,7 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&nodes=asterix_nc1
+# param dataverseName=tpch
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
index 6b713bc..16db13b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.10.post.http
@@ -17,4 +17,8 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&nodes=asterix_nc1%2Casterix_nc2
+# param dataverseName=tpch
+# param targetNode=asterix_nc1
+# param targetNode=asterix_nc2
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
index b3897e5..3b14ca7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataverse_compressed/single_dataverse_compressed.4.post.http
@@ -17,4 +17,7 @@
  * under the License.
  */
 
-/admin/rebalance?dataverseName=tpch&nodes=asterix_nc1
+# param dataverseName=tpch
+# param targetNode=asterix_nc1
+
+/admin/rebalance
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
index a3ea801..cd69f25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.10.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
+# param partition=3
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
index 36e1d00..ed393b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload/bulkload.9.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
+# param partition=2
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http
index a3ea801..cd69f25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.10.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
+# param partition=3
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http
index 36e1d00..ed393b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/bulkload_with_compression/bulkload.9.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
+# param partition=2
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
index 2e8fc63..7e80cbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.6.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=0&node=asterix_nc2
\ No newline at end of file
+# param partition=0
+# param node=asterix_nc2
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
index e8dca0b..73aaa09 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component/flushed_component.7.post.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/metadataNode?node=asterix_nc2
\ No newline at end of file
+# param node=asterix_nc2
+
+/admin/cluster/metadataNode
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http
index 36e1d00..ed393b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.10.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
+# param partition=2
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http
index a3ea801..cd69f25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/flushed_component_compressed/flushed_component_compressed.11.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
+# param partition=3
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
index a3ea801..cd69f25 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.10.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=3&node=asterix_nc1
\ No newline at end of file
+# param partition=3
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
index 36e1d00..ed393b9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/mem_component_recovery/mem_component_recovery.9.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=2&node=asterix_nc1
\ No newline at end of file
+# param partition=2
+# param node=asterix_nc1
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.6.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.6.post.http
index 2e8fc63..7e80cbb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.6.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.6.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=0&node=asterix_nc2
\ No newline at end of file
+# param partition=0
+# param node=asterix_nc2
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.7.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.7.post.http
index e8dca0b..73aaa09 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.7.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/metadata_failover/metadata_failover.7.post.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/metadataNode?node=asterix_nc2
\ No newline at end of file
+# param node=asterix_nc2
+
+/admin/cluster/metadataNode
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http
index bb942d8..86363c0 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/replication/release_partition/release_partition.3.post.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-nc:asterix_nc1 /admin/storage/release?partition=0
\ No newline at end of file
+# param partition=0
+
+nc:asterix_nc1 /admin/storage/release
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.1.ddl.sqlpp
new file mode 100644
index 0000000..180cf55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.1.ddl.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description: This test case is to verify the fix for ASTERIXDB-2845
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create dataset jds(jid integer not unknown) open type primary key jid;
+
+create dataset mds(mid integer not unknown) open type primary key mid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.2.update.sqlpp
new file mode 100644
index 0000000..dd1b82d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.2.update.sqlpp
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use test;
+
+insert into jds ([
+  { "jid":1, "a":100 },
+  { "jid":2, "a":200 },
+  { "jid":3, "a":300 }
+]);
+
+insert into mds ([
+  { "mid":1, "jid":1, "x": 1, "y": 10 },
+  { "mid":2, "jid":1, "x": 1, "y": 20 },
+  { "mid":3, "jid":1, "x": 2, "y": 10 },
+  { "mid":4, "jid":1, "x": 2, "y": 20 },
+  { "mid":5, "jid":2, "x": 1, "y": 10 },
+  { "mid":6, "jid":2, "x": 1, "y": 20 },
+  { "mid":7, "jid":2, "x": 2, "y": 10 },
+  { "mid":8, "jid":2, "x": 2, "y": 20 }
+]);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.3.query.sqlpp
new file mode 100644
index 0000000..e6db7f2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.3.query.sqlpp
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+SET `compiler.sort.parallel` "false";
+
+WITH
+j AS (
+  SELECT jid, a
+  FROM jds
+), ---> 3 rows (jid=1, 2, 3)
+
+m1 AS (
+  SELECT jid, x, COUNT(1) c1
+  FROM mds
+  GROUP BY jid, x
+), ---> 4 rows ( 2 with jid=1, 2 with jid=2 )
+
+m2 AS (
+  SELECT jid, y, COUNT(1) c2
+  FROM mds
+  GROUP BY jid, y
+)  ---> 4 rows ( 2 with jid=1, 2 with jid=2 )
+
+SELECT j.jid AS j_jid, j.a AS j_a,
+  m1.jid AS m1_jid, m1.x AS m1_x, m1.c1 AS m1_c1,
+  m2.jid AS m2_jid, m2.y AS m2_y, m2.c2 AS m2_c2
+FROM j
+LEFT OUTER JOIN m1 ON j.jid=m1.jid ---> 5 rows (2 with jid=1, 2 with jid=2, 1 with jid=3)
+LEFT OUTER JOIN m2 ON j.jid=m1.jid ---> this predicate is intentional to reproduce the issue
+                                   ---> 17 rows (4 x 4 + 1 with jid=3 because m1.jid is MISSING for it)
+ORDER BY j_jid, m1_x, m2_y, m2_jid;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml b/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
index ebafab8..45077a9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/rebalance.xml
@@ -28,7 +28,7 @@
         <output-dir compare="Text">empty_location</output-dir>
         <expected-error>HTTP operation failed:
 STATUS LINE: HTTP/1.1 400 Bad Request
-ERROR_BODY: {"results":"target nodes should not be empty"}</expected-error>
+ERROR_BODY: {"results":"at least one targetNode must be specified"}</expected-error>
       </compilation-unit>
     </test-case>
     <test-case FilePath="rebalance">
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.adm
new file mode 100644
index 0000000..3b9c0aa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-subclause/agg_filter_01/agg_filter_01.10.adm
@@ -0,0 +1,10 @@
+{ "ten": 0, "cnt": 500, "min2": 0, "max2": 0, "sum20": 5000 }
+{ "ten": 1, "cnt": 1000, "min2": 1, "max2": 1, "sum20": 6000 }
+{ "ten": 2, "cnt": 500, "min2": 0, "max2": 0, "sum20": 1000 }
+{ "ten": 3, "cnt": 1000, "min2": 1, "max2": 1, "sum20": 8000 }
+{ "ten": 4, "cnt": 500, "min2": 0, "max2": 0, "sum20": 7000 }
+{ "ten": 5, "cnt": 1000, "min2": 1, "max2": 1, "sum20": 10000 }
+{ "ten": 6, "cnt": 500, "min2": 0, "max2": 0, "sum20": 3000 }
+{ "ten": 7, "cnt": 1000, "min2": 1, "max2": 1, "sum20": 12000 }
+{ "ten": 8, "cnt": 500, "min2": 0, "max2": 0, "sum20": 9000 }
+{ "ten": 9, "cnt": 1000, "min2": 1, "max2": 1, "sum20": 14000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.adm
new file mode 100644
index 0000000..935fe6d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/aggregate-subclause/agg_filter_01/agg_filter_01.9.adm
@@ -0,0 +1 @@
+{ "cnt": 7500, "min2": 0, "max2": 1, "sum20": 75000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/feed-stats/feed-stats.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/feed-stats/feed-stats.1.adm
deleted file mode 100644
index 0fcdf15..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/feed-stats/feed-stats.1.adm
+++ /dev/null
@@ -1,10 +0,0 @@
-{
-  "experiments.TweetFeed(Feed)" : {
-    "Stats" : [ {
-      "adapter-stats" : {
-        "incoming-records-count" : 13,
-        "failed-at-parser-records-count" : 3
-      }
-    } ]
-  }
-}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/feed-stats/feed-stats.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/feed-stats/feed-stats.5.regexjson
new file mode 100644
index 0000000..54028c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/feed-stats/feed-stats.5.regexjson
@@ -0,0 +1,13 @@
+{
+  "experiments.TweetFeed(Feed)": {
+    "Stats": [
+      {
+        "adapter-stats": {
+          "timestamp": "R{[0-9]+}",
+          "incoming-records-count": 13,
+          "failed-at-parser-records-count": 3
+        }
+      }
+    ]
+  }
+}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/invalid-library-params/invalid-library-params.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/invalid-library-params/invalid-library-params.1.regexjson
new file mode 100644
index 0000000..8cabc55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/invalid-library-params/invalid-library-params.1.regexjson
@@ -0,0 +1,3 @@
+{
+	"error": "ASX0053: Parameters type and delete cannot be used together",
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/crash/crash.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/crash/crash.1.adm
new file mode 100644
index 0000000..ec747fa
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/crash/crash.1.adm
@@ -0,0 +1 @@
+null
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.1.regexjson
new file mode 100644
index 0000000..9e26dfe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.1.regexjson
@@ -0,0 +1 @@
+{}
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.2.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.2.regexjson
new file mode 100644
index 0000000..9f289d1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api/library_list_ap1.2.regexjson
@@ -0,0 +1,5 @@
+[{
+	"dataverse": "externallibtest",
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+}]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.1.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.1.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.1.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.2.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.2.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.2.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.3.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.3.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.3.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.4.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.4.regexjson
new file mode 100644
index 0000000..0967ef4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.4.regexjson
@@ -0,0 +1 @@
+{}
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
new file mode 100644
index 0000000..c896e0d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/library_list_api_multipart/library_list_api.5.regexjson
@@ -0,0 +1,20 @@
+[{
+	"dataverse": ["external", "lib", "test"],
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+},
+{
+	"dataverse": "externallibtest",
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+},
+{
+	"dataverse": ["externallibtest", "foo"],
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+},
+{
+	"dataverse": ["externallibtest", "foo", "bar"],
+	"hash_md5": "R{[a-zA-Z0-9-]+}",
+	"name": "testlib"
+}]
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment/mysentiment.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment/mysentiment.4.adm
new file mode 100644
index 0000000..d7f41ee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment/mysentiment.4.adm
@@ -0,0 +1 @@
+{ "peachy": 1, "phlegmatic": 0, "indifferent": 0, "choleric": 0 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.1.adm
new file mode 100644
index 0000000..e9c02da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.1.adm
@@ -0,0 +1 @@
+5000
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.10.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.10.adm
new file mode 100644
index 0000000..878726a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.10.adm
@@ -0,0 +1 @@
+5100
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.11.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.11.adm
new file mode 100644
index 0000000..0ead4c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.11.adm
@@ -0,0 +1 @@
+5100
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.12.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.12.adm
new file mode 100644
index 0000000..878726a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.12.adm
@@ -0,0 +1 @@
+5100
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.13.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.13.adm
new file mode 100644
index 0000000..65a7c81
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.13.adm
@@ -0,0 +1,100 @@
+{ "id": 670301227662491648, "len": 20, "sent": 1 }
+{ "id": 670301227553566720, "len": 139, "sent": 0 }
+{ "id": 670301227041857536, "len": 112, "sent": 0 }
+{ "id": 670301227037519876, "len": 33, "sent": 0 }
+{ "id": 670301226987159552, "len": 57, "sent": 0 }
+{ "id": 670301226513391616, "len": 28, "sent": 1 }
+{ "id": 670301226337202180, "len": 77, "sent": 1 }
+{ "id": 670301226190278656, "len": 25, "sent": 0 }
+{ "id": 670301225959579648, "len": 112, "sent": 1 }
+{ "id": 670301225838125056, "len": 107, "sent": 0 }
+{ "id": 670301225598906369, "len": 64, "sent": 0 }
+{ "id": 670301225489817600, "len": 49, "sent": 0 }
+{ "id": 670301225456308224, "len": 103, "sent": 0 }
+{ "id": 670301225326391296, "len": 66, "sent": 1 }
+{ "id": 670301225162661889, "len": 28, "sent": 1 }
+{ "id": 670301224885837824, "len": 63, "sent": 0 }
+{ "id": 670301224814698496, "len": 59, "sent": 0 }
+{ "id": 670301224709849090, "len": 33, "sent": 1 }
+{ "id": 670301224684556288, "len": 21, "sent": 0 }
+{ "id": 670301224680480768, "len": 39, "sent": 0 }
+{ "id": 670301224348946433, "len": 64, "sent": 1 }
+{ "id": 670301224261058560, "len": 61, "sent": 1 }
+{ "id": 670301224231690240, "len": 33, "sent": 0 }
+{ "id": 670301224214794240, "len": 33, "sent": 0 }
+{ "id": 670301223753351168, "len": 105, "sent": 1 }
+{ "id": 670301223426367488, "len": 23, "sent": 0 }
+{ "id": 670301223216545792, "len": 31, "sent": 0 }
+{ "id": 670301223182974976, "len": 34, "sent": 1 }
+{ "id": 670301223128535041, "len": 21, "sent": 0 }
+{ "id": 670301222759301121, "len": 132, "sent": 0 }
+{ "id": 670301222734307329, "len": 110, "sent": 1 }
+{ "id": 670301222717419520, "len": 81, "sent": 0 }
+{ "id": 670301222318936064, "len": 110, "sent": 1 }
+{ "id": 670301222302150657, "len": 131, "sent": 0 }
+{ "id": 670301222222602240, "len": 43, "sent": 1 }
+{ "id": 670301222113517568, "len": 27, "sent": 0 }
+{ "id": 670301221836615680, "len": 44, "sent": 1 }
+{ "id": 670301221719310336, "len": 28, "sent": 0 }
+{ "id": 670301221442486272, "len": 34, "sent": 0 }
+{ "id": 670301221266153472, "len": 86, "sent": 0 }
+{ "id": 670301220960096256, "len": 102, "sent": 0 }
+{ "id": 670301220855136256, "len": 129, "sent": 1 }
+{ "id": 670301220637044736, "len": 43, "sent": 0 }
+{ "id": 670301220305821696, "len": 140, "sent": 0 }
+{ "id": 670301220247072770, "len": 83, "sent": 1 }
+{ "id": 670301220196626432, "len": 36, "sent": 0 }
+{ "id": 670301220079312901, "len": 31, "sent": 1 }
+{ "id": 670301219949305857, "len": 70, "sent": 1 }
+{ "id": 670301219739574273, "len": 131, "sent": 1 }
+{ "id": 670301219206877184, "len": 27, "sent": 0 }
+{ "id": 670301219139620864, "len": 124, "sent": 0 }
+{ "id": 670301218737123328, "len": 124, "sent": 0 }
+{ "id": 670301218640531458, "len": 31, "sent": 1 }
+{ "id": 670301218598756352, "len": 47, "sent": 0 }
+{ "id": 670301218565156865, "len": 44, "sent": 0 }
+{ "id": 670301218414206976, "len": 71, "sent": 1 }
+{ "id": 670301218376413185, "len": 14, "sent": 0 }
+{ "id": 670301218078629888, "len": 9, "sent": 0 }
+{ "id": 670301217851990017, "len": 111, "sent": 0 }
+{ "id": 670301217793269760, "len": 113, "sent": 0 }
+{ "id": 670301217508036608, "len": 47, "sent": 0 }
+{ "id": 670301217369657344, "len": 137, "sent": 0 }
+{ "id": 670301217311088641, "len": 28, "sent": 0 }
+{ "id": 670301217231347712, "len": 123, "sent": 0 }
+{ "id": 670301216891473920, "len": 44, "sent": 0 }
+{ "id": 670301216874721280, "len": 68, "sent": 0 }
+{ "id": 670301216799232000, "len": 50, "sent": 1 }
+{ "id": 670301216669171713, "len": 54, "sent": 0 }
+{ "id": 670301216493060097, "len": 113, "sent": 1 }
+{ "id": 670301216400924676, "len": 35, "sent": 1 }
+{ "id": 670301216371552258, "len": 58, "sent": 0 }
+{ "id": 670301216367185920, "len": 48, "sent": 0 }
+{ "id": 670301216228831232, "len": 130, "sent": 1 }
+{ "id": 670301215901802496, "len": 71, "sent": 1 }
+{ "id": 670301215725649921, "len": 20, "sent": 0 }
+{ "id": 670301215306199040, "len": 35, "sent": 0 }
+{ "id": 670301215138250754, "len": 48, "sent": 0 }
+{ "id": 670301214958055424, "len": 58, "sent": 1 }
+{ "id": 670301214605733888, "len": 139, "sent": 1 }
+{ "id": 670301214509129728, "len": 114, "sent": 1 }
+{ "id": 670301214442041344, "len": 18, "sent": 1 }
+{ "id": 670301214295392256, "len": 47, "sent": 0 }
+{ "id": 670301213737529344, "len": 9, "sent": 0 }
+{ "id": 670301213544595457, "len": 63, "sent": 1 }
+{ "id": 670301213515235333, "len": 107, "sent": 0 }
+{ "id": 670301213464899584, "len": 105, "sent": 1 }
+{ "id": 670301213120942080, "len": 39, "sent": 0 }
+{ "id": 670301212961603585, "len": 63, "sent": 0 }
+{ "id": 670301212961603584, "len": 20, "sent": 0 }
+{ "id": 670301212856737792, "len": 51, "sent": 0 }
+{ "id": 670301212760117248, "len": 133, "sent": 1 }
+{ "id": 670301211808010240, "len": 103, "sent": 0 }
+{ "id": 670301211774468096, "len": 40, "sent": 0 }
+{ "id": 670301211703144450, "len": 138, "sent": 1 }
+{ "id": 670301211581685761, "len": 25, "sent": 1 }
+{ "id": 670301211560685568, "len": 12, "sent": 1 }
+{ "id": 670301211090751490, "len": 140, "sent": 0 }
+{ "id": 670301210654699520, "len": 13, "sent": 0 }
+{ "id": 670301210486919168, "len": 38, "sent": 0 }
+{ "id": 670301210470195200, "len": 67, "sent": 1 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.2.adm
new file mode 100644
index 0000000..0b3e0a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.2.adm
@@ -0,0 +1 @@
+5000
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.3.adm
new file mode 100644
index 0000000..1995b70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.3.adm
@@ -0,0 +1,100 @@
+{ "text": 65, "sent": 0 }
+{ "text": 47, "sent": 0 }
+{ "text": 68, "sent": 0 }
+{ "text": 60, "sent": 0 }
+{ "text": 26, "sent": 1 }
+{ "text": 90, "sent": 1 }
+{ "text": 89, "sent": 0 }
+{ "text": 36, "sent": 0 }
+{ "text": 16, "sent": 0 }
+{ "text": 67, "sent": 0 }
+{ "text": 26, "sent": 0 }
+{ "text": 103, "sent": 1 }
+{ "text": 38, "sent": 1 }
+{ "text": 23, "sent": 1 }
+{ "text": 134, "sent": 1 }
+{ "text": 18, "sent": 0 }
+{ "text": 13, "sent": 1 }
+{ "text": 140, "sent": 0 }
+{ "text": 70, "sent": 1 }
+{ "text": 122, "sent": 0 }
+{ "text": 64, "sent": 0 }
+{ "text": 135, "sent": 0 }
+{ "text": 42, "sent": 1 }
+{ "text": 59, "sent": 0 }
+{ "text": 23, "sent": 1 }
+{ "text": 15, "sent": 1 }
+{ "text": 10, "sent": 0 }
+{ "text": 39, "sent": 1 }
+{ "text": 56, "sent": 0 }
+{ "text": 35, "sent": 0 }
+{ "text": 98, "sent": 1 }
+{ "text": 9, "sent": 0 }
+{ "text": 21, "sent": 0 }
+{ "text": 52, "sent": 0 }
+{ "text": 44, "sent": 0 }
+{ "text": 135, "sent": 0 }
+{ "text": 50, "sent": 0 }
+{ "text": 32, "sent": 0 }
+{ "text": 45, "sent": 0 }
+{ "text": 47, "sent": 0 }
+{ "text": 105, "sent": 0 }
+{ "text": 77, "sent": 0 }
+{ "text": 33, "sent": 0 }
+{ "text": 64, "sent": 0 }
+{ "text": 12, "sent": 0 }
+{ "text": 27, "sent": 1 }
+{ "text": 30, "sent": 0 }
+{ "text": 140, "sent": 1 }
+{ "text": 107, "sent": 1 }
+{ "text": 47, "sent": 0 }
+{ "text": 31, "sent": 0 }
+{ "text": 32, "sent": 1 }
+{ "text": 24, "sent": 0 }
+{ "text": 132, "sent": 0 }
+{ "text": 88, "sent": 0 }
+{ "text": 16, "sent": 0 }
+{ "text": 69, "sent": 0 }
+{ "text": 80, "sent": 0 }
+{ "text": 28, "sent": 1 }
+{ "text": 23, "sent": 0 }
+{ "text": 42, "sent": 0 }
+{ "text": 101, "sent": 1 }
+{ "text": 30, "sent": 0 }
+{ "text": 138, "sent": 0 }
+{ "text": 66, "sent": 0 }
+{ "text": 61, "sent": 0 }
+{ "text": 51, "sent": 1 }
+{ "text": 107, "sent": 0 }
+{ "text": 136, "sent": 0 }
+{ "text": 17, "sent": 0 }
+{ "text": 36, "sent": 1 }
+{ "text": 23, "sent": 0 }
+{ "text": 20, "sent": 1 }
+{ "text": 103, "sent": 0 }
+{ "text": 8, "sent": 0 }
+{ "text": 139, "sent": 0 }
+{ "text": 114, "sent": 0 }
+{ "text": 57, "sent": 1 }
+{ "text": 30, "sent": 0 }
+{ "text": 72, "sent": 0 }
+{ "text": 32, "sent": 0 }
+{ "text": 140, "sent": 0 }
+{ "text": 90, "sent": 1 }
+{ "text": 25, "sent": 0 }
+{ "text": 56, "sent": 0 }
+{ "text": 43, "sent": 0 }
+{ "text": 58, "sent": 0 }
+{ "text": 23, "sent": 0 }
+{ "text": 15, "sent": 1 }
+{ "text": 53, "sent": 1 }
+{ "text": 58, "sent": 1 }
+{ "text": 14, "sent": 0 }
+{ "text": 21, "sent": 1 }
+{ "text": 37, "sent": 0 }
+{ "text": 118, "sent": 0 }
+{ "text": 59, "sent": 0 }
+{ "text": 43, "sent": 0 }
+{ "text": 55, "sent": 0 }
+{ "text": 35, "sent": 1 }
+{ "text": 127, "sent": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.4.adm
new file mode 100644
index 0000000..1995b70
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.4.adm
@@ -0,0 +1,100 @@
+{ "text": 65, "sent": 0 }
+{ "text": 47, "sent": 0 }
+{ "text": 68, "sent": 0 }
+{ "text": 60, "sent": 0 }
+{ "text": 26, "sent": 1 }
+{ "text": 90, "sent": 1 }
+{ "text": 89, "sent": 0 }
+{ "text": 36, "sent": 0 }
+{ "text": 16, "sent": 0 }
+{ "text": 67, "sent": 0 }
+{ "text": 26, "sent": 0 }
+{ "text": 103, "sent": 1 }
+{ "text": 38, "sent": 1 }
+{ "text": 23, "sent": 1 }
+{ "text": 134, "sent": 1 }
+{ "text": 18, "sent": 0 }
+{ "text": 13, "sent": 1 }
+{ "text": 140, "sent": 0 }
+{ "text": 70, "sent": 1 }
+{ "text": 122, "sent": 0 }
+{ "text": 64, "sent": 0 }
+{ "text": 135, "sent": 0 }
+{ "text": 42, "sent": 1 }
+{ "text": 59, "sent": 0 }
+{ "text": 23, "sent": 1 }
+{ "text": 15, "sent": 1 }
+{ "text": 10, "sent": 0 }
+{ "text": 39, "sent": 1 }
+{ "text": 56, "sent": 0 }
+{ "text": 35, "sent": 0 }
+{ "text": 98, "sent": 1 }
+{ "text": 9, "sent": 0 }
+{ "text": 21, "sent": 0 }
+{ "text": 52, "sent": 0 }
+{ "text": 44, "sent": 0 }
+{ "text": 135, "sent": 0 }
+{ "text": 50, "sent": 0 }
+{ "text": 32, "sent": 0 }
+{ "text": 45, "sent": 0 }
+{ "text": 47, "sent": 0 }
+{ "text": 105, "sent": 0 }
+{ "text": 77, "sent": 0 }
+{ "text": 33, "sent": 0 }
+{ "text": 64, "sent": 0 }
+{ "text": 12, "sent": 0 }
+{ "text": 27, "sent": 1 }
+{ "text": 30, "sent": 0 }
+{ "text": 140, "sent": 1 }
+{ "text": 107, "sent": 1 }
+{ "text": 47, "sent": 0 }
+{ "text": 31, "sent": 0 }
+{ "text": 32, "sent": 1 }
+{ "text": 24, "sent": 0 }
+{ "text": 132, "sent": 0 }
+{ "text": 88, "sent": 0 }
+{ "text": 16, "sent": 0 }
+{ "text": 69, "sent": 0 }
+{ "text": 80, "sent": 0 }
+{ "text": 28, "sent": 1 }
+{ "text": 23, "sent": 0 }
+{ "text": 42, "sent": 0 }
+{ "text": 101, "sent": 1 }
+{ "text": 30, "sent": 0 }
+{ "text": 138, "sent": 0 }
+{ "text": 66, "sent": 0 }
+{ "text": 61, "sent": 0 }
+{ "text": 51, "sent": 1 }
+{ "text": 107, "sent": 0 }
+{ "text": 136, "sent": 0 }
+{ "text": 17, "sent": 0 }
+{ "text": 36, "sent": 1 }
+{ "text": 23, "sent": 0 }
+{ "text": 20, "sent": 1 }
+{ "text": 103, "sent": 0 }
+{ "text": 8, "sent": 0 }
+{ "text": 139, "sent": 0 }
+{ "text": 114, "sent": 0 }
+{ "text": 57, "sent": 1 }
+{ "text": 30, "sent": 0 }
+{ "text": 72, "sent": 0 }
+{ "text": 32, "sent": 0 }
+{ "text": 140, "sent": 0 }
+{ "text": 90, "sent": 1 }
+{ "text": 25, "sent": 0 }
+{ "text": 56, "sent": 0 }
+{ "text": 43, "sent": 0 }
+{ "text": 58, "sent": 0 }
+{ "text": 23, "sent": 0 }
+{ "text": 15, "sent": 1 }
+{ "text": 53, "sent": 1 }
+{ "text": 58, "sent": 1 }
+{ "text": 14, "sent": 0 }
+{ "text": 21, "sent": 1 }
+{ "text": 37, "sent": 0 }
+{ "text": 118, "sent": 0 }
+{ "text": 59, "sent": 0 }
+{ "text": 43, "sent": 0 }
+{ "text": 55, "sent": 0 }
+{ "text": 35, "sent": 1 }
+{ "text": 127, "sent": 0 }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.5.adm
new file mode 100644
index 0000000..e9c02da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.5.adm
@@ -0,0 +1 @@
+5000
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.6.adm
new file mode 100644
index 0000000..e9c02da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.6.adm
@@ -0,0 +1 @@
+5000
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.7.adm
new file mode 100644
index 0000000..e9c02da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.7.adm
@@ -0,0 +1 @@
+5000
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.8.adm
new file mode 100644
index 0000000..9fada79
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.8.adm
@@ -0,0 +1 @@
+{ "$1": 5000, "$2": 5000 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.9.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.9.adm
new file mode 100644
index 0000000..0ead4c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/mysentiment_twitter/mysentiment_twitter.9.adm
@@ -0,0 +1 @@
+5100
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/py_function_error/py_function_error.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/py_function_error/py_function_error.2.adm
new file mode 100644
index 0000000..f0405ed
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/py_function_error/py_function_error.2.adm
@@ -0,0 +1,4 @@
+{ "$1": null, "$2": [ "a" ] }
+{ "$1": null, "$2": [ "b" ] }
+{ "$1": null, "$2": [ "c" ] }
+{ "$1": null, "$2": [ "d" ] }
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/python_open_type_validation/type_validation.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/python_open_type_validation/type_validation.1.adm
new file mode 100644
index 0000000..93f8aec
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/python_open_type_validation/type_validation.1.adm
@@ -0,0 +1 @@
+[ 907, 9.07, "907", 9.07, true, 15706, 621173594 ]
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/toplevel_fn/toplevel_fn.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/toplevel_fn/toplevel_fn.1.adm
new file mode 100644
index 0000000..cd5ac03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/external-library/toplevel_fn/toplevel_fn.1.adm
@@ -0,0 +1 @@
+2.0
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
index 06a28e4..bb5ac24 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.8.adm
@@ -17,25 +17,21 @@
                 subplan {
                           aggregate [$$73] <- [listify($$72)]
                           -- AGGREGATE  |LOCAL|
-                            assign [$$72] <- [object-remove(object-remove(object-remove($$t1, "title"), "authors"), "misc")]
+                            assign [$$72] <- [object-remove(object-remove(object-remove($$t0, "title"), "authors"), "misc")]
                             -- ASSIGN  |LOCAL|
-                              unnest $$t1 <- scan-collection($$64)
+                              unnest $$t0 <- scan-collection(to-array($$paper))
                               -- UNNEST  |LOCAL|
                                 nested tuple source
                                 -- NESTED_TUPLE_SOURCE  |LOCAL|
                        }
                 -- SUBPLAN  |PARTITIONED|
-                  project ([$$77, $$64])
-                  -- STREAM_PROJECT  |PARTITIONED|
-                    assign [$$64] <- [to-array($$paper)]
-                    -- ASSIGN  |PARTITIONED|
-                      limit 10
-                      -- STREAM_LIMIT  |PARTITIONED|
+                  limit 10
+                  -- STREAM_LIMIT  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      data-scan []<-[$$77, $$paper] <- test.DBLP1 limit 10
+                      -- DATASOURCE_SCAN  |PARTITIONED|
                         exchange
                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          data-scan []<-[$$77, $$paper] <- test.DBLP1 limit 10
-                          -- DATASOURCE_SCAN  |PARTITIONED|
-                            exchange
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              empty-tuple-source
-                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
+                          empty-tuple-source
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.3.adm
new file mode 100644
index 0000000..c5939b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-2845/query-ASTERIXDB-2845.3.adm
@@ -0,0 +1,17 @@
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 1, "m2_jid": 1, "m2_y": 10 }
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 1, "m2_jid": 2, "m2_y": 10 }
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 1, "m2_jid": 1, "m2_y": 20 }
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 1, "m2_jid": 2, "m2_y": 20 }
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 2, "m2_jid": 1, "m2_y": 10 }
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 2, "m2_jid": 2, "m2_y": 10 }
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 2, "m2_jid": 1, "m2_y": 20 }
+{ "j_jid": 1, "m1_c1": 2, "m2_c2": 2, "j_a": 100, "m1_jid": 1, "m1_x": 2, "m2_jid": 2, "m2_y": 20 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 1, "m2_jid": 1, "m2_y": 10 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 1, "m2_jid": 2, "m2_y": 10 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 1, "m2_jid": 1, "m2_y": 20 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 1, "m2_jid": 2, "m2_y": 20 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 2, "m2_jid": 1, "m2_y": 10 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 2, "m2_jid": 2, "m2_y": 10 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 2, "m2_jid": 1, "m2_y": 20 }
+{ "j_jid": 2, "m1_c1": 2, "m2_c2": 2, "j_a": 200, "m1_jid": 2, "m1_x": 2, "m2_jid": 2, "m2_y": 20 }
+{ "j_jid": 3, "j_a": 300 }
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_1.09.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_1.09.ast
new file mode 100644
index 0000000..3cecd60
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_1.09.ast
@@ -0,0 +1,125 @@
+DataverseUse test
+Query:
+SELECT [
+FunctionCall asterix.sql-count@1[
+  (
+    SELECT ELEMENT [
+    LiteralExpr [LONG] [1]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#2 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#2 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+cnt
+FunctionCall asterix.sql-min@1[
+  (
+    SELECT ELEMENT [
+    FieldAccessor [
+      FieldAccessor [
+        Variable [ Name=#3 ]
+        Field=tenk
+      ]
+      Field=two
+    ]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#3 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#3 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+min2
+FunctionCall asterix.sql-max@1[
+  (
+    SELECT ELEMENT [
+    FieldAccessor [
+      FieldAccessor [
+        Variable [ Name=#4 ]
+        Field=tenk
+      ]
+      Field=two
+    ]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#4 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#4 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+max2
+FunctionCall asterix.sql-sum@1[
+  (
+    SELECT ELEMENT [
+    FieldAccessor [
+      FieldAccessor [
+        Variable [ Name=#5 ]
+        Field=tenk
+      ]
+      Field=twenty
+    ]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#5 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#5 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+sum20
+]
+FROM [  FunctionCall asterix.dataset@1[
+    LiteralExpr [STRING] [test.tenk]
+  ]
+  AS Variable [ Name=$tenk ]
+]
+Group All
+  GROUP AS Variable [ Name=#1 ]
+  (
+    tenk:=Variable [ Name=$tenk ]
+  )
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_1.10.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_1.10.ast
new file mode 100644
index 0000000..e82d78e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/aggregate-subclause/agg_filter_01/agg_filter_1.10.ast
@@ -0,0 +1,137 @@
+DataverseUse test
+Query:
+SELECT [
+Variable [ Name=$ten ]
+ten
+FunctionCall asterix.sql-count@1[
+  (
+    SELECT ELEMENT [
+    LiteralExpr [LONG] [1]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#2 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#2 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+cnt
+FunctionCall asterix.sql-min@1[
+  (
+    SELECT ELEMENT [
+    FieldAccessor [
+      FieldAccessor [
+        Variable [ Name=#3 ]
+        Field=tenk
+      ]
+      Field=two
+    ]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#3 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#3 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+min2
+FunctionCall asterix.sql-max@1[
+  (
+    SELECT ELEMENT [
+    FieldAccessor [
+      FieldAccessor [
+        Variable [ Name=#4 ]
+        Field=tenk
+      ]
+      Field=two
+    ]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#4 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#4 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+max2
+FunctionCall asterix.sql-sum@1[
+  (
+    SELECT ELEMENT [
+    FieldAccessor [
+      FieldAccessor [
+        Variable [ Name=#5 ]
+        Field=tenk
+      ]
+      Field=twenty
+    ]
+    ]
+    FROM [      Variable [ Name=#1 ]
+      AS Variable [ Name=#5 ]
+    ]
+    Where
+      OperatorExpr [
+        FieldAccessor [
+          FieldAccessor [
+            Variable [ Name=#5 ]
+            Field=tenk
+          ]
+          Field=four
+        ]
+        >
+        LiteralExpr [LONG] [0]
+      ]
+  )
+]
+sum20
+]
+FROM [  FunctionCall asterix.dataset@1[
+    LiteralExpr [STRING] [test.tenk]
+  ]
+  AS Variable [ Name=$tenk ]
+]
+Groupby
+  Variable [ Name=$ten ]
+  :=
+  FieldAccessor [
+    Variable [ Name=$tenk ]
+    Field=ten
+  ]
+  GROUP AS Variable [ Name=#1 ]
+  (
+    tenk:=Variable [ Name=$tenk ]
+  )
+
+Orderby
+  Variable [ Name=$ten ]
+  ASC
+
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
index e4669da..4e1d5b2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_python.xml
@@ -29,6 +29,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
+      <compilation-unit name="mysentiment_multipart">
+        <output-dir compare="Text">mysentiment</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
       <compilation-unit name="python-fn-escape">
         <output-dir compare="Text">python-fn-escape</output-dir>
         <expected-error>ImportError: Module was not found in library</expected-error>
@@ -39,10 +44,40 @@
         <output-dir compare="Clean-JSON">py_nested_access</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="python_open_type_validation">
+        <output-dir compare="Clean-JSON">python_open_type_validation</output-dir>
+      </compilation-unit>
+    </test-case>
     <test-case FilePath="external-library" check-warnings="true">
       <compilation-unit name="py_function_error">
         <output-dir compare="Clean-JSON">py_function_error</output-dir>
-        <expected-warn>ASX0201: External UDF returned exception. Returned exception was: ArithmeticError: oof</expected-warn>
+        <expected-warn>ASX0201: External UDF returned exception. Returned exception was: Traceback (most recent call last):
+  File "entrypoint.py", line 181, in handle_call
+    result[0].append(self.next_tuple(*arg, key=self.mid))
+  File "entrypoint.py", line 99, in next_tuple
+    return self.wrapped_fns[key](*args)
+  File "site-packages/roundtrip.py", line 28, in warning
+    raise ArithmeticError("oof")
+ArithmeticError: oof
+ (in line 28, at column 1)</expected-warn>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="mysentiment_twitter">
+        <output-dir compare="Text">mysentiment_twitter</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="toplevel_fn">
+        <output-dir compare="Text">toplevel_fn</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library" check-warnings="true">
+      <compilation-unit name="crash">
+        <output-dir compare="Text">crash</output-dir>
+        <expected-warn>ASX0201: External UDF returned exception. Returned exception was: Function externallibtest:crash#0 failed to execute (in line 23, at column 1)</expected-warn>
+        <expected-warn>ASX0201: External UDF returned exception. Returned exception was: java.io.IOException: Python process exited with code: 1 (in line 23, at column 1)</expected-warn>
       </compilation-unit>
     </test-case>
   </test-group>
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
index a9f52b4..28bbb98 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_it_sqlpp.xml
@@ -62,6 +62,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
+      <compilation-unit name="invalid_library_requests">
+        <output-dir compare="Text">mysum_bad_credential</output-dir>
+        <expected-error>ASX3042: Unsupported function language badType</expected-error>
+        <expected-error>ASX1110: The parameters \"type\" and \"delete\" cannot be provided at the same time</expected-error>
+        <expected-error>ASX0049: Parameter(s) type or delete must be specified</expected-error>
+        <expected-error>ASX0047: Invalid value for parameter \"data\": Attribute</expected-error>
+        <expected-error>ASX0049: Parameter(s) data must be specified</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
       <compilation-unit name="mysum_dropinuse">
         <output-dir compare="Text">mysum_dropinuse</output-dir>
         <expected-error>ASX1148: Cannot drop library externallibtest.testlib being used by function externallibtest.mysum(2)</expected-error>
@@ -73,6 +83,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="external-library">
+      <compilation-unit name="library_list_api">
+        <output-dir compare="Text">library_list_api</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
+      <compilation-unit name="library_list_api_multipart">
+        <output-dir compare="Text">library_list_api_multipart</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="external-library">
       <compilation-unit name="getCapital">
         <output-dir compare="Text">getCapital</output-dir>
       </compilation-unit>
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 2cdf027..6492ef9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -10840,6 +10840,11 @@
         <output-dir compare="Text">query-ASTERIXDB-2815</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="subquery">
+      <compilation-unit name="query-ASTERIXDB-2845">
+        <output-dir compare="Text">query-ASTERIXDB-2845</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="subset-collection">
     <test-case FilePath="subset-collection">
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
index a99a306..fc33b1a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/StorageProperties.java
@@ -55,7 +55,10 @@
         STORAGE_COMPRESSION_BLOCK(STRING, "snappy"),
         STORAGE_DISK_FORCE_BYTES(LONG_BYTE_UNIT, StorageUtil.getLongSizeInBytes(16, MEGABYTE)),
         STORAGE_IO_SCHEDULER(STRING, "greedy"),
-        STORAGE_WRITE_RATE_LIMIT(LONG_BYTE_UNIT, 0l);
+        STORAGE_WRITE_RATE_LIMIT(LONG_BYTE_UNIT, 0l),
+        STORAGE_MAX_RUNNING_FLUSHES_PER_PARTITION(NONNEGATIVE_INTEGER, 2),
+        STORAGE_MAX_SCHEDULED_MERGES_PER_PARTITION(NONNEGATIVE_INTEGER, 8),
+        STORAGE_MAX_RUNNING_MERGES_PER_PARTITION(NONNEGATIVE_INTEGER, 2);
 
         private final IOptionType interpreter;
         private final Object defaultValue;
@@ -111,6 +114,12 @@
                     return "The number of bytes before each disk force (fsync)";
                 case STORAGE_IO_SCHEDULER:
                     return "The I/O scheduler for LSM flush and merge operations";
+                case STORAGE_MAX_RUNNING_FLUSHES_PER_PARTITION:
+                    return "The maximum number of running flushes per partition (0 means unlimited)";
+                case STORAGE_MAX_SCHEDULED_MERGES_PER_PARTITION:
+                    return "The maximum number of scheduled merges per partition (0 means unlimited)";
+                case STORAGE_MAX_RUNNING_MERGES_PER_PARTITION:
+                    return "The maximum number of running merges per partition (0 means unlimited)";
                 default:
                     throw new IllegalStateException("NYI: " + this);
             }
@@ -204,6 +213,21 @@
         return accessor.getString(Option.STORAGE_IO_SCHEDULER);
     }
 
+    public int getMaxRunningFlushes(int numPartitions) {
+        int value = accessor.getInt(Option.STORAGE_MAX_RUNNING_FLUSHES_PER_PARTITION);
+        return value != 0 ? value * numPartitions : Integer.MAX_VALUE;
+    }
+
+    public int getMaxScheduledMerges(int numPartitions) {
+        int value = accessor.getInt(Option.STORAGE_MAX_SCHEDULED_MERGES_PER_PARTITION);
+        return value != 0 ? value * numPartitions : Integer.MAX_VALUE;
+    }
+
+    public int getMaxRunningMerges(int numPartitions) {
+        int value = accessor.getInt(Option.STORAGE_MAX_RUNNING_MERGES_PER_PARTITION);
+        return value != 0 ? value * numPartitions : Integer.MAX_VALUE;
+    }
+
     protected int getMetadataDatasets() {
         return MetadataIndexImmutableProperties.METADATA_DATASETS_COUNT;
     }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
index ef390f4..93fe92d 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/ILibraryManager.java
@@ -23,9 +23,11 @@
 import java.io.InputStream;
 import java.net.URI;
 import java.security.MessageDigest;
+import java.util.List;
 
 import org.apache.asterix.common.metadata.DataverseName;
 import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.api.io.FileReference;
@@ -33,6 +35,10 @@
 
 public interface ILibraryManager {
 
+    List<Pair<DataverseName, String>> getLibraryListing() throws IOException;
+
+    String getLibraryHash(DataverseName dataverseName, String libraryName) throws IOException;
+
     ILibrary getLibrary(DataverseName dataverseName, String libraryName) throws HyracksDataException;
 
     void closeLibrary(DataverseName dataverseName, String libraryName) throws HyracksDataException;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java
index 6f128dc..3825d98 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/library/LibraryDescriptor.java
@@ -31,14 +31,10 @@
  */
 public class LibraryDescriptor implements IJsonSerializable {
 
-    private static final long serialVersionUID = 2L;
+    private static final long serialVersionUID = 3L;
 
     private static final String FIELD_LANGUAGE = "lang";
-    private static final String FIELD_HASH = "hash";
-
-    public static final String FILE_EXT_ZIP = "zip";
-
-    public static final String FILE_EXT_PYZ = "pyz";
+    public static final String FIELD_HASH = "hash_md5";
 
     /**
      * The library's language
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/DataverseName.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/DataverseName.java
index 0eb3b5d..b8124a2 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/DataverseName.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/metadata/DataverseName.java
@@ -42,9 +42,8 @@
  * <p>
  * E.g. the canonical form for a dataverse name {@code ["a", "b", "c"]} is {@code "a.b.c"}
  * <p>
- * {@link #toString()} returns a display form which is a {@link #CANONICAL_FORM_SEPARATOR_CHAR '.'} separated
- * concatenation of name parts without escaping. In general it's impossible to reconstruct a dataverse name from
- * its display form.
+ * {@link #toString()} returns a display form which is suitable for error messages,
+ * and is a valid SQL++ multi-part identifier parsable by {@code IParser#parseMultipartIdentifier()}
  * <p>
  * Notes:
  * <li>
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/external/ipc/ExternalFunctionResultRouter.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/external/ipc/ExternalFunctionResultRouter.java
index 8d56eb7..94baa64 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/external/ipc/ExternalFunctionResultRouter.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/external/ipc/ExternalFunctionResultRouter.java
@@ -22,7 +22,7 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
-import org.apache.commons.lang3.mutable.MutableObject;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksException;
 import org.apache.hyracks.ipc.api.IIPCHandle;
@@ -33,9 +33,8 @@
 
 public class ExternalFunctionResultRouter implements IIPCI {
 
-    AtomicLong maxId = new AtomicLong(0);
-    ConcurrentHashMap<Long, MutableObject<ByteBuffer>> activeClients = new ConcurrentHashMap<>();
-    ConcurrentHashMap<Long, Exception> exceptionInbox = new ConcurrentHashMap<>();
+    private final AtomicLong maxId = new AtomicLong(0);
+    private final ConcurrentHashMap<Long, Pair<ByteBuffer, Exception>> activeClients = new ConcurrentHashMap<>();
     private static int MAX_BUF_SIZE = 32 * 1024 * 1024; //32MB
 
     @Override
@@ -44,7 +43,8 @@
         ByteBuffer buf = (ByteBuffer) payload;
         int end = buf.position();
         buf.position(end - rewind);
-        ByteBuffer copyTo = activeClients.get(rmid).getValue();
+        Pair<ByteBuffer, Exception> route = activeClients.get(rmid);
+        ByteBuffer copyTo = route.getFirst();
         if (copyTo.capacity() < handle.getAttachmentLen()) {
             int nextSize = closestPow2(handle.getAttachmentLen());
             if (nextSize > MAX_BUF_SIZE) {
@@ -52,44 +52,43 @@
                 return;
             }
             copyTo = ByteBuffer.allocate(nextSize);
-            activeClients.get(rmid).setValue(copyTo);
+            route.setFirst(copyTo);
         }
         copyTo.position(0);
         System.arraycopy(buf.array(), buf.position() + buf.arrayOffset(), copyTo.array(), copyTo.arrayOffset(),
                 handle.getAttachmentLen());
-        synchronized (copyTo) {
+        synchronized (route) {
             copyTo.limit(handle.getAttachmentLen() + 1);
-            copyTo.notify();
+            route.notifyAll();
         }
         buf.position(end);
     }
 
     @Override
     public void onError(IIPCHandle handle, long mid, long rmid, Exception exception) {
-        exceptionInbox.put(rmid, exception);
-        ByteBuffer route = activeClients.get(rmid).getValue();
+        Pair<ByteBuffer, Exception> route = activeClients.get(rmid);
         synchronized (route) {
-            route.notify();
+            route.setSecond(exception);
+            route.notifyAll();
         }
     }
 
-    public Long insertRoute(ByteBuffer buf) {
-        Long id = maxId.incrementAndGet();
-        activeClients.put(id, new MutableObject<>(buf));
-        return id;
+    public Pair<Long, Pair<ByteBuffer, Exception>> insertRoute(ByteBuffer buf) {
+        Long id = maxId.getAndIncrement();
+        Pair<ByteBuffer, Exception> bufferHolder = new Pair<>(buf, null);
+        activeClients.put(id, bufferHolder);
+        return new Pair<>(id, bufferHolder);
     }
 
-    public Exception getException(Long id) {
-        return exceptionInbox.remove(id);
-    }
-
-    public boolean hasException(long id) {
-        return exceptionInbox.get(id) == null;
+    public Exception getAndRemoveException(Long id) {
+        Pair<ByteBuffer, Exception> route = activeClients.get(id);
+        Exception e = route.getSecond();
+        route.setSecond(null);
+        return e;
     }
 
     public void removeRoute(Long id) {
         activeClients.remove(id);
-        exceptionInbox.remove(id);
     }
 
     public static int closestPow2(int n) {
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index cc56ede..de21959 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -283,7 +283,7 @@
 3039 = Cannot parse list item of type %1$s
 3040 = Argument type: %1$s
 3041 = Unable to load/instantiate class %1$s
-3042 = UDF of kind %1$s not supported
+3042 = Unsupported function language %1$s
 3043 = Unknown function kind %1$s
 3044 = Library class loader already registered!
 3045 = Cannot handle a function argument of type %1$s
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
index 3fb7db7..56257a8 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/dataflow/FeedRecordDataFlowController.java
@@ -42,6 +42,7 @@
     public static final String INCOMING_RECORDS_COUNT_FIELD_NAME = "incoming-records-count";
     public static final String FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME = "failed-at-parser-records-count";
     public static final String READER_STATS_FIELD_NAME = "reader-stats";
+    public static final String TIMESTAMP_FIELD_NAME = "timestamp";
 
     public enum State {
         CREATED,
@@ -271,11 +272,12 @@
         StringBuilder str = new StringBuilder();
         str.append("{");
         if (readerStats != null) {
-            str.append("\"").append(READER_STATS_FIELD_NAME).append("\":").append(readerStats).append(", ");
+            str.append("\"" + READER_STATS_FIELD_NAME + "\":").append(readerStats).append(",");
         }
-        str.append("\"").append(INCOMING_RECORDS_COUNT_FIELD_NAME).append("\": ").append(incomingRecordsCount)
-                .append(", \"").append(FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME).append("\": ")
-                .append(failedRecordsCount).append("}");
+        str.append("\"" + TIMESTAMP_FIELD_NAME + "\":").append(System.currentTimeMillis()).append(",");
+        str.append("\"" + INCOMING_RECORDS_COUNT_FIELD_NAME + "\":").append(incomingRecordsCount)
+                .append(",\"" + FAILED_AT_PARSER_RECORDS_COUNT_FIELD_NAME + "\":").append(failedRecordsCount)
+                .append("}");
         return str.toString();
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
index feb52cf..cd7ec18 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonIPCProto.java
@@ -24,32 +24,41 @@
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.hyracks.ipc.impl.IPCSystem;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.ipc.impl.Message;
 import org.msgpack.core.MessagePack;
+import org.msgpack.core.MessageUnpacker;
+import org.msgpack.core.buffer.ArrayBufferInput;
 
 public class PythonIPCProto {
 
-    public PythonMessageBuilder messageBuilder;
-    OutputStream sockOut;
-    ByteBuffer headerBuffer = ByteBuffer.allocate(21);
-    ByteBuffer recvBuffer = ByteBuffer.allocate(4096);
-    ExternalFunctionResultRouter router;
-    IPCSystem ipcSys;
-    Message outMsg;
-    Long key;
+    private PythonMessageBuilder messageBuilder;
+    private OutputStream sockOut;
+    private ByteBuffer headerBuffer = ByteBuffer.allocate(21);
+    private ByteBuffer recvBuffer = ByteBuffer.allocate(32768);
+    private ExternalFunctionResultRouter router;
+    private long routeId;
+    private Pair<ByteBuffer, Exception> bufferBox;
+    private Process pythonProc;
+    private long maxFunctionId;
+    private ArrayBufferInput unpackerInput;
+    private MessageUnpacker unpacker;
 
-    public PythonIPCProto(OutputStream sockOut, ExternalFunctionResultRouter router, IPCSystem ipcSys)
-            throws IOException {
+    public PythonIPCProto(OutputStream sockOut, ExternalFunctionResultRouter router, Process pythonProc) {
         this.sockOut = sockOut;
         messageBuilder = new PythonMessageBuilder();
         this.router = router;
-        this.ipcSys = ipcSys;
-        this.outMsg = new Message(null);
+        this.pythonProc = pythonProc;
+        this.maxFunctionId = 0l;
+        unpackerInput = new ArrayBufferInput(new byte[0]);
+        unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
     }
 
     public void start() {
-        this.key = router.insertRoute(recvBuffer);
+        Pair<Long, Pair<ByteBuffer, Exception>> keyAndBufferBox = router.insertRoute(recvBuffer);
+        this.routeId = keyAndBufferBox.getFirst();
+        this.bufferBox = keyAndBufferBox.getSecond();
     }
 
     public void helo() throws IOException, AsterixException {
@@ -59,78 +68,106 @@
         messageBuilder.buf.clear();
         messageBuilder.buf.position(0);
         messageBuilder.hello();
-        sendMsg();
+        sendMsg(routeId);
         receiveMsg();
         if (getResponseType() != MessageType.HELO) {
-            throw new IllegalStateException("Illegal reply received, expected HELO");
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected HELO, recieved " + getResponseType().name());
         }
     }
 
-    public void init(String module, String clazz, String fn) throws IOException, AsterixException {
+    public long init(String module, String clazz, String fn) throws IOException, AsterixException {
+        long functionId = maxFunctionId++;
         recvBuffer.clear();
         recvBuffer.position(0);
         recvBuffer.limit(0);
         messageBuilder.buf.clear();
         messageBuilder.buf.position(0);
         messageBuilder.init(module, clazz, fn);
-        sendMsg();
+        sendMsg(functionId);
         receiveMsg();
         if (getResponseType() != MessageType.INIT_RSP) {
-            throw new IllegalStateException("Illegal reply received, expected INIT_RSP");
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected INIT_RSP, recieved " + getResponseType().name());
         }
+        return functionId;
     }
 
-    public ByteBuffer call(ByteBuffer args, int numArgs) throws Exception {
+    public ByteBuffer call(long functionId, ByteBuffer args, int numArgs) throws IOException, AsterixException {
         recvBuffer.clear();
         recvBuffer.position(0);
         recvBuffer.limit(0);
         messageBuilder.buf.clear();
         messageBuilder.buf.position(0);
         messageBuilder.call(args.array(), args.position(), numArgs);
-        sendMsg();
+        sendMsg(functionId);
         receiveMsg();
         if (getResponseType() != MessageType.CALL_RSP) {
-            throw new IllegalStateException("Illegal reply received, expected CALL_RSP, recvd: " + getResponseType());
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected CALL_RSP, recieved " + getResponseType().name());
         }
         return recvBuffer;
     }
 
-    public void quit() throws IOException {
+    public ByteBuffer callMulti(long key, ByteBuffer args, int numTuples) throws IOException, AsterixException {
+        recvBuffer.clear();
+        recvBuffer.position(0);
+        recvBuffer.limit(0);
+        messageBuilder.buf.clear();
+        messageBuilder.buf.position(0);
+        messageBuilder.callMulti(args.array(), args.position(), numTuples);
+        sendMsg(key);
+        receiveMsg();
+        if (getResponseType() != MessageType.CALL_RSP) {
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "Expected CALL_RSP, recieved " + getResponseType().name());
+        }
+        return recvBuffer;
+    }
+
+    //For future use with interpreter reuse between jobs.
+    public void quit() throws HyracksDataException {
         messageBuilder.quit();
-        router.removeRoute(key);
+        router.removeRoute(routeId);
     }
 
     public void receiveMsg() throws IOException, AsterixException {
         Exception except = null;
         try {
-            synchronized (recvBuffer) {
-                while (recvBuffer.limit() == 0) {
-                    recvBuffer.wait(100);
+            synchronized (bufferBox) {
+                while ((bufferBox.getFirst().limit() == 0 || bufferBox.getSecond() != null) && pythonProc.isAlive()) {
+                    bufferBox.wait(100);
                 }
             }
-            if (router.hasException(key)) {
-                except = router.getException(key);
+            except = router.getAndRemoveException(routeId);
+            if (!pythonProc.isAlive()) {
+                except = new IOException("Python process exited with code: " + pythonProc.exitValue());
             }
         } catch (InterruptedException e) {
             Thread.currentThread().interrupt();
             throw new AsterixException(ErrorCode.EXTERNAL_UDF_EXCEPTION, e);
         }
         if (except != null) {
-            throw new AsterixException(ErrorCode.EXTERNAL_UDF_EXCEPTION, except);
+            throw new AsterixException(except);
+        }
+        if (bufferBox.getFirst() != recvBuffer) {
+            recvBuffer = bufferBox.getFirst();
         }
         messageBuilder.readHead(recvBuffer);
         if (messageBuilder.type == MessageType.ERROR) {
-            throw new AsterixException(ErrorCode.EXTERNAL_UDF_EXCEPTION,
-                    MessagePack.newDefaultUnpacker(recvBuffer).unpackString());
+            unpackerInput.reset(recvBuffer.array(), recvBuffer.position() + recvBuffer.arrayOffset(),
+                    recvBuffer.remaining());
+            unpacker.reset(unpackerInput);
+            throw new AsterixException(unpacker.unpackString());
         }
     }
 
-    public void sendMsg() throws IOException {
+    public void sendMsg(long key) throws IOException {
         headerBuffer.clear();
         headerBuffer.position(0);
-        headerBuffer.putInt(HEADER_SIZE + messageBuilder.buf.position());
-        headerBuffer.putLong(-1);
+        headerBuffer.putInt(HEADER_SIZE + Integer.BYTES + messageBuilder.buf.position());
         headerBuffer.putLong(key);
+        headerBuffer.putLong(routeId);
         headerBuffer.put(Message.NORMAL);
         sockOut.write(headerBuffer.array(), 0, HEADER_SIZE + Integer.BYTES);
         sockOut.write(messageBuilder.buf.array(), 0, messageBuilder.buf.position());
@@ -141,4 +178,8 @@
         return messageBuilder.type;
     }
 
+    public long getRouteId() {
+        return routeId;
+    }
+
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
index 506e80d..5052eb4 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/ipc/PythonMessageBuilder.java
@@ -25,19 +25,16 @@
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 
 import org.apache.asterix.external.library.msgpack.MessagePackerFromADM;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class PythonMessageBuilder {
-    private static final int MAX_BUF_SIZE = 21 * 1024 * 1024; //21MB.
-    private static final Logger LOGGER = LogManager.getLogger();
+    private static final int MAX_BUF_SIZE = 64 * 1024 * 1024; //64MB.
     MessageType type;
     long dataLength;
     ByteBuffer buf;
-    String[] initAry = new String[3];
 
     public PythonMessageBuilder() {
         this.type = null;
@@ -49,12 +46,12 @@
         this.type = type;
     }
 
-    public void packHeader() {
+    public void packHeader() throws HyracksDataException {
         MessagePackerFromADM.packFixPos(buf, (byte) type.ordinal());
     }
 
     //TODO: this is wrong for any multibyte chars
-    private int getStringLength(String s) {
+    private static int getStringLength(String s) {
         return s.length();
     }
 
@@ -66,7 +63,7 @@
     public void hello() throws IOException {
         this.type = MessageType.HELO;
         byte[] serAddr = serialize(new InetSocketAddress(InetAddress.getLoopbackAddress(), 1));
-        dataLength = serAddr.length + 5;
+        dataLength = serAddr.length + 1;
         packHeader();
         //TODO:make this cleaner
         buf.put(BIN32);
@@ -74,32 +71,38 @@
         buf.put(serAddr);
     }
 
-    public void quit() {
+    public void quit() throws HyracksDataException {
         this.type = MessageType.QUIT;
         dataLength = getStringLength("QUIT");
         packHeader();
         MessagePackerFromADM.packFixStr(buf, "QUIT");
     }
 
-    public void init(String module, String clazz, String fn) {
+    public void init(final String module, final String clazz, final String fn) throws HyracksDataException {
         this.type = MessageType.INIT;
-        initAry[0] = module;
-        initAry[1] = clazz;
-        initAry[2] = fn;
-        dataLength = Arrays.stream(initAry).mapToInt(s -> getStringLength(s)).sum() + 2;
-        packHeader();
-        MessagePackerFromADM.packFixArrayHeader(buf, (byte) initAry.length);
-        for (String s : initAry) {
-            MessagePackerFromADM.packStr(buf, s);
+        // sum(string lengths) + 2 from fix array tag and message type
+        if (clazz != null) {
+            dataLength =
+                    PythonMessageBuilder.getStringLength(module) + getStringLength(clazz) + getStringLength(fn) + 2;
+        } else {
+            dataLength = PythonMessageBuilder.getStringLength(module) + getStringLength(fn) + 2;
         }
+        packHeader();
+        int numArgs = clazz == null ? 2 : 3;
+        MessagePackerFromADM.packFixArrayHeader(buf, (byte) numArgs);
+        MessagePackerFromADM.packStr(buf, module);
+        if (clazz != null) {
+            MessagePackerFromADM.packStr(buf, clazz);
+        }
+        MessagePackerFromADM.packStr(buf, fn);
     }
 
-    public void call(byte[] args, int lim, int numArgs) {
+    public void call(byte[] args, int lim, int numArgs) throws HyracksDataException {
         if (args.length > buf.capacity()) {
             int growTo = ExternalFunctionResultRouter.closestPow2(args.length);
             if (growTo > MAX_BUF_SIZE) {
-                //TODO: something more graceful
-                throw new IllegalArgumentException("Reached maximum buffer size");
+                throw HyracksDataException.create(ErrorCode.ILLEGAL_STATE,
+                        "Unable to allocate message buffer larger than:" + MAX_BUF_SIZE + " bytes");
             }
             buf = ByteBuffer.allocate(growTo);
         }
@@ -109,11 +112,32 @@
         dataLength = 5 + 1 + lim;
         packHeader();
         //TODO: make this switch between fixarray/array16/array32
-        if (numArgs == 0) {
-            buf.put(NIL);
-        } else {
-            buf.put(ARRAY32);
-            buf.putInt(numArgs);
+        buf.put((byte) (FIXARRAY_PREFIX + 1));
+        buf.put(ARRAY32);
+        buf.putInt(numArgs);
+        if (numArgs > 0) {
+            buf.put(args, 0, lim);
+        }
+    }
+
+    public void callMulti(byte[] args, int lim, int numArgs) throws HyracksDataException {
+        if (args.length > buf.capacity()) {
+            int growTo = ExternalFunctionResultRouter.closestPow2(args.length);
+            if (growTo > MAX_BUF_SIZE) {
+                throw HyracksDataException.create(ErrorCode.ILLEGAL_STATE,
+                        "Unable to allocate message buffer larger than:" + MAX_BUF_SIZE + " bytes");
+            }
+            buf = ByteBuffer.allocate(growTo);
+        }
+        buf.clear();
+        buf.position(0);
+        this.type = MessageType.CALL;
+        dataLength = 5 + 1 + lim;
+        packHeader();
+        //TODO: make this switch between fixarray/array16/array32
+        buf.put(ARRAY16);
+        buf.putShort((short) numArgs);
+        if (numArgs > 0) {
             buf.put(args, 0, lim);
         }
     }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
index c5b9b53..dce5503 100755
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalLibraryManager.java
@@ -21,6 +21,7 @@
 import static com.fasterxml.jackson.databind.MapperFeature.SORT_PROPERTIES_ALPHABETICALLY;
 import static com.fasterxml.jackson.databind.SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS;
 
+import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
@@ -41,9 +42,11 @@
 import java.security.DigestOutputStream;
 import java.security.KeyStore;
 import java.security.MessageDigest;
+import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.zip.ZipEntry;
@@ -51,17 +54,21 @@
 
 import javax.net.ssl.SSLContext;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.functions.ExternalFunctionLanguage;
 import org.apache.asterix.common.library.ILibrary;
 import org.apache.asterix.common.library.ILibraryManager;
 import org.apache.asterix.common.library.LibraryDescriptor;
 import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.asterix.common.utils.StoragePathUtil;
 import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.commons.compress.archivers.zip.ZipArchiveEntry;
 import org.apache.commons.compress.archivers.zip.ZipArchiveOutputStream;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.http.HttpEntity;
 import org.apache.http.HttpHeaders;
 import org.apache.http.HttpStatus;
@@ -221,7 +228,7 @@
     }
 
     private FileReference getDataverseDir(DataverseName dataverseName) throws HyracksDataException {
-        return getChildFileRef(storageDir, dataverseName.getCanonicalForm());
+        return getChildFileRef(storageDir, StoragePathUtil.prepareDataverseName(dataverseName));
     }
 
     @Override
@@ -236,6 +243,68 @@
     }
 
     @Override
+    public List<Pair<DataverseName, String>> getLibraryListing() throws IOException {
+        List<Pair<DataverseName, String>> libs = new ArrayList<>();
+        Files.walkFileTree(storageDirPath, new SimpleFileVisitor<Path>() {
+            @Override
+            public FileVisitResult visitFile(Path currPath, BasicFileAttributes attrs) {
+                //never want to see any files
+                return FileVisitResult.TERMINATE;
+            }
+
+            @Override
+            public FileVisitResult preVisitDirectory(Path currPath, BasicFileAttributes attrs)
+                    throws HyracksDataException {
+                if (currPath.equals(storageDirPath) || currPath.getParent().equals(storageDirPath)) {
+                    return FileVisitResult.CONTINUE;
+                }
+                if (currPath.getFileName().toString().codePointAt(0) == StoragePathUtil.DATAVERSE_CONTINUATION_MARKER) {
+                    return FileVisitResult.CONTINUE;
+                }
+                final String candidateDvAndLib = storageDirPath.toAbsolutePath().normalize()
+                        .relativize(currPath.toAbsolutePath().normalize()).toString();
+                List<String> dvParts = new ArrayList<>();
+                final String[] tokens = StringUtils.split(candidateDvAndLib, File.separatorChar);
+                if (tokens == null || tokens.length < 2) {
+                    //? shouldn't happen
+                    return FileVisitResult.TERMINATE;
+                }
+                //add first part, then all multiparts
+                dvParts.add(tokens[0]);
+                int currToken = 1;
+                for (; currToken < tokens.length && tokens[currToken]
+                        .codePointAt(0) == StoragePathUtil.DATAVERSE_CONTINUATION_MARKER; currToken++) {
+                    dvParts.add(tokens[currToken].substring(1));
+                }
+                //we should only arrive at foo/^bar/^baz/.../^bat/lib
+                //anything else is fishy or empty
+                if (currToken != tokens.length - 1) {
+                    return FileVisitResult.SKIP_SUBTREE;
+                }
+                String candidateLib = tokens[currToken];
+                DataverseName candidateDv = DataverseName.create(dvParts);
+                FileReference candidateLibPath = findLibraryRevDir(candidateDv, candidateLib);
+                if (candidateLibPath != null) {
+                    libs.add(new Pair<>(candidateDv, candidateLib));
+                }
+                return FileVisitResult.SKIP_SUBTREE;
+            }
+        });
+        return libs;
+    }
+
+    @Override
+    public String getLibraryHash(DataverseName dataverseName, String libraryName) throws IOException {
+        FileReference revDir = findLibraryRevDir(dataverseName, libraryName);
+        if (revDir == null) {
+            throw HyracksDataException
+                    .create(AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION, "Library does not exist"));
+        }
+        LibraryDescriptor desc = getLibraryDescriptor(revDir);
+        return desc.getHash();
+    }
+
+    @Override
     public ILibrary getLibrary(DataverseName dataverseName, String libraryName) throws HyracksDataException {
         Pair<DataverseName, String> key = getKey(dataverseName, libraryName);
         synchronized (this) {
@@ -545,9 +614,8 @@
         outputFile.getFile().createNewFile();
         IFileHandle fHandle = ioManager.open(outputFile, IIOManager.FileReadWriteMode.READ_WRITE,
                 IIOManager.FileSyncMode.METADATA_ASYNC_DATA_ASYNC);
-        try {
-            WritableByteChannel outChannel = ioManager.newWritableChannel(fHandle);
-            OutputStream outputStream = Channels.newOutputStream(outChannel);
+        WritableByteChannel outChannel = ioManager.newWritableChannel(fHandle);
+        try (OutputStream outputStream = Channels.newOutputStream(outChannel)) {
             IOUtils.copyLarge(dataStream, outputStream, copyBuffer);
             outputStream.flush();
             ioManager.sync(fHandle, true);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
index f9d30b1..e664f47 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/ExternalScalarPythonFunctionEvaluator.java
@@ -19,50 +19,31 @@
 
 package org.apache.asterix.external.library;
 
+import static org.msgpack.core.MessagePack.Code.FIXARRAY_PREFIX;
+
 import java.io.DataOutput;
-import java.io.File;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Objects;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.ErrorCode;
-import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.common.metadata.DataverseName;
-import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
-import org.apache.asterix.external.ipc.PythonIPCProto;
-import org.apache.asterix.external.library.msgpack.MessagePackerFromADM;
 import org.apache.asterix.external.library.msgpack.MessageUnpackerToADM;
 import org.apache.asterix.om.functions.IExternalFunctionInfo;
 import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
 import org.apache.asterix.om.types.IAType;
-import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.asterix.runtime.evaluators.functions.PointableHelper;
 import org.apache.hyracks.algebricks.runtime.base.IEvaluatorContext;
 import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
-import org.apache.hyracks.api.config.IApplicationConfig;
-import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.dataflow.TaskAttemptId;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.IWarningCollector;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 import org.apache.hyracks.api.exceptions.Warning;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.resources.IDeallocatable;
-import org.apache.hyracks.control.common.controllers.NCConfig;
 import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.api.IValueReference;
-import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
 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;
-import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
-import org.apache.hyracks.ipc.impl.IPCSystem;
+import org.msgpack.core.MessagePack;
+import org.msgpack.core.MessageUnpacker;
+import org.msgpack.core.buffer.ArrayBufferInput;
 
 class ExternalScalarPythonFunctionEvaluator extends ExternalScalarFunctionEvaluator {
 
@@ -72,46 +53,22 @@
     private final ByteBuffer argHolder;
     private final ByteBuffer outputWrapper;
     private final IEvaluatorContext evaluatorContext;
-    private static final String ENTRYPOINT = "entrypoint.py";
-    private static final String SITE_PACKAGES = "site-packages";
 
     private final IPointable[] argValues;
+    private final SourceLocation sourceLocation;
+
+    private MessageUnpacker unpacker;
+    private ArrayBufferInput unpackerInput;
+
+    private long fnId;
 
     ExternalScalarPythonFunctionEvaluator(IExternalFunctionInfo finfo, IScalarEvaluatorFactory[] args,
             IAType[] argTypes, IEvaluatorContext ctx, SourceLocation sourceLoc) throws HyracksDataException {
         super(finfo, args, argTypes, ctx);
-        IApplicationConfig cfg = ctx.getServiceContext().getAppConfig();
-        String pythonPathCmd = cfg.getString(NCConfig.Option.PYTHON_CMD);
-        List<String> pythonArgs = new ArrayList<>();
-        if (pythonPathCmd == null) {
-            //if absolute path to interpreter is not specified, use environmental python
-            pythonPathCmd = "/usr/bin/env";
-            pythonArgs.add("python3");
-        }
-        File pythonPath = new File(pythonPathCmd);
-        List<String> sitePkgs = new ArrayList<>();
-        sitePkgs.add(SITE_PACKAGES);
-        String[] addlSitePackages =
-                ctx.getServiceContext().getAppConfig().getStringArray((NCConfig.Option.PYTHON_ADDITIONAL_PACKAGES));
-        sitePkgs.addAll(Arrays.asList(addlSitePackages));
-        if (cfg.getBoolean(NCConfig.Option.PYTHON_USE_BUNDLED_MSGPACK)) {
-            sitePkgs.add("ipc" + File.separator + SITE_PACKAGES + File.separator);
-        }
-        String[] pythonArgsRaw = ctx.getServiceContext().getAppConfig().getStringArray(NCConfig.Option.PYTHON_ARGS);
-        if (pythonArgsRaw != null) {
-            pythonArgs.addAll(Arrays.asList(pythonArgsRaw));
-        }
-        StringBuilder sitePackagesPathBuilder = new StringBuilder();
-        for (int i = 0; i < sitePkgs.size() - 1; i++) {
-            sitePackagesPathBuilder.append(sitePkgs.get(i));
-            sitePackagesPathBuilder.append(File.pathSeparator);
-        }
-        sitePackagesPathBuilder.append(sitePkgs.get(sitePkgs.size() - 1));
-
         try {
-            libraryEvaluator = PythonLibraryEvaluator.getInstance(finfo, libraryManager, router, ipcSys, pythonPath,
-                    ctx.getTaskContext(), sitePackagesPathBuilder.toString(), pythonArgs, ctx.getWarningCollector(),
-                    sourceLoc);
+            PythonLibraryEvaluatorFactory evaluatorFactory = new PythonLibraryEvaluatorFactory(ctx.getTaskContext());
+            this.libraryEvaluator = evaluatorFactory.getEvaluator(finfo, sourceLoc);
+            this.fnId = libraryEvaluator.initialize(finfo);
         } catch (IOException | AsterixException e) {
             throw new HyracksDataException("Failed to initialize Python", e);
         }
@@ -123,6 +80,9 @@
         this.argHolder = ByteBuffer.wrap(new byte[Short.MAX_VALUE * 2]);
         this.outputWrapper = ByteBuffer.wrap(new byte[Short.MAX_VALUE * 2]);
         this.evaluatorContext = ctx;
+        this.sourceLocation = sourceLoc;
+        this.unpackerInput = new ArrayBufferInput(new byte[0]);
+        this.unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
     }
 
     @Override
@@ -130,193 +90,57 @@
         argHolder.clear();
         for (int i = 0, ln = argEvals.length; i < ln; i++) {
             argEvals[i].evaluate(tuple, argValues[i]);
+            if (!finfo.getNullCall() && PointableHelper.checkAndSetMissingOrNull(result, argValues[i])) {
+                return;
+            }
             try {
-                setArgument(i, argValues[i]);
+                PythonLibraryEvaluator.setArgument(argTypes[i], argValues[i], argHolder, finfo.getNullCall());
             } catch (IOException e) {
                 throw new HyracksDataException("Error evaluating Python UDF", e);
             }
         }
         try {
-            ByteBuffer res = libraryEvaluator.callPython(argHolder, argTypes.length);
+            ByteBuffer res = libraryEvaluator.callPython(fnId, argHolder, argTypes.length);
             resultBuffer.reset();
             wrap(res, resultBuffer.getDataOutput());
         } catch (Exception e) {
             throw new HyracksDataException("Error evaluating Python UDF", e);
         }
-        result.set(resultBuffer.getByteArray(), resultBuffer.getStartOffset(), resultBuffer.getLength());
-    }
-
-    private static class PythonLibraryEvaluator extends AbstractStateObject implements IDeallocatable {
-        Process p;
-        IExternalFunctionInfo finfo;
-        ILibraryManager libMgr;
-        File pythonHome;
-        PythonIPCProto proto;
-        ExternalFunctionResultRouter router;
-        IPCSystem ipcSys;
-        String module;
-        String clazz;
-        String fn;
-        String sitePkgs;
-        List<String> pythonArgs;
-        TaskAttemptId task;
-        IWarningCollector warningCollector;
-        SourceLocation sourceLoc;
-
-        private PythonLibraryEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, IExternalFunctionInfo finfo,
-                ILibraryManager libMgr, File pythonHome, String sitePkgs, List<String> pythonArgs,
-                ExternalFunctionResultRouter router, IPCSystem ipcSys, TaskAttemptId task,
-                IWarningCollector warningCollector, SourceLocation sourceLoc) {
-            super(jobId, evaluatorId);
-            this.finfo = finfo;
-            this.libMgr = libMgr;
-            this.pythonHome = pythonHome;
-            this.sitePkgs = sitePkgs;
-            this.pythonArgs = pythonArgs;
-            this.router = router;
-            this.task = task;
-            this.ipcSys = ipcSys;
-            this.warningCollector = warningCollector;
-            this.sourceLoc = sourceLoc;
-
-        }
-
-        public void initialize() throws IOException, AsterixException {
-            PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
-            List<String> externalIdents = finfo.getExternalIdentifier();
-            PythonLibrary library = (PythonLibrary) libMgr.getLibrary(fnId.libraryDataverseName, fnId.libraryName);
-            String wd = library.getFile().getAbsolutePath();
-            String packageModule = externalIdents.get(0);
-            String clazz;
-            String fn;
-            String externalIdent1 = externalIdents.get(1);
-            int idx = externalIdent1.lastIndexOf('.');
-            if (idx >= 0) {
-                clazz = externalIdent1.substring(0, idx);
-                fn = externalIdent1.substring(idx + 1);
-            } else {
-                clazz = "None";
-                fn = externalIdent1;
-            }
-            this.fn = fn;
-            this.clazz = clazz;
-            this.module = packageModule;
-            int port = ipcSys.getSocketAddress().getPort();
-            List<String> args = new ArrayList<>();
-            args.add(pythonHome.getAbsolutePath());
-            args.addAll(pythonArgs);
-            args.add(ENTRYPOINT);
-            args.add(InetAddress.getLoopbackAddress().getHostAddress());
-            args.add(Integer.toString(port));
-            args.add(sitePkgs);
-            ProcessBuilder pb = new ProcessBuilder(args.toArray(new String[0]));
-            pb.directory(new File(wd));
-            p = pb.start();
-            proto = new PythonIPCProto(p.getOutputStream(), router, ipcSys);
-            proto.start();
-            proto.helo();
-            proto.init(packageModule, clazz, fn);
-        }
-
-        ByteBuffer callPython(ByteBuffer arguments, int numArgs) throws Exception {
-            ByteBuffer ret = null;
-            try {
-                ret = proto.call(arguments, numArgs);
-            } catch (AsterixException e) {
-                warningCollector.warn(Warning.of(sourceLoc, ErrorCode.EXTERNAL_UDF_EXCEPTION, e.getMessage()));
-            }
-            return ret;
-        }
-
-        @Override
-        public void deallocate() {
-            if (p != null) {
-                boolean dead = false;
-                try {
-                    p.destroy();
-                    dead = p.waitFor(100, TimeUnit.MILLISECONDS);
-                } catch (InterruptedException e) {
-                    //gonna kill it anyway
-                }
-                if (!dead) {
-                    p.destroyForcibly();
-                }
-            }
-        }
-
-        private static PythonLibraryEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
-                ExternalFunctionResultRouter router, IPCSystem ipcSys, File pythonHome, IHyracksTaskContext ctx,
-                String sitePkgs, List<String> pythonArgs, IWarningCollector warningCollector, SourceLocation sourceLoc)
-                throws IOException, AsterixException {
-            PythonLibraryEvaluatorId evaluatorId =
-                    new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(), finfo.getLibraryName());
-            PythonLibraryEvaluator evaluator = (PythonLibraryEvaluator) ctx.getStateObject(evaluatorId);
-            if (evaluator == null) {
-                evaluator = new PythonLibraryEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, finfo, libMgr,
-                        pythonHome, sitePkgs, pythonArgs, router, ipcSys, ctx.getTaskAttemptId(), warningCollector,
-                        sourceLoc);
-                ctx.registerDeallocatable(evaluator);
-                evaluator.initialize();
-                ctx.setStateObject(evaluator);
-            }
-            return evaluator;
-        }
-    }
-
-    private static final class PythonLibraryEvaluatorId {
-
-        private final DataverseName libraryDataverseName;
-
-        private final String libraryName;
-
-        private PythonLibraryEvaluatorId(DataverseName libraryDataverseName, String libraryName) {
-            this.libraryDataverseName = Objects.requireNonNull(libraryDataverseName);
-            this.libraryName = Objects.requireNonNull(libraryName);
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o)
-                return true;
-            if (o == null || getClass() != o.getClass())
-                return false;
-            PythonLibraryEvaluatorId that = (PythonLibraryEvaluatorId) o;
-            return libraryDataverseName.equals(that.libraryDataverseName) && libraryName.equals(that.libraryName);
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(libraryDataverseName, libraryName);
-        }
-    }
-
-    private void setArgument(int index, IValueReference valueReference) throws IOException {
-        IAType type = argTypes[index];
-        ATypeTag tag = type.getTypeTag();
-        switch (tag) {
-            case ANY:
-                TaggedValuePointable pointy = TaggedValuePointable.FACTORY.createPointable();
-                pointy.set(valueReference);
-                ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
-                IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
-                MessagePackerFromADM.pack(valueReference, rtType, argHolder);
-                break;
-            default:
-                MessagePackerFromADM.pack(valueReference, type, argHolder);
-                break;
-        }
+        result.set(resultBuffer);
     }
 
     private void wrap(ByteBuffer resultWrapper, DataOutput out) throws HyracksDataException {
         //TODO: output wrapper needs to grow with result wrapper
         outputWrapper.clear();
         outputWrapper.position(0);
-        MessageUnpackerToADM.unpack(resultWrapper, outputWrapper, true);
         try {
+            if (resultWrapper == null) {
+                outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                out.write(outputWrapper.array(), 0, outputWrapper.position() + outputWrapper.arrayOffset());
+                return;
+            }
+            if ((resultWrapper.get() ^ FIXARRAY_PREFIX) != (byte) 2) {
+                throw HyracksDataException.create(AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                        "Returned result missing outer wrapper"));
+            }
+            int numresults = resultWrapper.get() ^ FIXARRAY_PREFIX;
+            if (numresults > 0) {
+                MessageUnpackerToADM.unpack(resultWrapper, outputWrapper, true);
+            }
+            unpackerInput.reset(resultWrapper.array(), resultWrapper.position() + resultWrapper.arrayOffset(),
+                    resultWrapper.remaining());
+            unpacker.reset(unpackerInput);
+            int numEntries = unpacker.unpackArrayHeader();
+            for (int j = 0; j < numEntries; j++) {
+                outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                if (evaluatorContext.getWarningCollector().shouldWarn()) {
+                    evaluatorContext.getWarningCollector().warn(
+                            Warning.of(sourceLocation, ErrorCode.EXTERNAL_UDF_EXCEPTION, unpacker.unpackString()));
+                }
+            }
             out.write(outputWrapper.array(), 0, outputWrapper.position() + outputWrapper.arrayOffset());
         } catch (IOException e) {
-            throw new HyracksDataException(e.getMessage());
+            throw HyracksDataException.create(e);
         }
-
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
new file mode 100644
index 0000000..e2229ee
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluator.java
@@ -0,0 +1,216 @@
+/*
+ * 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.external.library;
+
+import static org.apache.asterix.common.exceptions.ErrorCode.EXTERNAL_UDF_EXCEPTION;
+import static org.msgpack.core.MessagePack.Code.ARRAY16;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.asterix.external.ipc.PythonIPCProto;
+import org.apache.asterix.external.library.msgpack.MessagePackerFromADM;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.TypeTagUtil;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.TaskAttemptId;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.IWarningCollector;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.resources.IDeallocatable;
+import org.apache.hyracks.data.std.api.IValueReference;
+import org.apache.hyracks.data.std.primitive.TaggedValuePointable;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+import org.apache.hyracks.ipc.impl.IPCSystem;
+
+public class PythonLibraryEvaluator extends AbstractStateObject implements IDeallocatable {
+
+    public static final String ENTRYPOINT = "entrypoint.py";
+    public static final String SITE_PACKAGES = "site-packages";
+
+    private Process p;
+    private ILibraryManager libMgr;
+    private File pythonHome;
+    private PythonIPCProto proto;
+    private ExternalFunctionResultRouter router;
+    private IPCSystem ipcSys;
+    private String sitePkgs;
+    private List<String> pythonArgs;
+    private TaskAttemptId task;
+    private IWarningCollector warningCollector;
+    private SourceLocation sourceLoc;
+
+    public PythonLibraryEvaluator(JobId jobId, PythonLibraryEvaluatorId evaluatorId, ILibraryManager libMgr,
+            File pythonHome, String sitePkgs, List<String> pythonArgs, ExternalFunctionResultRouter router,
+            IPCSystem ipcSys, TaskAttemptId task, IWarningCollector warningCollector, SourceLocation sourceLoc) {
+        super(jobId, evaluatorId);
+        this.libMgr = libMgr;
+        this.pythonHome = pythonHome;
+        this.sitePkgs = sitePkgs;
+        this.pythonArgs = pythonArgs;
+        this.router = router;
+        this.task = task;
+        this.ipcSys = ipcSys;
+        this.warningCollector = warningCollector;
+        this.sourceLoc = sourceLoc;
+
+    }
+
+    private void initialize() throws IOException, AsterixException {
+        PythonLibraryEvaluatorId fnId = (PythonLibraryEvaluatorId) id;
+        PythonLibrary library =
+                (PythonLibrary) libMgr.getLibrary(fnId.getLibraryDataverseName(), fnId.getLibraryName());
+        String wd = library.getFile().getAbsolutePath();
+        int port = ipcSys.getSocketAddress().getPort();
+        List<String> args = new ArrayList<>();
+        args.add(pythonHome.getAbsolutePath());
+        args.addAll(pythonArgs);
+        args.add(ENTRYPOINT);
+        args.add(InetAddress.getLoopbackAddress().getHostAddress());
+        args.add(Integer.toString(port));
+        args.add(sitePkgs);
+
+        ProcessBuilder pb = new ProcessBuilder(args.toArray(new String[0]));
+        pb.directory(new File(wd));
+        p = pb.start();
+        proto = new PythonIPCProto(p.getOutputStream(), router, p);
+        proto.start();
+        proto.helo();
+    }
+
+    public long initialize(IExternalFunctionInfo finfo) throws IOException, AsterixException {
+        List<String> externalIdents = finfo.getExternalIdentifier();
+        String packageModule = externalIdents.get(0);
+        String clazz;
+        String fn;
+        String externalIdent1 = externalIdents.get(1);
+        int idx = externalIdent1.lastIndexOf('.');
+        if (idx >= 0) {
+            clazz = externalIdent1.substring(0, idx);
+            fn = externalIdent1.substring(idx + 1);
+        } else {
+            clazz = null;
+            fn = externalIdent1;
+        }
+        return proto.init(packageModule, clazz, fn);
+    }
+
+    public ByteBuffer callPython(long id, ByteBuffer arguments, int numArgs) throws IOException {
+        ByteBuffer ret = null;
+        try {
+            ret = proto.call(id, arguments, numArgs);
+        } catch (AsterixException e) {
+            if (warningCollector.shouldWarn()) {
+                warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
+            }
+        }
+        return ret;
+    }
+
+    public ByteBuffer callPythonMulti(long id, ByteBuffer arguments, int numTuples) throws IOException {
+        ByteBuffer ret = null;
+        try {
+            ret = proto.callMulti(id, arguments, numTuples);
+        } catch (AsterixException e) {
+            if (warningCollector.shouldWarn()) {
+                warningCollector.warn(Warning.of(sourceLoc, EXTERNAL_UDF_EXCEPTION, e.getMessage()));
+            }
+        }
+        return ret;
+    }
+
+    @Override
+    public void deallocate() {
+        if (p != null) {
+            boolean dead = false;
+            try {
+                p.destroy();
+                dead = p.waitFor(100, TimeUnit.MILLISECONDS);
+            } catch (InterruptedException e) {
+                //gonna kill it anyway
+            }
+            if (!dead) {
+                p.destroyForcibly();
+            }
+        }
+        router.removeRoute(proto.getRouteId());
+    }
+
+    public static ATypeTag setArgument(IAType type, IValueReference valueReference, ByteBuffer argHolder,
+            boolean nullCall) throws IOException {
+        ATypeTag tag = type.getTypeTag();
+        if (tag == ATypeTag.ANY) {
+            TaggedValuePointable pointy = TaggedValuePointable.FACTORY.createPointable();
+            pointy.set(valueReference);
+            ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
+            IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
+            return MessagePackerFromADM.pack(valueReference, rtType, argHolder, nullCall);
+        } else {
+            return MessagePackerFromADM.pack(valueReference, type, argHolder, nullCall);
+        }
+    }
+
+    public static ATypeTag peekArgument(IAType type, IValueReference valueReference) throws HyracksDataException {
+        ATypeTag tag = type.getTypeTag();
+        if (tag == ATypeTag.ANY) {
+            TaggedValuePointable pointy = TaggedValuePointable.FACTORY.createPointable();
+            pointy.set(valueReference);
+            ATypeTag rtTypeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(pointy.getTag());
+            IAType rtType = TypeTagUtil.getBuiltinTypeByTag(rtTypeTag);
+            return MessagePackerFromADM.peekUnknown(rtType);
+        } else {
+            return MessagePackerFromADM.peekUnknown(type);
+        }
+    }
+
+    public static void setVoidArgument(ByteBuffer argHolder) {
+        argHolder.put(ARRAY16);
+        argHolder.putShort((short) 0);
+    }
+
+    public static PythonLibraryEvaluator getInstance(IExternalFunctionInfo finfo, ILibraryManager libMgr,
+            ExternalFunctionResultRouter router, IPCSystem ipcSys, File pythonHome, IHyracksTaskContext ctx,
+            String sitePkgs, List<String> pythonArgs, IWarningCollector warningCollector, SourceLocation sourceLoc)
+            throws IOException, AsterixException {
+        PythonLibraryEvaluatorId evaluatorId = new PythonLibraryEvaluatorId(finfo.getLibraryDataverseName(),
+                finfo.getLibraryName(), Thread.currentThread());
+        PythonLibraryEvaluator evaluator = (PythonLibraryEvaluator) ctx.getStateObject(evaluatorId);
+        if (evaluator == null) {
+            evaluator = new PythonLibraryEvaluator(ctx.getJobletContext().getJobId(), evaluatorId, libMgr, pythonHome,
+                    sitePkgs, pythonArgs, router, ipcSys, ctx.getTaskAttemptId(), warningCollector, sourceLoc);
+            ctx.getJobletContext().registerDeallocatable(evaluator);
+            evaluator.initialize();
+            ctx.setStateObject(evaluator);
+        }
+        return evaluator;
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
new file mode 100644
index 0000000..86d51de
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorFactory.java
@@ -0,0 +1,96 @@
+/*
+ * 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.external.library;
+
+import static org.apache.asterix.external.library.PythonLibraryEvaluator.SITE_PACKAGES;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.asterix.common.api.INcApplicationContext;
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.external.ipc.ExternalFunctionResultRouter;
+import org.apache.asterix.om.functions.IExternalFunctionInfo;
+import org.apache.hyracks.api.config.IApplicationConfig;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.control.common.controllers.NCConfig;
+import org.apache.hyracks.ipc.impl.IPCSystem;
+
+public class PythonLibraryEvaluatorFactory {
+    private final ILibraryManager libraryManager;
+    private final IPCSystem ipcSys;
+    private final File pythonPath;
+    private final IHyracksTaskContext ctx;
+    private final ExternalFunctionResultRouter router;
+    private final String sitePackagesPath;
+    private final List<String> pythonArgs;
+
+    public PythonLibraryEvaluatorFactory(IHyracksTaskContext ctx) throws AsterixException {
+        this.ctx = ctx;
+        libraryManager = ((INcApplicationContext) ctx.getJobletContext().getServiceContext().getApplicationContext())
+                .getLibraryManager();
+        router = libraryManager.getRouter();
+        ipcSys = libraryManager.getIPCI();
+        IApplicationConfig appCfg = ctx.getJobletContext().getServiceContext().getAppConfig();
+        String pythonPathCmd = appCfg.getString(NCConfig.Option.PYTHON_CMD);
+        boolean findPython = appCfg.getBoolean(NCConfig.Option.PYTHON_CMD_AUTOLOCATE);
+        pythonArgs = new ArrayList<>();
+        if (pythonPathCmd == null) {
+            if (findPython) {
+                //if absolute path to interpreter is not specified, try to use environmental python
+                pythonPathCmd = "/usr/bin/env";
+                pythonArgs.add("python3");
+            } else {
+                throw AsterixException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION, "Python interpreter not specified, and "
+                        + NCConfig.Option.PYTHON_CMD_AUTOLOCATE.ini() + " is false");
+            }
+        }
+        pythonPath = new File(pythonPathCmd);
+        List<String> sitePkgs = new ArrayList<>();
+        sitePkgs.add(SITE_PACKAGES);
+        String[] addlSitePackages = appCfg.getStringArray((NCConfig.Option.PYTHON_ADDITIONAL_PACKAGES));
+        sitePkgs.addAll(Arrays.asList(addlSitePackages));
+        if (appCfg.getBoolean(NCConfig.Option.PYTHON_USE_BUNDLED_MSGPACK)) {
+            sitePkgs.add("ipc" + File.separator + SITE_PACKAGES + File.separator);
+        }
+        String[] pythonArgsRaw = appCfg.getStringArray(NCConfig.Option.PYTHON_ARGS);
+        if (pythonArgsRaw != null) {
+            pythonArgs.addAll(Arrays.asList(pythonArgsRaw));
+        }
+        StringBuilder sitePackagesPathBuilder = new StringBuilder();
+        for (int i = 0; i < sitePkgs.size() - 1; i++) {
+            sitePackagesPathBuilder.append(sitePkgs.get(i));
+            sitePackagesPathBuilder.append(File.pathSeparator);
+        }
+        sitePackagesPathBuilder.append(sitePkgs.get(sitePkgs.size() - 1));
+        sitePackagesPath = sitePackagesPathBuilder.toString();
+    }
+
+    public PythonLibraryEvaluator getEvaluator(IExternalFunctionInfo fnInfo, SourceLocation sourceLoc)
+            throws IOException, AsterixException {
+        return PythonLibraryEvaluator.getInstance(fnInfo, libraryManager, router, ipcSys, pythonPath, ctx,
+                sitePackagesPath, pythonArgs, ctx.getWarningCollector(), sourceLoc);
+    }
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorId.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorId.java
new file mode 100644
index 0000000..c2f6f00
--- /dev/null
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/PythonLibraryEvaluatorId.java
@@ -0,0 +1,63 @@
+/*
+ * 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.external.library;
+
+import java.util.Objects;
+
+import org.apache.asterix.common.metadata.DataverseName;
+
+final class PythonLibraryEvaluatorId {
+
+    private final DataverseName libraryDataverseName;
+
+    private final String libraryName;
+
+    private final Thread thread;
+
+    PythonLibraryEvaluatorId(DataverseName libraryDataverseName, String libraryName, Thread thread) {
+        this.libraryDataverseName = Objects.requireNonNull(libraryDataverseName);
+        this.libraryName = Objects.requireNonNull(libraryName);
+        this.thread = Objects.requireNonNull(thread);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        PythonLibraryEvaluatorId that = (PythonLibraryEvaluatorId) o;
+        return libraryDataverseName.equals(that.libraryDataverseName) && libraryName.equals(that.libraryName)
+                && thread.equals(that.thread);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(libraryDataverseName, libraryName);
+    }
+
+    public DataverseName getLibraryDataverseName() {
+        return libraryDataverseName;
+    }
+
+    public String getLibraryName() {
+        return libraryName;
+    }
+
+}
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackerFromADM.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackerFromADM.java
index 383b2f1..f0ac56e 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackerFromADM.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessagePackerFromADM.java
@@ -27,16 +27,15 @@
 import static org.msgpack.core.MessagePack.Code.INT64;
 import static org.msgpack.core.MessagePack.Code.INT8;
 import static org.msgpack.core.MessagePack.Code.MAP32;
+import static org.msgpack.core.MessagePack.Code.NIL;
 import static org.msgpack.core.MessagePack.Code.STR32;
 import static org.msgpack.core.MessagePack.Code.TRUE;
-import static org.msgpack.core.MessagePack.Code.UINT16;
-import static org.msgpack.core.MessagePack.Code.UINT32;
-import static org.msgpack.core.MessagePack.Code.UINT64;
-import static org.msgpack.core.MessagePack.Code.UINT8;
 
 import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.ATypeTag;
 import org.apache.asterix.om.types.AUnionType;
@@ -64,11 +63,12 @@
     private static final int ITEM_COUNT_SIZE = 4;
     private static final int ITEM_OFFSET_SIZE = 4;
 
-    public static void pack(IValueReference ptr, IAType type, ByteBuffer out) throws HyracksDataException {
-        pack(ptr.getByteArray(), ptr.getStartOffset(), type, true, out);
+    public static ATypeTag pack(IValueReference ptr, IAType type, ByteBuffer out, boolean packUnknown)
+            throws HyracksDataException {
+        return pack(ptr.getByteArray(), ptr.getStartOffset(), type, true, packUnknown, out);
     }
 
-    public static void pack(byte[] ptr, int offs, IAType type, boolean tagged, ByteBuffer out)
+    public static ATypeTag pack(byte[] ptr, int offs, IAType type, boolean tagged, boolean packUnknown, ByteBuffer out)
             throws HyracksDataException {
         int relOffs = tagged ? offs + 1 : offs;
         ATypeTag tag = type.getTypeTag();
@@ -108,32 +108,33 @@
             case OBJECT:
                 packObject(ptr, offs, type, out);
                 break;
+            case MISSING:
+            case NULL:
+                if (packUnknown) {
+                    packNull(out);
+                    break;
+                } else {
+                    return tag;
+                }
             default:
-                throw new IllegalArgumentException("NYI");
+                throw HyracksDataException.create(AsterixException.create(ErrorCode.PARSER_ADM_DATA_PARSER_CAST_ERROR,
+                        tag.name(), "to a msgpack"));
+        }
+        return ATypeTag.TYPE;
+    }
+
+    public static ATypeTag peekUnknown(IAType type) {
+        switch (type.getTypeTag()) {
+            case MISSING:
+            case NULL:
+                return type.getTypeTag();
+            default:
+                return ATypeTag.TYPE;
         }
     }
 
-    public static byte minPackPosLong(ByteBuffer out, long in) {
-        if (in < 127) {
-            packFixPos(out, (byte) in);
-            return 1;
-        } else if (in < Byte.MAX_VALUE) {
-            out.put(UINT8);
-            out.put((byte) in);
-            return 2;
-        } else if (in < Short.MAX_VALUE) {
-            out.put(UINT16);
-            out.putShort((short) in);
-            return 3;
-        } else if (in < Integer.MAX_VALUE) {
-            out.put(UINT32);
-            out.putInt((int) in);
-            return 5;
-        } else {
-            out.put(UINT64);
-            out.putLong(in);
-            return 9;
-        }
+    public static void packNull(ByteBuffer out) {
+        out.put(NIL);
     }
 
     public static void packByte(ByteBuffer out, byte in) {
@@ -167,18 +168,20 @@
         out.putDouble(in);
     }
 
-    public static void packFixPos(ByteBuffer out, byte in) {
+    public static void packFixPos(ByteBuffer out, byte in) throws HyracksDataException {
         byte mask = (byte) (1 << 7);
         if ((in & mask) != 0) {
-            throw new IllegalArgumentException("fixint7 must be positive");
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "fixint7 must be positive");
         }
         out.put(in);
     }
 
-    public static void packFixStr(ByteBuffer buf, String in) {
-        byte[] strBytes = in.getBytes(Charset.forName("UTF-8"));
+    public static void packFixStr(ByteBuffer buf, String in) throws HyracksDataException {
+        byte[] strBytes = in.getBytes(StandardCharsets.UTF_8);
         if (strBytes.length > 31) {
-            throw new IllegalArgumentException("fixstr cannot be longer than 31");
+            throw HyracksDataException.create(org.apache.hyracks.api.exceptions.ErrorCode.ILLEGAL_STATE,
+                    "fixint7 must be positive");
         }
         buf.put((byte) (FIXSTR_PREFIX + strBytes.length));
         buf.put(strBytes);
@@ -186,7 +189,7 @@
 
     public static void packStr(ByteBuffer out, String in) {
         out.put(STR32);
-        byte[] strBytes = in.getBytes(Charset.forName("UTF-8"));
+        byte[] strBytes = in.getBytes(StandardCharsets.UTF_8);
         out.putInt(strBytes.length);
         out.put(strBytes);
     }
@@ -195,14 +198,14 @@
         out.put(STR32);
         //TODO: tagged/untagged. closed support is borked so always tagged rn
         String str = UTF8StringUtil.toString(in, offs);
-        byte[] strBytes = str.getBytes(Charset.forName("UTF-8"));
+        byte[] strBytes = str.getBytes(StandardCharsets.UTF_8);
         out.putInt(strBytes.length);
         out.put(strBytes);
     }
 
     public static void packStr(String str, ByteBuffer out) {
         out.put(STR32);
-        byte[] strBytes = str.getBytes(Charset.forName("UTF-8"));
+        byte[] strBytes = str.getBytes(StandardCharsets.UTF_8);
         out.putInt(strBytes.length);
         out.put(strBytes);
     }
@@ -221,12 +224,12 @@
             if (fixType) {
                 int itemOffs = itemCtOffs + ITEM_COUNT_SIZE + (i
                         * NonTaggedFormatUtil.getFieldValueLength(in, 0, collType.getItemType().getTypeTag(), false));
-                pack(in, itemOffs, collType.getItemType(), false, out);
+                pack(in, itemOffs, collType.getItemType(), false, true, out);
             } else {
                 int itemOffs =
                         offs + IntegerPointable.getInteger(in, itemCtOffs + ITEM_COUNT_SIZE + (i * ITEM_OFFSET_SIZE));
                 ATypeTag tag = ATypeTag.VALUE_TYPE_MAPPING[BytePointable.getByte(in, itemOffs)];
-                pack(in, itemOffs, TypeTagUtil.getBuiltinTypeByTag(tag), true, out);
+                pack(in, itemOffs, TypeTagUtil.getBuiltinTypeByTag(tag), true, true, out);
             }
         }
     }
@@ -240,14 +243,14 @@
             String field = recType.getFieldNames()[i];
             IAType fieldType = RecordUtils.getClosedFieldType(recType, i);
             packStr(field, out);
-            pack(in, RecordUtils.getClosedFieldOffset(in, offs, recType, i), fieldType, false, out);
+            pack(in, RecordUtils.getClosedFieldOffset(in, offs, recType, i), fieldType, false, true, out);
         }
         if (RecordUtils.isExpanded(in, offs, recType)) {
             for (int i = 0; i < RecordUtils.getOpenFieldCount(in, offs, recType); i++) {
                 packStr(in, RecordUtils.getOpenFieldNameOffset(in, offs, recType, i), out);
                 ATypeTag tag = ATypeTag.VALUE_TYPE_MAPPING[RecordUtils.getOpenFieldTag(in, offs, recType, i)];
                 pack(in, RecordUtils.getOpenFieldValueOffset(in, offs, recType, i),
-                        TypeTagUtil.getBuiltinTypeByTag(tag), true, out);
+                        TypeTagUtil.getBuiltinTypeByTag(tag), true, true, out);
             }
         }
 
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java
index fedd1f6..4af1121 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/msgpack/MessageUnpackerToADM.java
@@ -20,13 +20,16 @@
 
 import java.nio.ByteBuffer;
 
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.encoding.VarLenIntEncoderDecoder;
 import org.apache.hyracks.util.string.UTF8StringUtil;
 
 public class MessageUnpackerToADM {
 
-    public static void unpack(ByteBuffer in, ByteBuffer out, boolean tagged) {
+    public static void unpack(ByteBuffer in, ByteBuffer out, boolean tagged) throws HyracksDataException {
         byte tag = NIL;
         if (in != null) {
             tag = in.get();
@@ -68,6 +71,9 @@
                 case UINT32:
                     unpackUInt(in, out, tagged);
                     break;
+                case UINT64:
+                    unpackULong(in, out, tagged);
+                    break;
                 case INT8:
                     unpackByte(in, out, tagged);
                     break;
@@ -109,42 +115,12 @@
                     break;
 
                 default:
-                    throw new IllegalArgumentException("NYI");
+                    throw HyracksDataException.create(AsterixException.create(
+                            ErrorCode.PARSER_ADM_DATA_PARSER_CAST_ERROR, "msgpack tag " + tag + " ", "to an ADM type"));
             }
         }
     }
 
-    public static long unpackNextInt(ByteBuffer in) {
-        byte tag = in.get();
-        if (isFixInt(tag)) {
-            if (isPosFixInt(tag)) {
-                return tag;
-            } else if (isNegFixInt(tag)) {
-                return (tag ^ NEGFIXINT_PREFIX);
-            }
-        } else {
-            switch (tag) {
-                case INT8:
-                    return in.get();
-                case UINT8:
-                    return Byte.toUnsignedInt(in.get());
-                case INT16:
-                    return in.getShort();
-                case UINT16:
-                    return Short.toUnsignedInt(in.getShort());
-                case INT32:
-                    return in.getInt();
-                case UINT32:
-                    return Integer.toUnsignedLong(in.getInt());
-                case INT64:
-                    return in.getLong();
-                default:
-                    throw new IllegalArgumentException("NYI");
-            }
-        }
-        return -1;
-    }
-
     public static void unpackByte(ByteBuffer in, ByteBuffer out, boolean tagged) {
         if (tagged) {
             out.put(ATypeTag.SERIALIZED_INT8_TYPE_TAG);
@@ -194,6 +170,17 @@
         out.putLong(in.getInt() & 0x00000000FFFFFFFFl);
     }
 
+    public static void unpackULong(ByteBuffer in, ByteBuffer out, boolean tagged) {
+        if (tagged) {
+            out.put(ATypeTag.SERIALIZED_INT64_TYPE_TAG);
+        }
+        long val = in.getLong();
+        if (val < 0) {
+            throw new IllegalArgumentException("Integer overflow");
+        }
+        out.putLong(val);
+    }
+
     public static void unpackFloat(ByteBuffer in, ByteBuffer out, boolean tagged) {
         if (tagged) {
             out.put(ATypeTag.SERIALIZED_FLOAT_TYPE_TAG);
@@ -209,9 +196,9 @@
         out.putDouble(in.getDouble());
     }
 
-    public static void unpackArray(ByteBuffer in, ByteBuffer out, long uLen) {
+    public static void unpackArray(ByteBuffer in, ByteBuffer out, long uLen) throws HyracksDataException {
         if (uLen > Integer.MAX_VALUE) {
-            throw new UnsupportedOperationException("String is too long");
+            throw new UnsupportedOperationException("Array is too long");
         }
         int count = (int) uLen;
         int offs = out.position();
@@ -233,7 +220,7 @@
         out.putInt(asxLenPos, totalLen);
     }
 
-    public static void unpackMap(ByteBuffer in, ByteBuffer out, int count) {
+    public static void unpackMap(ByteBuffer in, ByteBuffer out, int count) throws HyracksDataException {
         //TODO: need to handle typed records. this only produces a completely open record.
         //hdr size = 6?
         int startOffs = out.position();
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
index 44a17a9..39e480a 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/ExternalAssignBatchRuntimeFactory.java
@@ -19,18 +19,44 @@
 
 package org.apache.asterix.external.operators;
 
+import static org.msgpack.core.MessagePack.Code.ARRAY16;
+import static org.msgpack.core.MessagePack.Code.ARRAY32;
+import static org.msgpack.core.MessagePack.Code.FIXARRAY_PREFIX;
+import static org.msgpack.core.MessagePack.Code.isFixedArray;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.external.library.PythonLibraryEvaluator;
+import org.apache.asterix.external.library.PythonLibraryEvaluatorFactory;
+import org.apache.asterix.external.library.msgpack.MessageUnpackerToADM;
 import org.apache.asterix.om.functions.IExternalFunctionDescriptor;
-import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.Counter;
+import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputOneFramePushRuntime;
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputPushRuntime;
 import org.apache.hyracks.algebricks.runtime.operators.base.AbstractOneInputOneOutputRuntimeFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
-import org.apache.hyracks.api.exceptions.ErrorCode;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.Warning;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
+import org.msgpack.core.MessagePack;
+import org.msgpack.core.MessagePackException;
+import org.msgpack.core.MessageUnpacker;
+import org.msgpack.core.buffer.ArrayBufferInput;
 
 public final class ExternalAssignBatchRuntimeFactory extends AbstractOneInputOneOutputRuntimeFactory {
 
     private static final long serialVersionUID = 1L;
-
     private int[] outColumns;
     private final IExternalFunctionDescriptor[] fnDescs;
     private final int[][] fnArgColumns;
@@ -44,9 +70,242 @@
     }
 
     @Override
-    public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx)
-            throws HyracksDataException {
-        throw new HyracksDataException(ErrorCode.OPERATOR_NOT_IMPLEMENTED, sourceLoc,
-                PhysicalOperatorTag.ASSIGN_BATCH.toString());
+    public AbstractOneInputOneOutputPushRuntime createOneOutputPushRuntime(IHyracksTaskContext ctx) {
+
+        final int[] projectionToOutColumns = new int[projectionList.length];
+        for (int j = 0; j < projectionList.length; j++) {
+            projectionToOutColumns[j] = Arrays.binarySearch(outColumns, projectionList[j]);
+        }
+
+        return new AbstractOneInputOneOutputOneFramePushRuntime() {
+
+            private ByteBuffer outputWrapper;
+            private List<ByteBuffer> argHolders;
+            ArrayTupleBuilder tupleBuilder;
+            private List<Pair<Long, PythonLibraryEvaluator>> libraryEvaluators;
+            private ATypeTag[][] nullCalls;
+            private int[] numCalls;
+            private VoidPointable ref;
+            private MessageUnpacker unpacker;
+            private ArrayBufferInput unpackerInput;
+            private List<Pair<ByteBuffer, Counter>> batchResults;
+
+            @Override
+            public void open() throws HyracksDataException {
+                super.open();
+                initAccessAppend(ctx);
+                tupleBuilder = new ArrayTupleBuilder(projectionList.length);
+                tRef = new FrameTupleReference();
+                ref = VoidPointable.FACTORY.createPointable();
+                libraryEvaluators = new ArrayList<>();
+                try {
+                    PythonLibraryEvaluatorFactory evalFactory = new PythonLibraryEvaluatorFactory(ctx);
+                    for (IExternalFunctionDescriptor fnDesc : fnDescs) {
+                        PythonLibraryEvaluator eval = evalFactory.getEvaluator(fnDesc.getFunctionInfo(), sourceLoc);
+                        long id = eval.initialize(fnDesc.getFunctionInfo());
+                        libraryEvaluators.add(new Pair<>(id, eval));
+                    }
+                } catch (IOException | AsterixException e) {
+                    throw RuntimeDataException.create(ErrorCode.EXTERNAL_UDF_EXCEPTION, e, sourceLoc, e.getMessage());
+                }
+                argHolders = new ArrayList<>(fnArgColumns.length);
+                for (int i = 0; i < fnArgColumns.length; i++) {
+                    argHolders.add(ctx.allocateFrame());
+                }
+                outputWrapper = ctx.allocateFrame();
+                nullCalls = new ATypeTag[argHolders.size()][0];
+                numCalls = new int[fnArgColumns.length];
+                batchResults = new ArrayList<>(argHolders.size());
+                for (int i = 0; i < argHolders.size(); i++) {
+                    batchResults.add(new Pair<>(ctx.allocateFrame(), new Counter(-1)));
+                }
+                unpackerInput = new ArrayBufferInput(new byte[0]);
+                unpacker = MessagePack.newDefaultUnpacker(unpackerInput);
+            }
+
+            private void resetBuffers(int numTuples, int[] numCalls) {
+                for (int func = 0; func < fnArgColumns.length; func++) {
+                    argHolders.get(func).clear();
+                    argHolders.get(func).position(0);
+                    if (nullCalls[func].length < numTuples) {
+                        nullCalls[func] = new ATypeTag[numTuples];
+                    }
+                    numCalls[func] = numTuples;
+                    Arrays.fill(nullCalls[func], ATypeTag.TYPE);
+                    for (Pair<ByteBuffer, Counter> batch : batchResults) {
+                        batch.getFirst().clear();
+                        batch.getFirst().position(0);
+                        batch.getSecond().set(-1);
+                    }
+                }
+            }
+
+            private ATypeTag handleNullMatrix(int func, int t, ATypeTag argumentPresence, ATypeTag argumentStatus) {
+                //If any argument is unknown, skip call. If any argument is null, return null, first.
+                //However, if any argument is missing, return missing instead.
+                if (nullCalls[func][t] == ATypeTag.TYPE && argumentPresence != ATypeTag.TYPE) {
+                    if (argumentPresence == ATypeTag.NULL && argumentStatus != ATypeTag.MISSING) {
+                        nullCalls[func][t] = argumentPresence;
+                        return ATypeTag.NULL;
+                    } else {
+                        nullCalls[func][t] = argumentPresence;
+                        return ATypeTag.MISSING;
+                    }
+                }
+                return argumentPresence;
+            }
+
+            private void collectFunctionWarnings(List<Pair<ByteBuffer, Counter>> batchResults) throws IOException {
+                for (Pair<ByteBuffer, Counter> result : batchResults) {
+                    if (result.getSecond().get() > -1) {
+                        ByteBuffer resBuf = result.getFirst();
+                        unpackerInput.reset(resBuf.array(), resBuf.position() + resBuf.arrayOffset(),
+                                resBuf.remaining());
+                        unpacker.reset(unpackerInput);
+                        try {
+                            int numEntries = unpacker.unpackArrayHeader();
+                            for (int j = 0; j < numEntries; j++) {
+                                if (ctx.getWarningCollector().shouldWarn()) {
+                                    ctx.getWarningCollector().warn(Warning.of(sourceLoc,
+                                            ErrorCode.EXTERNAL_UDF_EXCEPTION, unpacker.unpackString()));
+                                }
+                            }
+                        } catch (MessagePackException e) {
+                            if (ctx.getWarningCollector().shouldWarn()) {
+                                ctx.getWarningCollector().warn(Warning.of(sourceLoc, ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                                        "Error retrieving returned warnings from Python UDF"));
+                            }
+                        }
+                    }
+                }
+            }
+
+            @Override
+            public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+                tAccess.reset(buffer);
+                tupleBuilder.reset();
+                try {
+                    int numTuples = tAccess.getTupleCount();
+                    resetBuffers(numTuples, numCalls);
+                    //build columns of arguments for each function
+                    for (int t = 0; t < numTuples; t++) {
+                        for (int func = 0; func < fnArgColumns.length; func++) {
+                            tRef.reset(tAccess, t);
+                            int[] cols = fnArgColumns[func];
+                            //TODO: switch between fixarray/array16/array32 where appropriate
+                            ATypeTag argumentStatus = ATypeTag.TYPE;
+                            if (!fnDescs[func].getFunctionInfo().getNullCall()) {
+                                for (int colIdx = 0; colIdx < cols.length; colIdx++) {
+                                    ref.set(buffer.array(), tRef.getFieldStart(cols[colIdx]),
+                                            tRef.getFieldLength(cols[colIdx]));
+                                    ATypeTag argumentPresence = PythonLibraryEvaluator
+                                            .peekArgument(fnDescs[func].getArgumentTypes()[colIdx], ref);
+                                    argumentStatus = handleNullMatrix(func, t, argumentPresence, argumentStatus);
+                                }
+                            }
+                            if (argumentStatus == ATypeTag.TYPE) {
+                                if (cols.length > 0) {
+                                    argHolders.get(func).put(ARRAY16);
+                                    argHolders.get(func).putShort((short) cols.length);
+                                }
+                                for (int colIdx = 0; colIdx < cols.length; colIdx++) {
+                                    ref.set(buffer.array(), tRef.getFieldStart(cols[colIdx]),
+                                            tRef.getFieldLength(cols[colIdx]));
+                                    PythonLibraryEvaluator.setArgument(fnDescs[func].getArgumentTypes()[colIdx], ref,
+                                            argHolders.get(func), fnDescs[func].getFunctionInfo().getNullCall());
+                                }
+                            } else {
+                                numCalls[func]--;
+                            }
+                            if (cols.length == 0) {
+                                PythonLibraryEvaluator.setVoidArgument(argHolders.get(func));
+                            }
+                        }
+                    }
+                    //TODO: maybe this could be done in parallel for each unique library evaluator?
+                    for (int argHolderIdx = 0; argHolderIdx < argHolders.size(); argHolderIdx++) {
+                        Pair<Long, PythonLibraryEvaluator> fnEval = libraryEvaluators.get(argHolderIdx);
+                        ByteBuffer columnResult = fnEval.getSecond().callPythonMulti(fnEval.getFirst(),
+                                argHolders.get(argHolderIdx), numCalls[argHolderIdx]);
+                        if (columnResult != null) {
+                            Pair<ByteBuffer, Counter> resultholder = batchResults.get(argHolderIdx);
+                            if (resultholder.getFirst().capacity() < columnResult.capacity()) {
+                                resultholder.setFirst(ctx.allocateFrame(columnResult.capacity()));
+                            }
+                            ByteBuffer resultBuf = resultholder.getFirst();
+                            resultBuf.clear();
+                            resultBuf.position(0);
+                            //offset 1 to skip message type
+                            System.arraycopy(columnResult.array(), columnResult.arrayOffset() + 1, resultBuf.array(),
+                                    resultBuf.arrayOffset(), columnResult.capacity() - 1);
+                            //wrapper for results and warnings arrays. always length 2
+                            consumeAndGetBatchLength(resultBuf);
+                            int numResults = (int) consumeAndGetBatchLength(resultBuf);
+                            resultholder.getSecond().set(numResults);
+                        } else {
+                            if (ctx.getWarningCollector().shouldWarn()) {
+                                ctx.getWarningCollector()
+                                        .warn(Warning.of(sourceLoc, ErrorCode.EXTERNAL_UDF_EXCEPTION,
+                                                "Function "
+                                                        + fnDescs[argHolderIdx].getFunctionInfo()
+                                                                .getFunctionIdentifier().toString()
+                                                        + " failed to execute"));
+                            }
+                        }
+                    }
+                    //decompose returned function columns into frame tuple format
+                    for (int i = 0; i < numTuples; i++) {
+                        tupleBuilder.reset();
+                        for (int f = 0; f < projectionList.length; f++) {
+                            int k = projectionToOutColumns[f];
+                            if (k >= 0) {
+                                outputWrapper.clear();
+                                outputWrapper.position(0);
+                                Pair<ByteBuffer, Counter> result = batchResults.get(k);
+                                int start = outputWrapper.arrayOffset();
+                                ATypeTag functionCalled = nullCalls[k][i];
+                                if (functionCalled == ATypeTag.TYPE) {
+                                    if (result.getSecond().get() > 0) {
+                                        MessageUnpackerToADM.unpack(result.getFirst(), outputWrapper, true);
+                                        result.getSecond().set(result.getSecond().get() - 1);
+                                    } else {
+                                        //emit NULL for functions which failed with a warning
+                                        outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                                    }
+                                } else if (functionCalled == ATypeTag.NULL) {
+                                    outputWrapper.put(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                                } else {
+                                    outputWrapper.put(ATypeTag.SERIALIZED_MISSING_TYPE_TAG);
+                                }
+                                tupleBuilder.addField(outputWrapper.array(), start, start + outputWrapper.position());
+                            } else {
+                                tupleBuilder.addField(tAccess, i, projectionList[f]);
+                            }
+                        }
+                        appendToFrameFromTupleBuilder(tupleBuilder);
+                    }
+                    collectFunctionWarnings(batchResults);
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+            }
+
+            private long consumeAndGetBatchLength(ByteBuffer buf) {
+                byte tag = buf.get();
+                if (isFixedArray(tag)) {
+                    return tag ^ FIXARRAY_PREFIX;
+                } else if (tag == ARRAY16) {
+                    return Short.toUnsignedInt(buf.getShort());
+                } else if (tag == ARRAY32) {
+                    return Integer.toUnsignedLong(buf.getInt());
+                }
+                return -1L;
+            }
+
+            @Override
+            public void flush() throws HyracksDataException {
+                appender.flush(writer);
+            }
+        };
     }
 }
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java
index 4f91b1a..c12bb58 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/operators/LibraryDeployPrepareOperatorDescriptor.java
@@ -130,17 +130,9 @@
 
                 switch (language) {
                     case JAVA:
-                        if (!LibraryDescriptor.FILE_EXT_ZIP.equals(fileExt)) {
-                            // shouldn't happen
-                            throw new IOException("Unexpected file type: " + fileExt);
-                        }
                         libraryManager.unzip(targetFile, contentsDir);
                         break;
                     case PYTHON:
-                        if (!LibraryDescriptor.FILE_EXT_PYZ.equals(fileExt)) {
-                            // shouldn't happen
-                            throw new IOException("Unexpected file type: " + fileExt);
-                        }
                         boolean extractMsgPack = ctx.getJobletContext().getServiceContext().getAppConfig()
                                 .getBoolean(PYTHON_USE_BUNDLED_MSGPACK);
                         shiv(targetFile, stageDir, contentsDir, extractMsgPack);
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
index 1e3ab45..fd5b269 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalDataConstants.java
@@ -24,6 +24,8 @@
 import java.util.function.LongSupplier;
 import java.util.function.Supplier;
 
+import org.apache.hyracks.util.StorageUtil;
+
 public class ExternalDataConstants {
 
     private ExternalDataConstants() {
@@ -257,7 +259,7 @@
     /**
      * Size default values
      */
-    public static final int DEFAULT_BUFFER_SIZE = 4096;
+    public static final int DEFAULT_BUFFER_SIZE = StorageUtil.getIntSizeInBytes(8, StorageUtil.StorageUnit.KILOBYTE);
     public static final float DEFAULT_BUFFER_INCREMENT_FACTOR = 1.5F;
     public static final int DEFAULT_QUEUE_SIZE = 64;
     public static final int MAX_RECORD_SIZE = 32000000;
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java
index 6e3be21..9ab07eb 100644
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java
+++ b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/util/ExternalLibraryUtils.java
@@ -21,13 +21,18 @@
 import java.io.IOException;
 import java.io.StringWriter;
 import java.security.MessageDigest;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
 
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.common.metadata.DataverseName;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.util.bytes.HexPrinter;
 
 public class ExternalLibraryUtils {
 
     private ExternalLibraryUtils() {
-
     }
 
     public static String digestToHexString(MessageDigest digest) throws IOException {
@@ -36,4 +41,15 @@
         HexPrinter.printHexString(hashBytes, 0, hashBytes.length, hashBuilder);
         return hashBuilder.toString();
     }
+
+    public static Map<DataverseName, Map<String, String>> produceLibraryListing(ILibraryManager libraryManager)
+            throws IOException {
+        List<Pair<DataverseName, String>> libs = libraryManager.getLibraryListing();
+        Map<DataverseName, Map<String, String>> dvToLibHashes = new TreeMap<>();
+        for (Pair<DataverseName, String> lib : libs) {
+            dvToLibHashes.computeIfAbsent(lib.first, h -> new TreeMap<>()).put(lib.getSecond(),
+                    libraryManager.getLibraryHash(lib.first, lib.second));
+        }
+        return dvToLibHashes;
+    }
 }
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DataverseNameUtils.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DataverseNameUtils.java
deleted file mode 100644
index 1a3f26a..0000000
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/DataverseNameUtils.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.lang.common.util;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.asterix.common.metadata.DataverseName;
-import org.apache.asterix.lang.common.visitor.FormatPrintVisitor;
-
-public class DataverseNameUtils {
-
-    static protected Set<Character> validIdentifierChars = new HashSet<>();
-    static protected Set<Character> validIdentifierStartChars = new HashSet<>();
-
-    static {
-        for (char ch = 'a'; ch <= 'z'; ++ch) {
-            validIdentifierChars.add(ch);
-            validIdentifierStartChars.add(ch);
-        }
-        for (char ch = 'A'; ch <= 'Z'; ++ch) {
-            validIdentifierChars.add(ch);
-            validIdentifierStartChars.add(ch);
-        }
-        for (char ch = '0'; ch <= '9'; ++ch) {
-            validIdentifierChars.add(ch);
-        }
-        validIdentifierChars.add('_');
-        validIdentifierChars.add('$');
-    }
-
-    protected static boolean needQuotes(String str) {
-        if (str.length() == 0) {
-            return false;
-        }
-        if (!validIdentifierStartChars.contains(str.charAt(0))) {
-            return true;
-        }
-        for (char ch : str.toCharArray()) {
-            if (!validIdentifierChars.contains(ch)) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    protected static String normalize(String str) {
-        if (needQuotes(str)) {
-            return FormatPrintVisitor.revertStringToQuoted(str);
-        }
-        return str;
-    }
-
-    public static String generateDataverseName(DataverseName dataverseName) {
-        List<String> dataverseNameParts = new ArrayList<>();
-        StringBuilder sb = new StringBuilder();
-        dataverseNameParts.clear();
-        dataverseName.getParts(dataverseNameParts);
-        for (int i = 0, ln = dataverseNameParts.size(); i < ln; i++) {
-            if (i > 0) {
-                sb.append(DataverseName.CANONICAL_FORM_SEPARATOR_CHAR);
-            }
-            sb.append(normalize(dataverseNameParts.get(i)));
-        }
-        return sb.toString();
-    }
-}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
index f900c92..6751171 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalFunctionCompilerUtil.java
@@ -81,7 +81,7 @@
 
         return new ExternalScalarFunctionInfo(function.getSignature().createFunctionIdentifier(), paramTypes,
                 returnType, typeComputer, lang, function.getLibraryDataverseName(), function.getLibraryName(),
-                function.getExternalIdentifier(), function.getResources(), deterministic);
+                function.getExternalIdentifier(), function.getResources(), deterministic, function.getNullCall());
     }
 
     private static IFunctionInfo getUnnestFunctionInfo(MetadataProvider metadataProvider, Function function) {
@@ -182,7 +182,7 @@
             case JAVA:
                 return false;
             case PYTHON:
-                return false;
+                return true;
             default:
                 throw new CompilationException(ErrorCode.METADATA_ERROR, language.name());
         }
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalScalarFunctionInfo.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalScalarFunctionInfo.java
index 854320a..82f74d9 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalScalarFunctionInfo.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/functions/ExternalScalarFunctionInfo.java
@@ -35,8 +35,9 @@
 
     public ExternalScalarFunctionInfo(FunctionIdentifier fid, List<IAType> parameterTypes, IAType returnType,
             IResultTypeComputer rtc, ExternalFunctionLanguage language, DataverseName libraryDataverseName,
-            String libraryName, List<String> externalIdentifier, Map<String, String> resources, boolean deterministic) {
+            String libraryName, List<String> externalIdentifier, Map<String, String> resources, boolean deterministic,
+            boolean nullCall) {
         super(fid, FunctionKind.SCALAR, parameterTypes, returnType, rtc, language, libraryDataverseName, libraryName,
-                externalIdentifier, resources, deterministic);
+                externalIdentifier, resources, deterministic, nullCall);
     }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
index 7477330..a3ec9c6 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/ExternalFunctionInfo.java
@@ -30,7 +30,7 @@
 
 public class ExternalFunctionInfo extends FunctionInfo implements IExternalFunctionInfo {
 
-    private static final long serialVersionUID = 4L;
+    private static final long serialVersionUID = 5L;
 
     private final FunctionKind kind;
     private final List<IAType> parameterTypes;
@@ -40,11 +40,12 @@
     private final String libraryName;
     private final List<String> externalIdentifier;
     private final Map<String, String> resources;
+    private final boolean nullCall;
 
     public ExternalFunctionInfo(FunctionIdentifier fid, FunctionKind kind, List<IAType> parameterTypes,
             IAType returnType, IResultTypeComputer rtc, ExternalFunctionLanguage language,
             DataverseName libraryDataverseName, String libraryName, List<String> externalIdentifier,
-            Map<String, String> resources, boolean deterministic) {
+            Map<String, String> resources, boolean deterministic, boolean nullCall) {
         super(fid, rtc, deterministic);
         this.kind = kind;
         this.parameterTypes = parameterTypes;
@@ -54,6 +55,7 @@
         this.libraryName = libraryName;
         this.externalIdentifier = externalIdentifier;
         this.resources = resources;
+        this.nullCall = nullCall;
     }
 
     @Override
@@ -94,4 +96,9 @@
     public Map<String, String> getResources() {
         return resources;
     }
+
+    @Override
+    public boolean getNullCall() {
+        return nullCall;
+    }
 }
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java
index d87d6df..9eb9875 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/IExternalFunctionInfo.java
@@ -47,4 +47,6 @@
     List<String> getExternalIdentifier();
 
     Map<String, String> getResources();
+
+    boolean getNullCall();
 }
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
index 2e8fc63..7e80cbb 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.11.post.http
@@ -16,4 +16,7 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/partition/master?partition=0&node=asterix_nc2
\ No newline at end of file
+# param partition=0
+# param node=asterix_nc2
+
+/admin/cluster/partition/master
\ No newline at end of file
diff --git a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
index e8dca0b..73aaa09 100644
--- a/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
+++ b/asterixdb/asterix-server/src/test/resources/integrationts/replication/queries/failover/resync_failed_replica/resync_failed_replica.12.post.http
@@ -16,4 +16,6 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-/admin/cluster/metadataNode?node=asterix_nc2
\ No newline at end of file
+# param node=asterix_nc2
+
+/admin/cluster/metadataNode
\ No newline at end of file
diff --git a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
index 508236a..4ea9e0e 100644
--- a/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
+++ b/asterixdb/asterix-test-framework/src/main/resources/Catalog.xsd
@@ -298,6 +298,8 @@
       <xs:restriction base="xs:string">
          <xs:enumeration value="string"/>
          <xs:enumeration value="json"/>
+         <xs:enumeration value="multipart_text"/>
+         <xs:enumeration value="multipart_binary"/>
       </xs:restriction>
    </xs:simpleType>
 
diff --git a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
index f00161c..4b8179c 100644
--- a/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
+++ b/hyracks-fullstack/algebricks/algebricks-common/src/main/java/org/apache/hyracks/algebricks/common/exceptions/AlgebricksException.java
@@ -18,6 +18,9 @@
  */
 package org.apache.hyracks.algebricks.common.exceptions;
 
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.io.Serializable;
 import java.util.Optional;
 
@@ -28,7 +31,7 @@
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
 public class AlgebricksException extends Exception implements IFormattedException {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
     public static final int UNKNOWN = 0;
     private final String component;
@@ -36,7 +39,7 @@
     private final Serializable[] params;
     private final String nodeId;
     private final SourceLocation sourceLoc;
-    protected final transient IError error;
+    protected transient IError error;
 
     @SuppressWarnings("squid:S1165") // exception class not final
     private transient volatile String msgCache;
@@ -134,4 +137,12 @@
         }
         return msgCache;
     }
+
+    private void writeObject(java.io.ObjectOutputStream out) throws IOException {
+        ErrorMessageUtil.writeObjectWithError(error, out);
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
+        error = ErrorMessageUtil.readObjectWithError(in).orElse(null);
+    }
 }
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismUtilities.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismUtilities.java
index 6a2a333..e4f7790 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismUtilities.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismUtilities.java
@@ -37,7 +37,13 @@
 
     public static void mapVariablesTopDown(ILogicalOperator op, ILogicalOperator arg,
             Map<LogicalVariable, LogicalVariable> variableMapping) throws AlgebricksException {
-        IsomorphismVariableMappingVisitor visitor = new IsomorphismVariableMappingVisitor(variableMapping);
+        mapVariablesTopDown(op, arg, variableMapping, true);
+    }
+
+    public static void mapVariablesTopDown(ILogicalOperator op, ILogicalOperator arg,
+            Map<LogicalVariable, LogicalVariable> variableMapping, boolean goThroughNts) throws AlgebricksException {
+        IsomorphismVariableMappingVisitor visitor =
+                new IsomorphismVariableMappingVisitor(variableMapping, goThroughNts);
         op.accept(visitor, arg);
     }
 
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 153ac04..8ca2b83 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -76,11 +76,14 @@
 
 public class IsomorphismVariableMappingVisitor implements ILogicalOperatorVisitor<Void, ILogicalOperator> {
 
-    private final Map<ILogicalOperator, Set<ILogicalOperator>> alreadyMapped = new HashMap<>();
+    private final Map<ILogicalOperator, Set<ILogicalOperator>> alreadyMapped;
     private final Map<LogicalVariable, LogicalVariable> variableMapping;
+    private final boolean goThroughNts;
 
-    IsomorphismVariableMappingVisitor(Map<LogicalVariable, LogicalVariable> variableMapping) {
+    IsomorphismVariableMappingVisitor(Map<LogicalVariable, LogicalVariable> variableMapping, boolean goThroughNts) {
         this.variableMapping = variableMapping;
+        this.goThroughNts = goThroughNts;
+        this.alreadyMapped = goThroughNts ? new HashMap<>() : null;
     }
 
     @Override
@@ -145,6 +148,9 @@
         if (op.getOperatorTag() != arg.getOperatorTag()) {
             return null;
         }
+        if (!goThroughNts) {
+            return null;
+        }
         Set<ILogicalOperator> mappedOps = alreadyMapped.get(op);
         if (mappedOps != null && mappedOps.contains(arg)) {
             return null;
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
index 13a758c..fe3abf2 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalExpressionDeepCopyWithNewVariablesVisitor.java
@@ -156,7 +156,9 @@
             throws AlgebricksException {
         LogicalVariable var = expr.getVariableReference();
         if (freeVars.contains(var)) {
-            return expr;
+            VariableReferenceExpression varRef = new VariableReferenceExpression(var);
+            copySourceLocation(expr, varRef);
+            return varRef;
         }
         LogicalVariable givenVarReplacement = inVarMapping.get(var);
         if (givenVarReplacement != null) {
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
index dd7bc34..059a357 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/VariableUtilities.java
@@ -131,7 +131,7 @@
      *            a typing context that keeps track of types of each variable.
      * @throws AlgebricksException
      */
-    public static void substituteVariables(ILogicalOperator op, LinkedHashMap<LogicalVariable, LogicalVariable> varMap,
+    public static void substituteVariables(ILogicalOperator op, Map<LogicalVariable, LogicalVariable> varMap,
             ITypingContext ctx) throws AlgebricksException {
         for (Map.Entry<LogicalVariable, LogicalVariable> entry : varMap.entrySet()) {
             VariableUtilities.substituteVariables(op, entry.getKey(), entry.getValue(), ctx);
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifier.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifier.java
index 434b46e..a072d11 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifier.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/plan/PlanStructureVerifier.java
@@ -43,6 +43,7 @@
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
@@ -303,6 +304,12 @@
                             PlanStabilityVerifier.printExpression(expr, prettyPrinter), firstOp, currentOp);
                 }
             }
+            if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
+                AbstractFunctionCallExpression callExpr = (AbstractFunctionCallExpression) expr;
+                for (Mutable<ILogicalExpression> argRef : callExpr.getArguments()) {
+                    transform(argRef);
+                }
+            }
             return false;
         }
     }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index cf95d01..3ae1218 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -807,7 +807,7 @@
             fields.add(new MutableObject<>(varExprRef));
             // add the same field as input to the corresponding local function propagating the type of the field
             expr = new AggregateFunctionCallExpression(typeFun, false,
-                    Collections.singletonList(new MutableObject<>(varExprRef)));
+                    Collections.singletonList(new MutableObject<>(varExprRef.cloneExpression())));
             // add the type propagating function to the list of the local functions
             localOutVariable = context.newVar();
             localResultVariables.add(localOutVariable);
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineAssignIntoAggregateRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineAssignIntoAggregateRule.java
index ed43f29..d966ed1 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineAssignIntoAggregateRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineAssignIntoAggregateRule.java
@@ -138,7 +138,7 @@
                 ILogicalExpression e = eRef.getValue();
                 Pair<Boolean, ILogicalExpression> p = e.accept(this, arg);
                 if (p.first) {
-                    eRef.setValue(p.second);
+                    eRef.setValue(p.second.cloneExpression());
                     changed = true;
                 }
             }
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateIsomorphicSubplanRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateIsomorphicSubplanRule.java
index 1410a3a..7859f92 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateIsomorphicSubplanRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/EliminateIsomorphicSubplanRule.java
@@ -1,4 +1,5 @@
 /*
+/*
  * 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
@@ -19,70 +20,135 @@
 
 package org.apache.hyracks.algebricks.rewriter.rules.subplan;
 
+import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Deque;
+import java.util.HashMap;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.common.utils.ListSet;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SubplanOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.IsomorphismUtilities;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorManipulationUtil;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 
 /**
- * <pre>
- * This rules searches for
- *   SUBPLAN_1 { v1 = ... } (directly followed by)
- *   SUBPLAN_2 { v2 = ... }
- * and eliminates nested plans from subplan_1 that are isomorphic to nested plans defined by subplan_2.
- * The variables produced by eliminated nested plans are then ASSIGN-ed to variables produced by
- * matching nested plans in subplan_2:
- *   ASSIGN { v1 = v2 }
- *   SUBPLAN_2 { v2 = ...}
+ * This rule consolidates two subplan operators into a single subplan operator.
+ * It searches for two adjacent subplan operators in the plan
  *
- * Note: SUBPLAN_1 will remain in the plan (below ASSIGN) if some of its nested plans could not be eliminated.
+ * <pre>
+ * SUBPLAN_1 {
+ *   AGGREGATE_1 [v1=...]
+ *   ASSIGN_i_j (zero or more)
+ *   rest_ops_1
+ * }
+ * SUBPLAN_2 {
+ *   AGGREGATE_2 [v2=...]
+ *   ASSIGN_m_n (zero or more)
+ *   rest_ops_2
+ * }
+ * </pre>
+ *
+ * If {@code rest_ops_1} segment is isomorphic with {@code rest_ops_2} segment then
+ * this rule consolidates both subplans into a single (lower) one.
+ * Variables produced {@code rest_ops_1} and used by AGGREGATE_1 / ASSIGN_1_i
+ * are replaced with variables produced by {@code rest_ops_2}
+ *
+ * <pre>
+ * SUBPLAN_2 {
+ *   AGGREGATE [v1=..., v2=...]
+ *   ASSIGN_i_j (zero or more)
+ *   ASSIGN_m_n (zero or more)
+ *   rest_ops_2
+ * }
+ * </pre>
+ *
+ * Note: this rule keeps {@code SUBPLAN_1} if it had several nested plans and
+ * some of those nested plans could not be moved into the lower subplan operator.
  * </pre>
  */
-public class EliminateIsomorphicSubplanRule implements IAlgebraicRewriteRule {
+public final class EliminateIsomorphicSubplanRule implements IAlgebraicRewriteRule {
+
+    private List<AggregateOperator> targetSubplan1Roots;
+
+    private List<AggregateOperator> targetSubplan2Roots;
+
+    private List<Map<LogicalVariable, LogicalVariable>> targetVarMaps;
+
+    private Map<LogicalVariable, LogicalVariable> tmpVarMap;
+
+    private Mutable<AggregateOperator> tmpAggOpRef;
+
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
-        AbstractLogicalOperator op1 = (AbstractLogicalOperator) opRef.getValue();
-        if (op1.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
+        if (opRef.getValue().getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
             return false;
         }
-        SubplanOperator subplan1 = (SubplanOperator) op1;
+        if (context.checkIfInDontApplySet(this, opRef.getValue())) {
+            return false;
+        }
 
+        boolean applied = false;
+        for (;;) {
+            context.addToDontApplySet(this, opRef.getValue());
+            Pair<Boolean, Mutable<ILogicalOperator>> p = mergeSubplanIntoChildSubplan(opRef, context);
+            if (p == null) {
+                break;
+            }
+            applied |= p.first;
+            opRef = p.second;
+        }
+
+        return applied;
+    }
+
+    /**
+     * Returns {@code null} if given operator's child operator is not a SUBPLAN.
+     * Otherwise attempts to merge the given SUBPLAN into its child and returns a pair of values.
+     * The first value in the pair is a boolean indicating whether the rewriting succeeded or not,
+     * the second value is a reference to the lower subplan (always returned even if the rewriting did not happen)
+     */
+    private Pair<Boolean, Mutable<ILogicalOperator>> mergeSubplanIntoChildSubplan(Mutable<ILogicalOperator> op1Ref,
+            IOptimizationContext context) throws AlgebricksException {
+        AbstractLogicalOperator op1 = (AbstractLogicalOperator) op1Ref.getValue();
+        SubplanOperator subplan1 = (SubplanOperator) op1;
         Mutable<ILogicalOperator> op2Ref = subplan1.getInputs().get(0);
         ILogicalOperator op2 = op2Ref.getValue();
         if (op2.getOperatorTag() != LogicalOperatorTag.SUBPLAN) {
-            return false;
+            return null;
         }
         SubplanOperator subplan2 = (SubplanOperator) op2;
 
-        Map<LogicalVariable, LogicalVariable> assignVarMap = new LinkedHashMap<>();
-        Map<LogicalVariable, LogicalVariable> tmpVarMap = new LinkedHashMap<>();
-        List<LogicalVariable> tmpVarList = new ArrayList<>();
+        reset();
 
         for (Iterator<ILogicalPlan> nestedPlanIter = subplan1.getNestedPlans().iterator(); nestedPlanIter.hasNext();) {
             ILogicalPlan nestedPlan = nestedPlanIter.next();
             for (Iterator<Mutable<ILogicalOperator>> rootOpIter = nestedPlan.getRoots().iterator(); rootOpIter
                     .hasNext();) {
                 ILogicalOperator rootOp = rootOpIter.next().getValue();
-                if (findIsomorphicPlanRoot(rootOp, subplan2, assignVarMap, tmpVarList, tmpVarMap)) {
+                if (findIsomorphicPlanSegment(rootOp, subplan2, tmpAggOpRef, tmpVarMap)) {
+                    targetSubplan1Roots.add((AggregateOperator) rootOp);
+                    targetSubplan2Roots.add(Objects.requireNonNull(tmpAggOpRef.getValue()));
+                    targetVarMaps.add(new HashMap<>(tmpVarMap));
                     rootOpIter.remove();
                 }
             }
@@ -91,72 +157,155 @@
             }
         }
 
-        int assignVarCount = assignVarMap.size();
-        if (assignVarCount == 0) {
-            return false;
+        if (targetSubplan1Roots.isEmpty()) {
+            return new Pair<>(false, op2Ref);
         }
 
-        List<LogicalVariable> assignVars = new ArrayList<>(assignVarCount);
-        List<Mutable<ILogicalExpression>> assignExprs = new ArrayList<>(assignVarCount);
-
-        for (Map.Entry<LogicalVariable, LogicalVariable> me : assignVarMap.entrySet()) {
-            LogicalVariable subplan1Var = me.getKey();
-
-            LogicalVariable subplan2Var = me.getValue();
-            VariableReferenceExpression subplan2VarRef = new VariableReferenceExpression(subplan2Var);
-            subplan2VarRef.setSourceLocation(subplan2.getSourceLocation());
-
-            assignVars.add(subplan1Var);
-            assignExprs.add(new MutableObject<>(subplan2VarRef));
+        for (int i = 0, n = targetSubplan1Roots.size(); i < n; i++) {
+            AggregateOperator targetSubplan1Root = targetSubplan1Roots.get(i);
+            AggregateOperator targetSubplan2Root = targetSubplan2Roots.get(i);
+            Map<LogicalVariable, LogicalVariable> targetVarMap = targetVarMaps.get(i);
+            consolidateSubplans(targetSubplan1Root, targetSubplan2Root, targetVarMap, context);
         }
 
-        Mutable<ILogicalOperator> assignInputOp;
+        context.computeAndSetTypeEnvironmentForOperator(subplan2);
+
         if (subplan1.getNestedPlans().isEmpty()) {
-            assignInputOp = op2Ref;
+            // remove subplan1 from the tree
+            op1Ref.setValue(subplan2);
+            return new Pair<>(true, op1Ref);
         } else {
             // some nested plans were removed from subplan1 -> recompute its type environment
             context.computeAndSetTypeEnvironmentForOperator(subplan1);
-            assignInputOp = new MutableObject<>(subplan1);
+            return new Pair<>(true, op2Ref);
         }
-
-        AssignOperator assignOp = new AssignOperator(assignVars, assignExprs);
-        assignOp.setSourceLocation(subplan1.getSourceLocation());
-        assignOp.getInputs().add(assignInputOp);
-
-        context.computeAndSetTypeEnvironmentForOperator(assignOp);
-        opRef.setValue(assignOp);
-
-        return true;
     }
 
     /**
      * Finds nested plan root in given subplan that is isomorphic to given operator
      * and returns their variable mappings
      */
-    private boolean findIsomorphicPlanRoot(ILogicalOperator op, SubplanOperator subplanOp,
-            Map<LogicalVariable, LogicalVariable> outVarMap, List<LogicalVariable> tmpVarList,
-            Map<LogicalVariable, LogicalVariable> tmpVarMap) throws AlgebricksException {
+    private static boolean findIsomorphicPlanSegment(ILogicalOperator op, SubplanOperator subplanOp,
+            Mutable<AggregateOperator> outSubplanRootOpRef, Map<LogicalVariable, LogicalVariable> outVarMap)
+            throws AlgebricksException {
+        if (op.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+            return false;
+        }
+        AggregateOperator aggOp = (AggregateOperator) op;
+        if (aggOp.getMergeExpressions() != null) {
+            return false;
+        }
+
+        Set<LogicalVariable> freeVars = new ListSet<>();
+        OperatorPropertiesUtil.getFreeVariablesInSelfOrDesc(aggOp, freeVars);
+
+        // find first non-ASSIGN child. It'll be the root for the isomorphic segment search.
+        ILogicalOperator opChildIsomorphicCandidate = aggOp.getInputs().get(0).getValue();
+        while (opChildIsomorphicCandidate.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            if (!OperatorPropertiesUtil.isMovable(opChildIsomorphicCandidate)) {
+                return false;
+            }
+            opChildIsomorphicCandidate = opChildIsomorphicCandidate.getInputs().get(0).getValue();
+        }
+
         for (ILogicalPlan nestedPlan : subplanOp.getNestedPlans()) {
             for (Mutable<ILogicalOperator> rootOpRef : nestedPlan.getRoots()) {
                 ILogicalOperator rootOp = rootOpRef.getValue();
-                if (IsomorphismUtilities.isOperatorIsomorphicPlanSegment(op, rootOp)) {
-                    tmpVarList.clear();
-                    VariableUtilities.getProducedVariables(op, tmpVarList);
-                    tmpVarMap.clear();
-                    IsomorphismUtilities.mapVariablesTopDown(rootOp, op, tmpVarMap);
-                    tmpVarMap.keySet().retainAll(tmpVarList);
-                    if (tmpVarMap.size() == tmpVarList.size()) {
-                        outVarMap.putAll(tmpVarMap);
-                        return true;
-                    } else {
-                        return false;
-                    }
+                if (rootOp.getOperatorTag() != LogicalOperatorTag.AGGREGATE) {
+                    continue;
+                }
+                AggregateOperator aggRootOp = (AggregateOperator) rootOp;
+                if (aggRootOp.getMergeExpressions() != null) {
+                    continue;
+                }
+                if (!OperatorPropertiesUtil.disjoint(freeVars, aggRootOp.getVariables())) {
+                    // upper subplan uses variables from this subplan -> exit
+                    continue;
+                }
+
+                // find first non-ASSIGN child. It'll be the root for the isomorphic segment search.
+                ILogicalOperator rootOpChildIsomorphicCandidate = aggRootOp.getInputs().get(0).getValue();
+                while (rootOpChildIsomorphicCandidate.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+                    rootOpChildIsomorphicCandidate = rootOpChildIsomorphicCandidate.getInputs().get(0).getValue();
+                }
+
+                if (IsomorphismUtilities.isOperatorIsomorphicPlanSegment(opChildIsomorphicCandidate,
+                        rootOpChildIsomorphicCandidate)) {
+                    outSubplanRootOpRef.setValue(aggRootOp);
+                    IsomorphismUtilities.mapVariablesTopDown(rootOpChildIsomorphicCandidate, opChildIsomorphicCandidate,
+                            outVarMap, false);
+                    return true;
                 }
             }
         }
         return false;
     }
 
+    private static void consolidateSubplans(AggregateOperator upperAggRootOp, AggregateOperator targetAggRootOp,
+            Map<LogicalVariable, LogicalVariable> varMap, IOptimizationContext context) throws AlgebricksException {
+        ILogicalOperator upperChildOp = upperAggRootOp.getInputs().get(0).getValue();
+        if (upperChildOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
+            Deque<AssignOperator> upperAssignQueue = new ArrayDeque<>();
+            do {
+                upperAssignQueue.push((AssignOperator) upperChildOp);
+                upperChildOp = upperChildOp.getInputs().get(0).getValue();
+            } while (upperChildOp.getOperatorTag() == LogicalOperatorTag.ASSIGN);
+
+            ILogicalOperator targetChildOp = targetAggRootOp.getInputs().get(0).getValue();
+
+            AssignOperator upperAssignOp;
+            while ((upperAssignOp = upperAssignQueue.poll()) != null) {
+                AssignOperator upperAssignOpCopy = (AssignOperator) OperatorManipulationUtil.deepCopy(upperAssignOp);
+                upperAssignOpCopy.getInputs().clear();
+                VariableUtilities.substituteVariables(upperAssignOpCopy, varMap, null);
+
+                upperAssignOpCopy.getInputs().add(new MutableObject<>(targetChildOp));
+                context.computeAndSetTypeEnvironmentForOperator(upperAssignOpCopy);
+                targetChildOp = upperAssignOpCopy;
+            }
+
+            targetAggRootOp.getInputs().clear();
+            targetAggRootOp.getInputs().add(new MutableObject<>(targetChildOp));
+        }
+
+        AggregateOperator upperAggRootOpCopy = (AggregateOperator) OperatorManipulationUtil.deepCopy(upperAggRootOp);
+        upperAggRootOpCopy.getInputs().clear();
+        VariableUtilities.substituteVariables(upperAggRootOpCopy, varMap, null);
+
+        targetAggRootOp.getVariables().addAll(upperAggRootOpCopy.getVariables());
+        targetAggRootOp.getExpressions().addAll(upperAggRootOpCopy.getExpressions());
+
+        context.computeAndSetTypeEnvironmentForOperator(targetAggRootOp);
+    }
+
+    private void reset() {
+        if (targetSubplan1Roots == null) {
+            targetSubplan1Roots = new ArrayList<>();
+        } else {
+            targetSubplan1Roots.clear();
+        }
+        if (targetSubplan2Roots == null) {
+            targetSubplan2Roots = new ArrayList<>();
+        } else {
+            targetSubplan2Roots.clear();
+        }
+        if (targetVarMaps == null) {
+            targetVarMaps = new ArrayList<>();
+        } else {
+            targetVarMaps.clear();
+        }
+        if (tmpVarMap == null) {
+            tmpVarMap = new HashMap<>();
+        } else {
+            tmpVarMap.clear();
+        }
+        if (tmpAggOpRef == null) {
+            tmpAggOpRef = new MutableObject<>();
+        } else {
+            tmpAggOpRef.setValue(null);
+        }
+    }
+
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
index d6085f6..0769e18 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/HyracksException.java
@@ -25,15 +25,15 @@
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
 public class HyracksException extends IOException implements IFormattedException {
-    private static final long serialVersionUID = 1L;
+    private static final long serialVersionUID = 2L;
 
     public static final int UNKNOWN = 0;
     private final String component;
     private final int errorCode;
     private final Serializable[] params;
     private final String nodeId;
-    protected transient final IError error;
     private SourceLocation sourceLoc;
+    protected transient IError error;
     private transient volatile String msgCache;
 
     public static HyracksException create(Throwable cause) {
@@ -166,7 +166,11 @@
         return Optional.ofNullable(error);
     }
 
-    public boolean matches(ErrorCode errorCode) {
-        return component.equals(errorCode.component()) && getErrorCode() == errorCode.intValue();
+    private void writeObject(java.io.ObjectOutputStream out) throws IOException {
+        ErrorMessageUtil.writeObjectWithError(error, out);
+    }
+
+    private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
+        error = ErrorMessageUtil.readObjectWithError(in).orElse(null);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
index 1f814b4..33b3995 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/IFormattedException.java
@@ -46,8 +46,13 @@
     String getMessage();
 
     /**
+     * See {@link Throwable#getSuppressed()}
+     */
+    Throwable[] getSuppressed();
+
+    /**
      * If available, returns the {@link IError} associated with this exception
-     * @return the error instance, othewise {@link Optional#empty()}
+     * @return the error instance, otherwise {@link Optional#empty()}
      * @since 0.3.5.1
      */
     Optional<IError> getError();
@@ -85,4 +90,16 @@
     static boolean matchesAny(Throwable th, IError candidate, IError... otherCandidates) {
         return th instanceof IFormattedException && ((IFormattedException) th).matchesAny(candidate, otherCandidates);
     }
+
+    /**
+     * If the supplied {@link Throwable} is an instance of {@link IFormattedException}, return the {@link IError}
+     * associated with this exception if available
+     *
+     * @return the error instance, otherwise {@link Optional#empty()}
+     * @since 0.3.5.1
+     */
+    static Optional<IError> getError(Throwable throwable) {
+        return throwable instanceof IFormattedException ? ((IFormattedException) throwable).getError()
+                : Optional.empty();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
index a7372ae..6479c8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/util/ErrorMessageUtil.java
@@ -21,11 +21,14 @@
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.Formatter;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Optional;
 import java.util.Properties;
 
 import org.apache.hyracks.api.exceptions.IError;
@@ -140,4 +143,22 @@
         }
         return enumMessages;
     }
+
+    public static void writeObjectWithError(IError error, ObjectOutputStream out) throws IOException {
+        out.defaultWriteObject();
+        out.writeObject(error);
+    }
+
+    public static Optional<IError> readObjectWithError(ObjectInputStream in)
+            throws IOException, ClassNotFoundException {
+        in.defaultReadObject();
+        try {
+            return Optional.ofNullable((IError) in.readObject());
+        } catch (IllegalArgumentException e) {
+            // this is expected in case of error codes not available in this version; return null
+            LOGGER.debug("unable to deserialize error object due to {}, the error reference will be empty",
+                    String.valueOf(e));
+            return Optional.empty();
+        }
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
index d803732..22b240a 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/controllers/NCConfig.java
@@ -94,6 +94,7 @@
         IO_WORKERS_PER_PARTITION(POSITIVE_INTEGER, 2),
         IO_QUEUE_SIZE(POSITIVE_INTEGER, 10),
         PYTHON_CMD(STRING, (String) null),
+        PYTHON_CMD_AUTOLOCATE(BOOLEAN, false),
         PYTHON_ADDITIONAL_PACKAGES(STRING_ARRAY, new String[0]),
         PYTHON_USE_BUNDLED_MSGPACK(BOOLEAN, true),
         PYTHON_ARGS(STRING_ARRAY, (String[]) null),
@@ -235,13 +236,15 @@
                 case IO_QUEUE_SIZE:
                     return "Length of the queue used for requests to write and read";
                 case PYTHON_CMD:
-                    return "Absolute path to python interpreter. Defaults to environmental Python3";
+                    return "Absolute path to python interpreter";
                 case PYTHON_ADDITIONAL_PACKAGES:
                     return "List of additional paths, separated by a path separator character, to add to sys.path behind msgpack and library package paths";
                 case PYTHON_USE_BUNDLED_MSGPACK:
                     return "True to include bundled msgpack on Python sys.path, false to use system-provided msgpack";
                 case PYTHON_ARGS:
                     return "Python args to pass to Python interpreter";
+                case PYTHON_CMD_AUTOLOCATE:
+                    return "Whether or not to attempt to automatically set PYTHON_CMD to a usable interpreter";
                 case CREDENTIAL_FILE:
                     return "Path to HTTP basic credentials";
                 default:
diff --git a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
index 514a7dd..fa64003 100644
--- a/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
+++ b/hyracks-fullstack/hyracks/hyracks-http/src/main/java/org/apache/hyracks/http/server/AbstractServlet.java
@@ -22,6 +22,8 @@
 import static com.fasterxml.jackson.databind.SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS;
 
 import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URLDecoder;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
 import java.util.concurrent.ConcurrentMap;
@@ -175,7 +177,12 @@
     public String localPath(IServletRequest request) {
         final String uri = request.getHttpRequest().uri();
         int queryStart = uri.indexOf('?');
-        return queryStart == -1 ? uri.substring(trim(uri)) : uri.substring(trim(uri), queryStart);
+        String localPath = queryStart == -1 ? uri.substring(trim(uri)) : uri.substring(trim(uri), queryStart);
+        try {
+            return URLDecoder.decode(localPath, StandardCharsets.UTF_8.name());
+        } catch (UnsupportedEncodingException e) {
+            throw new IllegalArgumentException(e);
+        }
     }
 
     public String servletPath(IServletRequest request) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
index 753d27a..40998b7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperation.java
@@ -177,4 +177,8 @@
      */
     boolean isActive();
 
+    /**
+     * @return whether this IO operation is completed
+     */
+    boolean isCompleted();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerFactory.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerFactory.java
index 1c8a4e1..36bfc5a 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/api/ILSMIOOperationSchedulerFactory.java
@@ -21,7 +21,8 @@
 import java.util.concurrent.ThreadFactory;
 
 public interface ILSMIOOperationSchedulerFactory {
-    ILSMIOOperationScheduler createIoScheduler(ThreadFactory threadFactory, IIoOperationFailedCallback callback);
+    ILSMIOOperationScheduler createIoScheduler(ThreadFactory threadFactory, IIoOperationFailedCallback callback,
+            int maxNumRunningFlushes, int maxNumScheduledMerges, int maxNumRunningMerges);
 
     String getName();
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java
index 78185f0..e266a6f 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractAsynchronousScheduler.java
@@ -27,6 +27,7 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadFactory;
 
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
@@ -34,13 +35,18 @@
 
 public abstract class AbstractAsynchronousScheduler implements ILSMIOOperationScheduler, Closeable {
     protected final ExecutorService executor;
+
+    private final int maxNumFlushes;
     protected final Map<String, ILSMIOOperation> runningFlushOperations = new HashMap<>();
-    protected final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations = new HashMap<>();
+    protected final Deque<ILSMIOOperation> waitingFlushOperations = new ArrayDeque<>();
+    protected final Deque<ILSMIOOperation> waitingMergeOperations = new ArrayDeque<>();
+
     protected final Map<String, Throwable> failedGroups = new HashMap<>();
 
-    public AbstractAsynchronousScheduler(ThreadFactory threadFactory, final IIoOperationFailedCallback callback) {
-        executor = new IoOperationExecutor(threadFactory, this, callback, runningFlushOperations,
-                waitingFlushOperations, failedGroups);
+    public AbstractAsynchronousScheduler(ThreadFactory threadFactory, final IIoOperationFailedCallback callback,
+            int maxNumFlushes) {
+        executor = new IoOperationExecutor(threadFactory, this, callback, runningFlushOperations, failedGroups);
+        this.maxNumFlushes = maxNumFlushes;
     }
 
     @Override
@@ -61,27 +67,35 @@
         }
     }
 
+    @Override
+    public void completeOperation(ILSMIOOperation operation) throws HyracksDataException {
+        switch (operation.getIOOpertionType()) {
+            case FLUSH:
+                completeFlush(operation);
+                break;
+            case MERGE:
+                completeMerge(operation);
+            case NOOP:
+                return;
+            default:
+                // this should never happen
+                // just guard here to avoid silent failures in case of future extensions
+                throw new IllegalArgumentException("Unknown operation type " + operation.getIOOpertionType());
+        }
+    }
+
     protected abstract void scheduleMerge(ILSMIOOperation operation);
 
+    protected abstract void completeMerge(ILSMIOOperation operation);
+
     protected void scheduleFlush(ILSMIOOperation operation) {
         String id = operation.getIndexIdentifier();
         synchronized (executor) {
-            if (failedGroups.containsKey(id)) {
-                // Group failure. Fail the operation right away
-                operation.setStatus(LSMIOOperationStatus.FAILURE);
-                operation.setFailure(new RuntimeException("Operation group " + id + " has permanently failed",
-                        failedGroups.get(id)));
-                operation.complete();
+            if (checkFailedFlush(operation)) {
                 return;
             }
-            if (runningFlushOperations.containsKey(id)) {
-                if (waitingFlushOperations.containsKey(id)) {
-                    waitingFlushOperations.get(id).offer(operation);
-                } else {
-                    Deque<ILSMIOOperation> q = new ArrayDeque<>();
-                    q.offer(operation);
-                    waitingFlushOperations.put(id, q);
-                }
+            if (runningFlushOperations.size() >= maxNumFlushes || runningFlushOperations.containsKey(id)) {
+                waitingFlushOperations.add(operation);
             } else {
                 runningFlushOperations.put(id, operation);
                 executor.submit(operation);
@@ -89,6 +103,52 @@
         }
     }
 
+    private boolean checkFailedFlush(ILSMIOOperation operation) {
+        String id = operation.getIndexIdentifier();
+        if (failedGroups.containsKey(id)) {
+            // Group failure. Fail the operation right away
+            operation.setStatus(LSMIOOperationStatus.FAILURE);
+            operation.setFailure(
+                    new RuntimeException("Operation group " + id + " has permanently failed", failedGroups.get(id)));
+            operation.complete();
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    private void completeFlush(ILSMIOOperation operation) {
+        String id = operation.getIndexIdentifier();
+        synchronized (executor) {
+            runningFlushOperations.remove(id);
+
+            // Schedule flushes in FIFO order. Must make sure that there is at most one scheduled flush for each index.
+            for (ILSMIOOperation flushOp : waitingFlushOperations) {
+                String flushOpId = flushOp.getIndexIdentifier();
+                if (runningFlushOperations.size() < maxNumFlushes) {
+                    if (!runningFlushOperations.containsKey(flushOpId) && !flushOp.isCompleted()
+                            && !checkFailedFlush(flushOp)) {
+                        runningFlushOperations.put(flushOpId, flushOp);
+                        executor.submit(flushOp);
+                    }
+                } else {
+                    break;
+                }
+            }
+
+            // cleanup scheduled flushes
+            while (!waitingFlushOperations.isEmpty()) {
+                ILSMIOOperation top = waitingFlushOperations.peek();
+                if (top.isCompleted() || runningFlushOperations.get(top.getIndexIdentifier()) == top) {
+                    waitingFlushOperations.poll();
+                } else {
+                    break;
+                }
+            }
+
+        }
+    }
+
     @Override
     public void close() throws IOException {
         executor.shutdown();
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
index 0938b5f..8317ca7 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AbstractIoOperation.java
@@ -202,6 +202,11 @@
         return isActive.get();
     }
 
+    @Override
+    public synchronized boolean isCompleted() {
+        return completed;
+    }
+
     public void waitIfPaused() throws HyracksDataException {
         synchronized (this) {
             while (!isActive.get()) {
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
index ac3481c..afd9a49 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/AsynchronousScheduler.java
@@ -35,26 +35,49 @@
     public static final ILSMIOOperationSchedulerFactory FACTORY = new ILSMIOOperationSchedulerFactory() {
         @Override
         public ILSMIOOperationScheduler createIoScheduler(ThreadFactory threadFactory,
-                IIoOperationFailedCallback callback) {
-            return new AsynchronousScheduler(threadFactory, callback);
+                IIoOperationFailedCallback callback, int maxNumRunningFlushes, int maxNumScheduledMerges,
+                int maxNumRunningMerges) {
+            return new AsynchronousScheduler(threadFactory, callback, maxNumRunningFlushes, maxNumRunningMerges);
         }
 
+        @Override
         public String getName() {
             return "async";
         }
     };
 
-    public AsynchronousScheduler(ThreadFactory threadFactory, IIoOperationFailedCallback callback) {
-        super(threadFactory, callback);
+    private final int maxNumRunningMerges;
+    private int numRunningMerges = 0;
+
+    public AsynchronousScheduler(ThreadFactory threadFactory, IIoOperationFailedCallback callback,
+            int maxNumRunningFlushes, int maxNumRunningMerges) {
+        super(threadFactory, callback, maxNumRunningFlushes);
+        this.maxNumRunningMerges = maxNumRunningMerges;
     }
 
     @Override
     protected void scheduleMerge(ILSMIOOperation operation) {
-        executor.submit(operation);
+        synchronized (executor) {
+            if (numRunningMerges >= maxNumRunningMerges) {
+                waitingMergeOperations.add(operation);
+            } else {
+                doScheduleMerge(operation);
+            }
+        }
     }
 
     @Override
-    public void completeOperation(ILSMIOOperation operation) {
-        // no op
+    protected void completeMerge(ILSMIOOperation operation) {
+        synchronized (executor) {
+            --numRunningMerges;
+            if (!waitingMergeOperations.isEmpty() && numRunningMerges < maxNumRunningMerges) {
+                doScheduleMerge(waitingMergeOperations.poll());
+            }
+        }
+    }
+
+    private void doScheduleMerge(ILSMIOOperation operation) {
+        ++numRunningMerges;
+        executor.submit(operation);
     }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/GreedyScheduler.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/GreedyScheduler.java
index 742ae24..f3afa43 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/GreedyScheduler.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/GreedyScheduler.java
@@ -18,85 +18,141 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.ThreadFactory;
 
 import org.apache.hyracks.storage.am.lsm.common.api.IIoOperationFailedCallback;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationScheduler;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperationSchedulerFactory;
 
 /**
- * This is a greedy asynchronous scheduler that always allocates the full bandwidth for the merge operation
- * with the smallest required disk bandwidth to minimize the number of disk components. It has been proven
- * that if the number of components in all merge operations are the same, then this scheduler is optimal
- * by always minimizing the number of disk components over time; if not, this is still a good heuristic
+ * Under the greedy scheduler, a merge operation has the following lifecycles. When the merge policy submits a
+ * merge operation to the greedy scheduler, the merge operation is SCHEDULED if the number of scheduled merge
+ * operations is smaller than maxNumScheduledMergeOperations; otherwise, the merge operation is WAITING and is
+ * stored into a queue. WAITING merge operations will be scheduled after some existing merge operations finish
+ * in a FIFO order.
+ *
+ * The greedy scheduler always runs at most one (and smallest) merge operation for each LSM-tree. The maximum number of
+ * running merge operations is controlled by maxNumRunningMergeOperations. A SCHEDULED merge operation can become
+ * RUNNING if the greedy scheduler resumes this merge operation, and a RUNNING merge operation can become SCHEDULED
+ * if the greedy scheduler pauses this merge operation.
  *
  */
 public class GreedyScheduler extends AbstractAsynchronousScheduler {
-    public static final ILSMIOOperationSchedulerFactory FACTORY = new ILSMIOOperationSchedulerFactory() {
+    public static ILSMIOOperationSchedulerFactory FACTORY = new ILSMIOOperationSchedulerFactory() {
         @Override
         public ILSMIOOperationScheduler createIoScheduler(ThreadFactory threadFactory,
-                IIoOperationFailedCallback callback) {
-            return new GreedyScheduler(threadFactory, callback);
+                IIoOperationFailedCallback callback, int maxNumRunningFlushes, int maxNumScheduledMerges,
+                int maxNumRunningMerges) {
+            return new GreedyScheduler(threadFactory, callback, maxNumRunningFlushes, maxNumScheduledMerges,
+                    maxNumRunningMerges);
         }
 
+        @Override
         public String getName() {
             return "greedy";
         }
     };
 
-    private final Map<String, List<ILSMIOOperation>> mergeOperations = new HashMap<>();
+    private final int maxNumScheduledMerges;
+    private final int maxNumRunningMerges;
 
-    public GreedyScheduler(ThreadFactory threadFactory, IIoOperationFailedCallback callback) {
-        super(threadFactory, callback);
+    private int numScheduledMerges;
+    private final Map<String, Set<ILSMIOOperation>> scheduledMergeOperations = new HashMap<>();
+    private final Map<String, ILSMIOOperation> runningMergeOperations = new HashMap<>();
+
+    public GreedyScheduler(ThreadFactory threadFactory, IIoOperationFailedCallback callback, int maxNumRunningFlushes,
+            int maxNumScheduledMerges, int maxNumRunningMerges) {
+        super(threadFactory, callback, maxNumRunningFlushes);
+        this.maxNumScheduledMerges = maxNumScheduledMerges;
+        this.maxNumRunningMerges = maxNumRunningMerges;
     }
 
+    @Override
     protected void scheduleMerge(ILSMIOOperation operation) {
         operation.pause();
-        String id = operation.getIndexIdentifier();
         synchronized (executor) {
-            List<ILSMIOOperation> mergeOpList = mergeOperations.computeIfAbsent(id, key -> new ArrayList<>());
-            mergeOpList.add(operation);
-            dispatchMergeOperation(mergeOpList);
+            if (numScheduledMerges >= maxNumScheduledMerges) {
+                waitingMergeOperations.add(operation);
+            } else {
+                doScheduleMerge(operation);
+            }
         }
-        executor.submit(operation);
     }
 
-    private void dispatchMergeOperation(List<ILSMIOOperation> mergeOps) {
-        ILSMIOOperation activeOp = null;
+    private void doScheduleMerge(ILSMIOOperation operation) {
+        String indexIdentier = operation.getIndexIdentifier();
+        Set<ILSMIOOperation> mergeOps = scheduledMergeOperations.computeIfAbsent(indexIdentier, k -> new HashSet<>());
+        mergeOps.add(operation);
+        executor.submit(operation);
+        numScheduledMerges++;
+
+        dispatchMergeOperation(indexIdentier, mergeOps);
+    }
+
+    private void dispatchMergeOperation(String indexIdentier, Set<ILSMIOOperation> mergeOps) {
+        if (!runningMergeOperations.containsKey(indexIdentier)
+                && runningMergeOperations.size() >= maxNumRunningMerges) {
+            return;
+        }
+        ILSMIOOperation runningOp = null;
         ILSMIOOperation smallestMergeOp = null;
         for (ILSMIOOperation op : mergeOps) {
             if (op.isActive()) {
-                activeOp = op;
+                runningOp = op;
             }
             if (smallestMergeOp == null || op.getRemainingPages() < smallestMergeOp.getRemainingPages()) {
                 smallestMergeOp = op;
             }
         }
-        if (smallestMergeOp != activeOp) {
-            if (activeOp != null) {
-                activeOp.pause();
+        if (smallestMergeOp != runningOp) {
+            if (runningOp != null) {
+                runningOp.pause();
             }
             smallestMergeOp.resume();
+            runningMergeOperations.put(indexIdentier, smallestMergeOp);
         }
     }
 
     @Override
-    public void completeOperation(ILSMIOOperation op) {
-        if (op.getIOOpertionType() == LSMIOOperationType.MERGE) {
-            String id = op.getIndexIdentifier();
-            synchronized (executor) {
-                List<ILSMIOOperation> mergeOpList = mergeOperations.get(id);
-                mergeOpList.remove(op);
-                if (!mergeOpList.isEmpty()) {
-                    dispatchMergeOperation(mergeOpList);
+    protected void completeMerge(ILSMIOOperation op) {
+        String id = op.getIndexIdentifier();
+        synchronized (executor) {
+            Set<ILSMIOOperation> mergeOperations = scheduledMergeOperations.get(id);
+            mergeOperations.remove(op);
+            if (mergeOperations.isEmpty()) {
+                scheduledMergeOperations.remove(id);
+            }
+            runningMergeOperations.remove(id);
+            numScheduledMerges--;
+
+            if (!waitingMergeOperations.isEmpty() && numScheduledMerges < maxNumScheduledMerges) {
+                doScheduleMerge(waitingMergeOperations.poll());
+            }
+            if (runningMergeOperations.size() < maxNumRunningMerges) {
+                String indexWithMostScheduledMerges = findIndexWithMostScheduledMerges();
+                if (indexWithMostScheduledMerges != null) {
+                    dispatchMergeOperation(indexWithMostScheduledMerges,
+                            scheduledMergeOperations.get(indexWithMostScheduledMerges));
                 }
             }
         }
     }
+
+    private String findIndexWithMostScheduledMerges() {
+        String targetIndex = null;
+        int maxMerges = 0;
+        for (Map.Entry<String, Set<ILSMIOOperation>> e : scheduledMergeOperations.entrySet()) {
+            if (!runningMergeOperations.containsKey(e.getKey())
+                    && (targetIndex == null || maxMerges < e.getValue().size())) {
+                targetIndex = e.getKey();
+                maxMerges = e.getValue().size();
+            }
+        }
+        return targetIndex;
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
index d5354ed..2a48627 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/IoOperationExecutor.java
@@ -18,7 +18,6 @@
  */
 package org.apache.hyracks.storage.am.lsm.common.impls;
 
-import java.util.Deque;
 import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.RunnableFuture;
@@ -40,16 +39,14 @@
     private final IIoOperationFailedCallback callback;
     private final Map<String, ILSMIOOperation> runningFlushOperations;
     private final Map<String, Throwable> failedGroups;
-    private final Map<String, Deque<ILSMIOOperation>> waitingFlushOperations;
 
     public IoOperationExecutor(ThreadFactory threadFactory, ILSMIOOperationScheduler scheduler,
             IIoOperationFailedCallback callback, Map<String, ILSMIOOperation> runningFlushOperations,
-            Map<String, Deque<ILSMIOOperation>> waitingFlushOperations, Map<String, Throwable> failedGroups) {
+            Map<String, Throwable> failedGroups) {
         super(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS, new SynchronousQueue<>(), threadFactory);
         this.scheduler = scheduler;
         this.callback = callback;
         this.runningFlushOperations = runningFlushOperations;
-        this.waitingFlushOperations = waitingFlushOperations;
         this.failedGroups = failedGroups;
     }
 
@@ -80,20 +77,6 @@
             executedOp.complete(); // destroy if merge or successful flush
         }
         scheduler.completeOperation(executedOp);
-        if (executedOp.getIOOpertionType() == LSMIOOperationType.FLUSH) {
-            String id = executedOp.getIndexIdentifier();
-            synchronized (this) {
-                runningFlushOperations.remove(id);
-                if (waitingFlushOperations.containsKey(id)) {
-                    ILSMIOOperation op = waitingFlushOperations.get(id).poll();
-                    if (op != null) {
-                        scheduler.scheduleOperation(op);
-                    } else {
-                        waitingFlushOperations.remove(id);
-                    }
-                }
-            }
-        }
     }
 
     private void fail(ILSMIOOperation executedOp, Throwable t) throws HyracksDataException {
@@ -106,16 +89,6 @@
                 String id = executedOp.getIndexIdentifier();
                 failedGroups.put(id, t);
                 runningFlushOperations.remove(id);
-                if (waitingFlushOperations.containsKey(id)) {
-                    Deque<ILSMIOOperation> ops = waitingFlushOperations.remove(id);
-                    ILSMIOOperation next = ops.poll();
-                    while (next != null) {
-                        next.setFailure(new RuntimeException("Operation group " + id + " has permanently failed", t));
-                        next.setStatus(LSMIOOperationStatus.FAILURE);
-                        next.complete();
-                        next = ops.poll();
-                    }
-                }
             }
         }
     }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
index 7351bdf..036ade2 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/NoOpIoOperation.java
@@ -158,4 +158,9 @@
         return false;
     }
 
+    @Override
+    public boolean isCompleted() {
+        return true;
+    }
+
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
index 8adf5f7..4ab57c5 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/TracedIOOperation.java
@@ -194,4 +194,9 @@
     public boolean isActive() {
         return ioOp.isActive();
     }
+
+    @Override
+    public boolean isCompleted() {
+        return ioOp.isCompleted();
+    }
 }
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
index b4e4d84..7f8fd8a 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/LSMBTreeComponentLifecycleTest.java
@@ -250,7 +250,8 @@
                     public void operationFailed(ILSMIOOperation operation, Throwable failure) {
                         LOGGER.log(Level.ERROR, "Operation {} failed", operation, failure);
                     }
-                }), new EncapsulatingIoCallbackFactory(harness.getIOOperationCallbackFactory(), NoOpTestCallback.get(),
+                }, Integer.MAX_VALUE, Integer.MAX_VALUE),
+                new EncapsulatingIoCallbackFactory(harness.getIOOperationCallbackFactory(), NoOpTestCallback.get(),
                         NoOpTestCallback.get(), new ITestOpCallback<ILSMIOOperation>() {
                             @Override
                             public void before(ILSMIOOperation t) throws HyracksDataException {
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
index 3f36a34..f487bf1 100644
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-btree-test/src/test/java/org/apache/hyracks/storage/am/lsm/btree/perf/LSMTreeRunner.java
@@ -121,7 +121,7 @@
             public void schedulerFailed(ILSMIOOperationScheduler scheduler, Throwable failure) {
                 ExitUtil.exit(ExitUtil.EC_IO_SCHEDULER_FAILED);
             }
-        });
+        }, Integer.MAX_VALUE, Integer.MAX_VALUE);
 
         lsmtree = LSMBTreeUtil.createLSMTree(ioManager, virtualBufferCaches, file, bufferCache, typeTraits,
                 cmpFactories, bloomFilterKeyFields, bloomFilterFalsePositiveRate, new NoMergePolicy(),
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/GreedySchedulerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/GreedySchedulerTest.java
deleted file mode 100644
index d03f7a5..0000000
--- a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/GreedySchedulerTest.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.hyracks.storage.am.lsm.common.test;
-
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
-import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
-import org.apache.hyracks.storage.am.lsm.common.impls.GreedyScheduler;
-import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIoOperationFailedCallback;
-import org.junit.Assert;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-public class GreedySchedulerTest {
-
-    private static final String INDEX_1 = "index1";
-    private static final String INDEX_2 = "index2";
-
-    private final Object lock = new Object();
-
-    @Test
-    public void test() throws Exception {
-        GreedyScheduler scheduler = new GreedyScheduler(r -> new Thread(r), NoOpIoOperationFailedCallback.INSTANCE);
-        AtomicBoolean active1 = new AtomicBoolean(true);
-        ILSMIOOperation op1 = mockMergeOperation(INDEX_1, 10, active1);
-
-        scheduler.scheduleOperation(op1);
-        // op1 is activated
-        Assert.assertTrue(active1.get());
-
-        AtomicBoolean active2 = new AtomicBoolean(true);
-        ILSMIOOperation op2 = mockMergeOperation(INDEX_2, 5, active2);
-        scheduler.scheduleOperation(op2);
-        // op2 does not interactive with op1s
-        Assert.assertTrue(active1.get());
-        Assert.assertTrue(active2.get());
-
-        scheduler.completeOperation(op2);
-        Assert.assertTrue(active1.get());
-
-        AtomicBoolean active3 = new AtomicBoolean(true);
-        ILSMIOOperation op3 = mockMergeOperation(INDEX_1, 5, active3);
-        scheduler.scheduleOperation(op3);
-        Assert.assertTrue(active3.get());
-        Assert.assertFalse(active1.get());
-
-        AtomicBoolean active4 = new AtomicBoolean(true);
-        ILSMIOOperation op4 = mockMergeOperation(INDEX_1, 7, active4);
-        scheduler.scheduleOperation(op4);
-        // op3 is still active
-        Assert.assertFalse(active1.get());
-        Assert.assertTrue(active3.get());
-        Assert.assertFalse(active4.get());
-
-        // suppose op1 is completed (though unlikely in practice), now op3 is still active
-        scheduler.completeOperation(op1);
-        Assert.assertTrue(active3.get());
-        Assert.assertFalse(active4.get());
-
-        // op3 completed, op4 is active
-        scheduler.completeOperation(op3);
-        Assert.assertTrue(active4.get());
-
-        synchronized (lock) {
-            lock.notifyAll();
-        }
-        scheduler.close();
-    }
-
-    private ILSMIOOperation mockMergeOperation(String index, long remainingPages, AtomicBoolean isActive)
-            throws HyracksDataException {
-        ILSMIOOperation mergeOp = Mockito.mock(ILSMIOOperation.class);
-        Mockito.when(mergeOp.getIndexIdentifier()).thenReturn(index);
-        Mockito.when(mergeOp.getIOOpertionType()).thenReturn(LSMIOOperationType.MERGE);
-        Mockito.when(mergeOp.getRemainingPages()).thenReturn(remainingPages);
-
-        Mockito.doAnswer(new Answer<Boolean>() {
-            @Override
-            public Boolean answer(InvocationOnMock invocation) throws Throwable {
-                return isActive.get();
-            }
-        }).when(mergeOp).isActive();
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                isActive.set(true);
-                return null;
-            }
-        }).when(mergeOp).resume();
-
-        Mockito.doAnswer(new Answer<Void>() {
-            @Override
-            public Void answer(InvocationOnMock invocation) throws Throwable {
-                isActive.set(false);
-                return null;
-            }
-        }).when(mergeOp).pause();
-
-        Mockito.doAnswer(new Answer<LSMIOOperationStatus>() {
-            @Override
-            public LSMIOOperationStatus answer(InvocationOnMock invocation) throws Throwable {
-                synchronized (lock) {
-                    lock.wait();
-                }
-                return LSMIOOperationStatus.SUCCESS;
-            }
-        }).when(mergeOp).call();
-        return mergeOp;
-
-    }
-
-}
diff --git a/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/IoSchedulerTest.java b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/IoSchedulerTest.java
new file mode 100644
index 0000000..15f65a4
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-tests/hyracks-storage-am-lsm-common-test/src/test/java/org/apache/hyracks/storage/am/lsm/common/test/IoSchedulerTest.java
@@ -0,0 +1,267 @@
+/*
+ * 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.hyracks.storage.am.lsm.common.test;
+
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationStatus;
+import org.apache.hyracks.storage.am.lsm.common.api.ILSMIOOperation.LSMIOOperationType;
+import org.apache.hyracks.storage.am.lsm.common.impls.AsynchronousScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.GreedyScheduler;
+import org.apache.hyracks.storage.am.lsm.common.impls.NoOpIoOperationFailedCallback;
+import org.apache.hyracks.storage.am.lsm.common.test.IoSchedulerTest.MockedOperation;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class IoSchedulerTest {
+
+    protected static final String INDEX_1 = "index1";
+    protected static final String INDEX_2 = "index2";
+    protected static final String INDEX_3 = "index3";
+    protected static final String INDEX_4 = "index4";
+
+    protected static class MockedOperation {
+        public final ILSMIOOperation operation;
+        public final AtomicBoolean scheduled = new AtomicBoolean();
+        public final AtomicBoolean running = new AtomicBoolean();
+
+        public final Semaphore completedSemaphore = new Semaphore(0);
+
+        public MockedOperation(ILSMIOOperation mergeOp) {
+            this.operation = mergeOp;
+        }
+
+        public void waitForScheduled() throws InterruptedException {
+            synchronized (scheduled) {
+                while (!scheduled.get()) {
+                    scheduled.wait();
+                }
+            }
+        }
+
+        public void waitForRunning() throws InterruptedException {
+            synchronized (running) {
+                while (!running.get()) {
+                    running.wait();
+                }
+            }
+        }
+
+    }
+
+    @Test
+    public void testFlush() throws Exception {
+        int maxRunningFlushes = 2;
+
+        AsynchronousScheduler scheduler = (AsynchronousScheduler) AsynchronousScheduler.FACTORY
+                .createIoScheduler(r -> new Thread(r), NoOpIoOperationFailedCallback.INSTANCE, maxRunningFlushes, 0, 0);
+
+        MockedOperation op1_1 = mockFlushOperation(INDEX_1);
+        scheduler.scheduleOperation(op1_1.operation);
+        op1_1.waitForScheduled();
+
+        MockedOperation op1_2 = mockFlushOperation(INDEX_1);
+        scheduler.scheduleOperation(op1_2.operation);
+        Assert.assertFalse(op1_2.scheduled.get());
+
+        MockedOperation op2_1 = mockFlushOperation(INDEX_2);
+        scheduler.scheduleOperation(op2_1.operation);
+        op2_1.waitForScheduled();
+
+        MockedOperation op2_2 = mockFlushOperation(INDEX_2);
+        scheduler.scheduleOperation(op2_2.operation);
+        Assert.assertFalse(op2_2.scheduled.get());
+
+        // complete op1_1
+        op1_1.completedSemaphore.release();
+        op1_2.waitForScheduled();
+
+        // complete op1_2
+        op1_2.completedSemaphore.release();
+        Assert.assertFalse(op2_2.scheduled.get());
+
+        // complete op2_1
+        op2_1.completedSemaphore.release();
+        op2_2.waitForScheduled();
+
+        scheduler.close();
+    }
+
+    @Test
+    public void testAsynchronousMerge() throws Exception {
+        int maxRunningMerges = 2;
+
+        AsynchronousScheduler scheduler =
+                (AsynchronousScheduler) AsynchronousScheduler.FACTORY.createIoScheduler(r -> new Thread(r),
+                        NoOpIoOperationFailedCallback.INSTANCE, 0, maxRunningMerges, maxRunningMerges);
+
+        MockedOperation op1 = mockMergeOperation(INDEX_1, 10);
+        scheduler.scheduleOperation(op1.operation);
+        // op1 is scheduled
+        op1.waitForScheduled();
+
+        MockedOperation op2 = mockMergeOperation(INDEX_2, 10);
+        scheduler.scheduleOperation(op2.operation);
+        // op2 is scheduled
+        op2.waitForScheduled();
+
+        MockedOperation op3 = mockMergeOperation(INDEX_3, 10);
+        scheduler.scheduleOperation(op3.operation);
+        // op3 is waiting
+        Assert.assertFalse(op3.scheduled.get());
+        Assert.assertFalse(op3.running.get());
+
+        MockedOperation op4 = mockMergeOperation(INDEX_4, 10);
+        scheduler.scheduleOperation(op4.operation);
+        // op4 is waiting
+        Assert.assertFalse(op4.scheduled.get());
+        Assert.assertFalse(op4.running.get());
+
+        // complete op2 and wait for op3
+        op2.completedSemaphore.release();
+        op3.waitForScheduled();
+
+        // complete op3 and wait for op4
+        op3.completedSemaphore.release();
+        op4.waitForScheduled();
+
+        scheduler.close();
+    }
+
+    @Test
+    public void testGreedyMerge() throws Exception {
+        int maxScheduledMerges = 5;
+        int maxRunningMerges = 2;
+
+        GreedyScheduler scheduler = (GreedyScheduler) GreedyScheduler.FACTORY.createIoScheduler(r -> new Thread(r),
+                NoOpIoOperationFailedCallback.INSTANCE, 0, maxScheduledMerges, maxRunningMerges);
+
+        MockedOperation op1_1 = mockMergeOperation(INDEX_1, 10);
+        scheduler.scheduleOperation(op1_1.operation);
+        // op1_1 is running
+        op1_1.waitForScheduled();
+        op1_1.waitForRunning();
+
+        MockedOperation op2 = mockMergeOperation(INDEX_2, 10);
+        scheduler.scheduleOperation(op2.operation);
+        // op2 is running
+        op2.waitForScheduled();
+        op2.waitForRunning();
+
+        MockedOperation op3_1 = mockMergeOperation(INDEX_3, 10);
+        scheduler.scheduleOperation(op3_1.operation);
+        // op3_1 is scheduled, but not running
+        op3_1.waitForScheduled();
+        Assert.assertFalse(op3_1.running.get());
+
+        MockedOperation op3_2 = mockMergeOperation(INDEX_3, 5);
+        scheduler.scheduleOperation(op3_2.operation);
+        // op3_2 is scheduled, but not running
+        op3_2.waitForScheduled();
+        Assert.assertFalse(op3_2.running.get());
+
+        MockedOperation op4 = mockMergeOperation(INDEX_4, 10);
+        scheduler.scheduleOperation(op4.operation);
+        // op4 is scheduled, but not running
+        op4.waitForScheduled();
+        Assert.assertFalse(op4.running.get());
+
+        MockedOperation op1_2 = mockMergeOperation(INDEX_1, 5);
+        scheduler.scheduleOperation(op1_2.operation);
+        // op1_2 is waiting, not scheduled
+        Assert.assertFalse(op1_2.scheduled.get());
+        Assert.assertFalse(op1_2.running.get());
+
+        // complete op2
+        op2.completedSemaphore.release();
+
+        // op1_2 preempts op1_1 because op1_2 is smaller
+        op1_2.waitForRunning();
+        op1_2.waitForScheduled();
+
+        // op3_2 is running because index3 has more merges than index4
+        op3_2.waitForRunning();
+        Assert.assertFalse(op3_1.running.get());
+
+        scheduler.close();
+    }
+
+    protected MockedOperation mockMergeOperation(String index, long remainingPages) throws HyracksDataException {
+        return mockOperation(index, LSMIOOperationType.MERGE, remainingPages);
+    }
+
+    protected MockedOperation mockFlushOperation(String index) throws HyracksDataException {
+        return mockOperation(index, LSMIOOperationType.FLUSH, 0);
+    }
+
+    protected MockedOperation mockOperation(String index, LSMIOOperationType type, long remainingPages)
+            throws HyracksDataException {
+        ILSMIOOperation op = Mockito.mock(ILSMIOOperation.class);
+        MockedOperation mockedOp = new MockedOperation(op);
+        Mockito.when(op.getIndexIdentifier()).thenReturn(index);
+        Mockito.when(op.getIOOpertionType()).thenReturn(type);
+        Mockito.when(op.getRemainingPages()).thenReturn(remainingPages);
+
+        Mockito.doAnswer(new Answer<Boolean>() {
+            @Override
+            public Boolean answer(InvocationOnMock invocation) throws Throwable {
+                return mockedOp.running.get();
+            }
+        }).when(op).isActive();
+        Mockito.doAnswer(new Answer<Void>() {
+            @Override
+            public Void answer(InvocationOnMock invocation) throws Throwable {
+                mockedOp.running.set(true);
+                synchronized (mockedOp.running) {
+                    mockedOp.running.notifyAll();
+                }
+                return null;
+            }
+        }).when(op).resume();
+
+        Mockito.doAnswer(new Answer<Void>() {
+            @Override
+            public Void answer(InvocationOnMock invocation) throws Throwable {
+                mockedOp.running.set(false);
+                return null;
+            }
+        }).when(op).pause();
+
+        Mockito.doAnswer(new Answer<LSMIOOperationStatus>() {
+            @Override
+            public LSMIOOperationStatus answer(InvocationOnMock invocation) throws Throwable {
+                mockedOp.scheduled.set(true);
+                synchronized (mockedOp.scheduled) {
+                    mockedOp.scheduled.notifyAll();
+                }
+                mockedOp.completedSemaphore.acquire();
+                return LSMIOOperationStatus.SUCCESS;
+            }
+        }).when(op).call();
+        return mockedOp;
+
+    }
+
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingConsumer.java b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingConsumer.java
index 4f73680..36b5cf0 100644
--- a/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingConsumer.java
+++ b/hyracks-fullstack/hyracks/hyracks-util/src/main/java/org/apache/hyracks/util/ThrowingConsumer.java
@@ -46,4 +46,18 @@
             return null;
         };
     }
+
+    static <R> void forEach(Iterable<R> iterable, ThrowingConsumer<R> consumer) throws Exception {
+        try {
+            iterable.forEach(value -> {
+                try {
+                    consumer.process(value);
+                } catch (Exception e) {
+                    throw new UncheckedExecutionException(e);
+                }
+            });
+        } catch (UncheckedExecutionException e) {
+            throw (Exception) e.getCause();
+        }
+    }
 }