You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by dl...@apache.org on 2018/08/10 01:36:37 UTC

asterixdb git commit: [ASTERIXDB-2433][COMP] Optimize IN operator with constant list

Repository: asterixdb
Updated Branches:
  refs/heads/master 435831115 -> b674b1b54


[ASTERIXDB-2433][COMP] Optimize IN operator with constant list

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- Translate IN operator with a constant list into disjunctive expression
- Add compiler setting “rewrite_in_as_or” to control this feature
- Add tests

Change-Id: Ifb6c68eef86d0c002d8cb330a5c68e82cfd8cf3e
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2835
Reviewed-by: Till Westmann <ti...@apache.org>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/b674b1b5
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/b674b1b5
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/b674b1b5

Branch: refs/heads/master
Commit: b674b1b542b3d04f4befe2884e3a9dbe13e37475
Parents: 4358311
Author: Dmitry Lychagin <dm...@couchbase.com>
Authored: Thu Aug 2 11:46:08 2018 -0700
Committer: Dmitry Lychagin <dm...@couchbase.com>
Committed: Thu Aug 9 18:36:19 2018 -0700

----------------------------------------------------------------------
 .../LangExpressionToPlanTranslator.java         |   2 +-
 .../SqlppExpressionToPlanTranslator.java        | 226 +++++++++++++++++--
 .../apache/asterix/api/common/APIFramework.java |   5 +-
 .../queries/subquery/in_as_or_1.sqlpp           |  42 ++++
 .../queries/subquery/in_as_or_2.sqlpp           |  44 ++++
 .../queries/subquery/in_as_or_3.sqlpp           |  46 ++++
 .../queries/subquery/in_as_or_4.sqlpp           |  48 ++++
 .../queries/subquery/in_as_or_5.sqlpp           |  48 ++++
 .../queries/subquery/in_as_or_6.sqlpp           |  50 ++++
 .../subquery/query-ASTERIXDB-1572-2.sqlpp       |  43 ++++
 .../queries/subquery/query-ASTERIXDB-1572.sqlpp |   2 +
 ...rted-btree-search-return-optional-field.plan |  52 ++---
 .../results/subquery/in_as_or_1.plan            |  17 ++
 .../results/subquery/in_as_or_2.plan            |  28 +++
 .../results/subquery/in_as_or_3.plan            |  17 ++
 .../results/subquery/in_as_or_4.plan            |  28 +++
 .../results/subquery/in_as_or_5.plan            |  17 ++
 .../results/subquery/in_as_or_6.plan            |  28 +++
 .../subquery/query-ASTERIXDB-1572-2.plan        |  12 +
 .../subquery/in_as_or/in_as_or.1.ddl.sqlpp      |  38 ++++
 .../subquery/in_as_or/in_as_or.2.update.sqlpp   |  23 ++
 .../subquery/in_as_or/in_as_or.3.query.sqlpp    |  25 ++
 .../subquery/in_as_or/in_as_or.4.query.sqlpp    |  27 +++
 .../subquery/in_as_or/in_as_or.5.query.sqlpp    |  29 +++
 .../subquery/in_as_or/in_as_or.6.query.sqlpp    |  31 +++
 .../subquery/in_as_or/in_as_or.7.query.sqlpp    |  31 +++
 .../subquery/in_as_or/in_as_or.8.query.sqlpp    |  33 +++
 .../query-ASTERIXDB-1572.3.query.sqlpp          |   2 +
 .../query-ASTERIXDB-1572.4.query.sqlpp          |  31 +++
 .../results/subquery/in_as_or/in_as_or.3.adm    |   3 +
 .../results/subquery/in_as_or/in_as_or.4.adm    |   3 +
 .../results/subquery/in_as_or/in_as_or.5.adm    |   3 +
 .../results/subquery/in_as_or/in_as_or.6.adm    |   3 +
 .../results/subquery/in_as_or/in_as_or.7.adm    |   3 +
 .../results/subquery/in_as_or/in_as_or.8.adm    |   3 +
 .../query-ASTERIXDB-1572.2.adm                  |   0
 .../resources/runtimets/testsuite_sqlpp.xml     |   5 +
 .../lang/sqlpp/rewrites/SqlppQueryRewriter.java |   9 +-
 .../metadata/declared/MetadataProvider.java     |   5 +
 39 files changed, 999 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
----------------------------------------------------------------------
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 2c1e0f7..ae9682b 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
@@ -1407,7 +1407,7 @@ class LangExpressionToPlanTranslator
         return switchExpr;
     }
 
-    private static AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
+    protected static AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
             SourceLocation sourceLoc) throws CompilationException {
         FunctionIdentifier fid;
         switch (t) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
index 033c92f..c9ab86e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SqlppExpressionToPlanTranslator.java
@@ -38,11 +38,16 @@ import org.apache.asterix.lang.common.clause.LetClause;
 import org.apache.asterix.lang.common.expression.CallExpr;
 import org.apache.asterix.lang.common.expression.FieldBinding;
 import org.apache.asterix.lang.common.expression.GbyVariableExpressionPair;
+import org.apache.asterix.lang.common.expression.ListConstructor;
 import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.expression.OperatorExpr;
+import org.apache.asterix.lang.common.expression.QuantifiedExpression;
 import org.apache.asterix.lang.common.expression.RecordConstructor;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.common.literal.StringLiteral;
 import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.struct.OperatorType;
+import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
@@ -69,8 +74,9 @@ import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.base.ABoolean;
 import org.apache.asterix.om.base.AInt32;
-import org.apache.asterix.om.base.AMissing;
 import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IACollection;
+import org.apache.asterix.om.base.IACursor;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.om.constants.AsterixConstantValue;
 import org.apache.asterix.om.functions.BuiltinFunctions;
@@ -116,16 +122,24 @@ import org.apache.hyracks.api.exceptions.SourceLocation;
  * which is translated. The second argument of a visit method is the tuple
  * source for the current subtree.
  */
-class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator implements ILangExpressionToPlanTranslator,
+public class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator
+        implements ILangExpressionToPlanTranslator,
         ISqlppVisitor<Pair<ILogicalOperator, LogicalVariable>, Mutable<ILogicalOperator>> {
+
     private static final String ERR_MSG = "Translator should never enter this method!";
+
+    public static final String REWRITE_IN_AS_OR_OPTION = "rewrite_in_as_or";
+    private static final boolean REWRITE_IN_AS_OR_OPTION_DEFAULT = true;
+
     private Deque<Mutable<ILogicalOperator>> uncorrelatedLeftBranchStack = new ArrayDeque<>();
     private final Map<VarIdentifier, IAObject> externalVars;
+    private final boolean translateInAsOr;
 
     public SqlppExpressionToPlanTranslator(MetadataProvider metadataProvider, int currentVarCounter,
             Map<VarIdentifier, IAObject> externalVars) throws AlgebricksException {
         super(metadataProvider, currentVarCounter);
         this.externalVars = externalVars != null ? externalVars : Collections.emptyMap();
+        translateInAsOr = metadataProvider.getBooleanProperty(REWRITE_IN_AS_OR_OPTION, REWRITE_IN_AS_OR_OPTION_DEFAULT);
     }
 
     @Override
@@ -634,32 +648,39 @@ class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator imp
     protected ILogicalExpression translateVariableRef(VariableExpr varExpr) throws CompilationException {
         VarIdentifier varId = varExpr.getVar();
         if (SqlppVariableUtil.isExternalVariableIdentifier(varId)) {
-            IAObject value = externalVars.get(varId);
             SourceLocation sourceLoc = varExpr.getSourceLocation();
-            if (value == null) {
-                throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, varId.toString());
-            }
+            IAObject value = getExternalVariableValue(varId, sourceLoc);
+            return translateConstantValue(value, sourceLoc);
+        }
 
-            ILogicalExpression resultExpr;
-            ConstantExpression constExpr = new ConstantExpression(new AsterixConstantValue(value));
-            constExpr.setSourceLocation(sourceLoc);
-            resultExpr = constExpr;
-
-            IAType valueType = value.getType();
-            if (valueType.getTypeTag().isDerivedType()) {
-                ScalarFunctionCallExpression castExpr =
-                        new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE));
-                castExpr.setSourceLocation(sourceLoc);
-                // The first argument is the field
-                castExpr.getArguments().add(new MutableObject<>(resultExpr));
-                TypeCastUtils.setRequiredAndInputTypes(castExpr, BuiltinType.ANY, valueType);
-                resultExpr = castExpr;
-            }
+        return super.translateVariableRef(varExpr);
+    }
 
-            return resultExpr;
+    private IAObject getExternalVariableValue(VarIdentifier varId, SourceLocation sourceLoc)
+            throws CompilationException {
+        IAObject value = externalVars.get(varId);
+        if (value == null) {
+            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc, varId.toString());
         }
+        return value;
+    }
 
-        return super.translateVariableRef(varExpr);
+    private ILogicalExpression translateConstantValue(IAObject value, SourceLocation sourceLoc)
+            throws CompilationException {
+        ConstantExpression constExpr = new ConstantExpression(new AsterixConstantValue(value));
+        constExpr.setSourceLocation(sourceLoc);
+
+        IAType valueType = value.getType();
+        if (valueType.getTypeTag().isDerivedType()) {
+            ScalarFunctionCallExpression castExpr =
+                    new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(BuiltinFunctions.CAST_TYPE));
+            castExpr.setSourceLocation(sourceLoc);
+            castExpr.getArguments().add(new MutableObject<>(constExpr));
+            TypeCastUtils.setRequiredAndInputTypes(castExpr, BuiltinType.ANY, valueType);
+            return castExpr;
+        } else {
+            return constExpr;
+        }
     }
 
     private Pair<ILogicalOperator, LogicalVariable> produceSelectPlan(boolean isSubquery,
@@ -803,4 +824,163 @@ class SqlppExpressionToPlanTranslator extends LangExpressionToPlanTranslator imp
         return new FieldBinding(fieldName, var);
     }
 
+    @Override
+    protected boolean expressionNeedsNoNesting(Expression expr) {
+        return super.expressionNeedsNoNesting(expr) || (translateInAsOr && expr.getKind() == Kind.QUANTIFIED_EXPRESSION
+                && isInOperatorWithStaticList((QuantifiedExpression) expr));
+    }
+
+    @Override
+    public Pair<ILogicalOperator, LogicalVariable> visit(QuantifiedExpression qe, Mutable<ILogicalOperator> tupSource)
+            throws CompilationException {
+        return translateInAsOr && isInOperatorWithStaticList(qe) ? translateInOperatorWithStaticList(qe, tupSource)
+                : super.visit(qe, tupSource);
+    }
+
+    // At this point "$x in list_expr" is a quantified expression:
+    // "some $y in list_expr satisfies $x = $y"
+    // Look for such quantified expression with a constant list_expr ([e1, e2, ... eN])
+    // and translate it into "$x=e1 || $x=e2 || ... || $x=eN"
+    private boolean isInOperatorWithStaticList(QuantifiedExpression qe) {
+        if (qe.getQuantifier() != QuantifiedExpression.Quantifier.SOME) {
+            return false;
+        }
+        List<QuantifiedPair> qpList = qe.getQuantifiedList();
+        if (qpList.size() != 1) {
+            return false;
+        }
+        QuantifiedPair qp = qpList.get(0);
+
+        Expression condExpr = qe.getSatisfiesExpr();
+        if (condExpr.getKind() != Kind.OP_EXPRESSION) {
+            return false;
+        }
+        OperatorExpr opExpr = (OperatorExpr) condExpr;
+        if (opExpr.getOpList().get(0) != OperatorType.EQ) {
+            return false;
+        }
+        List<Expression> operandExprs = opExpr.getExprList();
+        if (operandExprs.size() != 2) {
+            return false;
+        }
+        int varPos = operandExprs.indexOf(qp.getVarExpr());
+        if (varPos < 0) {
+            return false;
+        }
+        Expression inExpr = qp.getExpr();
+        switch (inExpr.getKind()) {
+            case LIST_CONSTRUCTOR_EXPRESSION:
+                ListConstructor listExpr = (ListConstructor) inExpr;
+                List<Expression> itemExprs = listExpr.getExprList();
+                if (itemExprs.isEmpty()) {
+                    return false;
+                }
+                for (Expression itemExpr : itemExprs) {
+                    boolean isConst = itemExpr.getKind() == Kind.LITERAL_EXPRESSION
+                            || (itemExpr.getKind() == Kind.VARIABLE_EXPRESSION
+                                    && SqlppVariableUtil.isExternalVariableReference((VariableExpr) itemExpr));
+                    if (!isConst) {
+                        return false;
+                    }
+                }
+                return true;
+            case VARIABLE_EXPRESSION:
+                VarIdentifier inVarId = ((VariableExpr) inExpr).getVar();
+                if (!SqlppVariableUtil.isExternalVariableIdentifier(inVarId)) {
+                    return false;
+                }
+                IAObject inValue = externalVars.get(inVarId);
+                return inValue != null && inValue.getType().getTypeTag().isListType()
+                        && ((IACollection) inValue).size() > 0;
+            default:
+                return false;
+        }
+    }
+
+    private Pair<ILogicalOperator, LogicalVariable> translateInOperatorWithStaticList(QuantifiedExpression qe,
+            Mutable<ILogicalOperator> tupSource) throws CompilationException {
+        SourceLocation sourceLoc = qe.getSourceLocation();
+
+        QuantifiedPair qp = qe.getQuantifiedList().get(0);
+        VariableExpr varExpr = qp.getVarExpr();
+        List<Expression> operandExprs = ((OperatorExpr) qe.getSatisfiesExpr()).getExprList();
+        int varIdx = operandExprs.indexOf(varExpr);
+        Expression operandExpr = operandExprs.get(1 - varIdx);
+
+        Mutable<ILogicalOperator> topOp = tupSource;
+
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> eo1 = langExprToAlgExpression(operandExpr, topOp);
+        topOp = eo1.second;
+
+        LogicalVariable operandVar = context.newVar();
+        AssignOperator operandAssign = new AssignOperator(operandVar, new MutableObject<>(eo1.first));
+        operandAssign.getInputs().add(topOp);
+        operandAssign.setSourceLocation(sourceLoc);
+        topOp = new MutableObject<>(operandAssign);
+
+        List<MutableObject<ILogicalExpression>> disjuncts = new ArrayList<>();
+        Expression inExpr = qp.getExpr();
+        switch (inExpr.getKind()) {
+            case LIST_CONSTRUCTOR_EXPRESSION:
+                ListConstructor listExpr = (ListConstructor) inExpr;
+                for (Expression itemExpr : listExpr.getExprList()) {
+                    IAObject inValue;
+                    switch (itemExpr.getKind()) {
+                        case LITERAL_EXPRESSION:
+                            inValue = ConstantHelper.objectFromLiteral(((LiteralExpr) itemExpr).getValue());
+                            break;
+                        case VARIABLE_EXPRESSION:
+                            inValue = getExternalVariableValue(((VariableExpr) itemExpr).getVar(), sourceLoc);
+                            break;
+                        default:
+                            throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, sourceLoc,
+                                    itemExpr.getKind());
+                    }
+                    ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, sourceLoc);
+                    disjuncts.add(new MutableObject<>(eqExpr));
+                }
+                break;
+            case VARIABLE_EXPRESSION:
+                VarIdentifier inVarId = ((VariableExpr) inExpr).getVar();
+                IAObject inVarValue = externalVars.get(inVarId);
+                IACursor inVarCursor = ((IACollection) inVarValue).getCursor();
+                inVarCursor.reset();
+                while (inVarCursor.next()) {
+                    IAObject inValue = inVarCursor.get();
+                    ILogicalExpression eqExpr = createEqExpr(operandVar, inValue, sourceLoc);
+                    disjuncts.add(new MutableObject<>(eqExpr));
+                }
+                break;
+            default:
+                throw new IllegalStateException(String.valueOf(inExpr.getKind()));
+        }
+
+        MutableObject<ILogicalExpression> condExpr;
+        if (disjuncts.size() == 1) {
+            condExpr = disjuncts.get(0);
+        } else {
+            AbstractFunctionCallExpression orExpr =
+                    createFunctionCallExpressionForBuiltinOperator(OperatorType.OR, sourceLoc);
+            orExpr.getArguments().addAll(disjuncts);
+            condExpr = new MutableObject<>(orExpr);
+        }
+
+        LogicalVariable assignVar = context.newVar();
+        AssignOperator assignOp = new AssignOperator(assignVar, condExpr);
+        assignOp.getInputs().add(topOp);
+        assignOp.setSourceLocation(sourceLoc);
+        return new Pair<>(assignOp, assignVar);
+    }
+
+    private ILogicalExpression createEqExpr(LogicalVariable lhsVar, IAObject rhsValue, SourceLocation sourceLoc)
+            throws CompilationException {
+        VariableReferenceExpression lhsExpr = new VariableReferenceExpression(lhsVar);
+        lhsExpr.setSourceLocation(sourceLoc);
+        ILogicalExpression rhsExpr = translateConstantValue(rhsValue, sourceLoc);
+        AbstractFunctionCallExpression opExpr =
+                createFunctionCallExpressionForBuiltinOperator(OperatorType.EQ, sourceLoc);
+        opExpr.getArguments().add(new MutableObject<>(lhsExpr));
+        opExpr.getArguments().add(new MutableObject<>(rhsExpr));
+        return opExpr;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index d57be82..cfec2de 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -65,6 +65,7 @@ import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.lang.common.statement.StartFeedStatement;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.sqlpp.rewrites.SqlppQueryRewriter;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.optimizer.base.FuzzyUtils;
@@ -75,6 +76,7 @@ import org.apache.asterix.translator.ExecutionPlans;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
+import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
 import org.apache.asterix.utils.ResourceUtils;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
@@ -137,7 +139,8 @@ public class APIFramework {
                     CompilerProperties.COMPILER_PARALLELISM_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
                     FuzzyUtils.SIM_FUNCTION_PROP_NAME, FuzzyUtils.SIM_THRESHOLD_PROP_NAME,
                     StartFeedStatement.WAIT_FOR_COMPLETION, FeedActivityDetails.FEED_POLICY_NAME,
-                    FeedActivityDetails.COLLECT_LOCATIONS, "inline_with", "hash_merge", "output-record-type",
+                    FeedActivityDetails.COLLECT_LOCATIONS, SqlppQueryRewriter.INLINE_WITH_OPTION,
+                    SqlppExpressionToPlanTranslator.REWRITE_IN_AS_OR_OPTION, "hash_merge", "output-record-type",
                     AbstractIntroduceAccessMethodRule.NO_INDEX_ONLY_PLAN_OPTION);
 
     private final IRewriterFactory rewriterFactory;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1.sqlpp
new file mode 100644
index 0000000..ee3501a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1.sqlpp
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2.sqlpp
new file mode 100644
index 0000000..1387c1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3.sqlpp
new file mode 100644
index 0000000..15dec28
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3.sqlpp
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4.sqlpp
new file mode 100644
index 0000000..2b2c112
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5.sqlpp
new file mode 100644
index 0000000..34f33a4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6.sqlpp
new file mode 100644
index 0000000..b6478cb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6.sqlpp
@@ -0,0 +1,50 @@
+/*
+ * 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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572-2.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572-2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572-2.sqlpp
new file mode 100644
index 0000000..eaebd39
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572-2.sqlpp
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/* default: set rewrite_in_as_or "true" */
+
+drop dataverse sampdb if exists;
+create dataverse sampdb;
+use sampdb;
+
+drop dataset samptable if exists;
+drop type samptabletype if exists;
+
+create type samptabletype as closed {
+  id: bigint
+};
+
+create dataset samptable(samptabletype) primary key id;
+
+select *
+from
+(
+  select id from samptable
+  where (id in [0] and id in [1])
+        or (id in [1] and id in [2])
+) st1;
+
+drop dataverse sampdb;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp
index 76568bb..3850d35 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/query-ASTERIXDB-1572.sqlpp
@@ -17,6 +17,8 @@
  * under the License.
  */
 
+set rewrite_in_as_or "false";
+
 drop dataverse sampdb if exists;
 create dataverse sampdb;
 use sampdb;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
index 5ce7537..16c0765 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/filter/inverted-btree-search-return-optional-field.plan
@@ -3,41 +3,27 @@
     -- STREAM_LIMIT  |UNPARTITIONED|
       -- STREAM_PROJECT  |PARTITIONED|
         -- ASSIGN  |PARTITIONED|
-          -- SORT_MERGE_EXCHANGE [$$38(DESC) ]  |PARTITIONED|
+          -- SORT_MERGE_EXCHANGE [$$37(DESC) ]  |PARTITIONED|
             -- STREAM_LIMIT  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                -- STABLE_SORT [topK: 2147483647] [$$38(DESC)]  |PARTITIONED|
+                -- STABLE_SORT [topK: 2147483647] [$$37(DESC)]  |PARTITIONED|
                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                     -- STREAM_PROJECT  |PARTITIONED|
-                      -- STREAM_SELECT  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- PRE_CLUSTERED_GROUP_BY[$$39]  |PARTITIONED|
-                                  {
-                                    -- AGGREGATE  |LOCAL|
-                                      -- STREAM_SELECT  |LOCAL|
-                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
-                                  }
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
-                                -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
-                                  -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                      -- HYBRID_HASH_JOIN [$$40][$#1]  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
-                                          -- STREAM_PROJECT  |PARTITIONED|
-                                            -- ASSIGN  |PARTITIONED|
-                                              -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- HYBRID_HASH_JOIN [$$24][$$39]  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                 -- ASSIGN  |PARTITIONED|
-                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                    -- BTREE_SEARCH  |PARTITIONED|
-                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- STABLE_SORT [$$53(ASC)]  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- SINGLE_PARTITION_INVERTED_INDEX_SEARCH  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- ASSIGN  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
-                                          -- ASSIGN  |UNPARTITIONED|
-                                            -- UNNEST  |UNPARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
new file mode 100644
index 0000000..62c2b89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
new file mode 100644
index 0000000..b36c3ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                          -- ASSIGN  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
new file mode 100644
index 0000000..62c2b89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
new file mode 100644
index 0000000..35d91ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$22(ASC) ]  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$18]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$19][$#1]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                          -- ASSIGN  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
new file mode 100644
index 0000000..62c2b89
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$18(ASC) ]  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ASSIGN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- DATASOURCE_SCAN  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- UNNEST  |UNPARTITIONED|
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
new file mode 100644
index 0000000..b36c3ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- SORT_MERGE_EXCHANGE [$$23(ASC) ]  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- STREAM_SELECT  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                  {
+                    -- AGGREGATE  |LOCAL|
+                      -- STREAM_SELECT  |LOCAL|
+                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                  }
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                        -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                          -- ASSIGN  |UNPARTITIONED|
+                            -- UNNEST  |UNPARTITIONED|
+                              -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
new file mode 100644
index 0000000..10b5853
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/query-ASTERIXDB-1572-2.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.1.ddl.sqlpp
new file mode 100644
index 0000000..be2bcbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.1.ddl.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.2.update.sqlpp
new file mode 100644
index 0000000..c88c5bc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.2.update.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 tpch;
+
+load  dataset Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.3.query.sqlpp
new file mode 100644
index 0000000..a63b792
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.3.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use tpch;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.4.query.sqlpp
new file mode 100644
index 0000000..5a77f27
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.4.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 tpch;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.5.query.sqlpp
new file mode 100644
index 0000000..a432d03
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.5.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+use tpch;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.6.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.6.query.sqlpp
new file mode 100644
index 0000000..b22ce59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.6.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+use tpch;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.7.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.7.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.7.query.sqlpp
new file mode 100644
index 0000000..774debe
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.7.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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+use tpch;
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.8.query.sqlpp
new file mode 100644
index 0000000..16e36ce
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/in_as_or/in_as_or.8.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+use tpch;
+
+set rewrite_in_as_or "false";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp
index a4b144f..a2fd910 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.3.query.sqlpp
@@ -19,6 +19,8 @@
 
 use sampdb;
 
+set `rewrite_in_as_or` "false";
+
 select *
 from
 (

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.4.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.4.query.sqlpp
new file mode 100644
index 0000000..5935946
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.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.
+ */
+
+use sampdb;
+
+/* default: set `rewrite_in_as_or` "true" */
+
+select *
+from
+(
+  select id from samptable
+  where (id in [0] and id in [1])
+        or (id in [1] and id in [2])
+) st1;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.3.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.3.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.4.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.4.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.4.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.4.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.5.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.5.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.5.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.6.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.6.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.6.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.6.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.7.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.7.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.7.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.7.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.8.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.8.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.8.adm
new file mode 100644
index 0000000..5f5fbe7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/in_as_or/in_as_or.8.adm
@@ -0,0 +1,3 @@
+1
+2
+3
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.2.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/subquery/query-ASTERIXDB-1572/query-ASTERIXDB-1572.2.adm
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
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 81b53bd..fc777d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -6829,6 +6829,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="subquery">
+      <compilation-unit name="in_as_or">
+        <output-dir compare="Text">in_as_or</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="subquery">
       <compilation-unit name="in_correlated">
         <output-dir compare="Text">in</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
index ecd8c8e..41ff3e0 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
+++ b/asterixdb/asterix-lang-sqlpp/src/main/java/org/apache/asterix/lang/sqlpp/rewrites/SqlppQueryRewriter.java
@@ -70,9 +70,9 @@ import org.apache.asterix.lang.sqlpp.util.FunctionMapUtil;
 import org.apache.asterix.lang.sqlpp.visitor.base.ISqlppVisitor;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 
-class SqlppQueryRewriter implements IQueryRewriter {
-    private static final String INLINE_WITH = "inline_with";
-    private static final String NOT_INLINE_WITH = "false";
+public class SqlppQueryRewriter implements IQueryRewriter {
+    public static final String INLINE_WITH_OPTION = "inline_with";
+    private static final boolean INLINE_WITH_OPTION_DEFAULT = true;
     private final FunctionParser functionRepository = new FunctionParser(new SqlppParserFactory());
     private IReturningStatement topExpr;
     private List<FunctionDecl> declaredFunctions;
@@ -168,8 +168,7 @@ class SqlppQueryRewriter implements IQueryRewriter {
     }
 
     protected void inlineWithExpressions() throws CompilationException {
-        String inlineWith = (String) metadataProvider.getConfig().get(INLINE_WITH);
-        if (inlineWith != null && inlineWith.equalsIgnoreCase(NOT_INLINE_WITH)) {
+        if (!metadataProvider.getBooleanProperty(INLINE_WITH_OPTION, INLINE_WITH_OPTION_DEFAULT)) {
             return;
         }
         // Inlines with expressions.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/b674b1b5/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index b5bcece..e212d11 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -188,6 +188,11 @@ public class MetadataProvider implements IMetadataProvider<DataSourceId, String>
         return (T) config.remove(name);
     }
 
+    public boolean getBooleanProperty(String name, boolean defaultValue) {
+        Object v = config.get(name);
+        return v != null ? Boolean.parseBoolean(String.valueOf(v)) : defaultValue;
+    }
+
     public void disableBlockingOperator() {
         blockingOperatorDisabled = true;
     }