You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by gg...@apache.org on 2021/08/03 18:59:28 UTC

[asterixdb] branch master updated: [ASTERIXDB-2929][IDX] More robust array index applicability recognition

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

ggalvizo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/asterixdb.git


The following commit(s) were added to refs/heads/master by this push:
     new e627046  [ASTERIXDB-2929][IDX] More robust array index applicability recognition
e627046 is described below

commit e627046060773afc4ac7b80bf3725645e0f8de11
Author: ggalvizo <gg...@uci.edu>
AuthorDate: Fri Jul 30 08:15:29 2021 -1000

    [ASTERIXDB-2929][IDX] More robust array index applicability recognition
    
    - user mode changes: no
    - storage format changes: no
    - interface changes: no
    
    Issue pertains to fixing a bug with multiple quantifiers, but this
    change also adds support for more complex nesting structures with
    arrays. We now recognize array index applicability by matching
    subsequences of UNNEST and ASSIGN operators with the array index
    structure, in contrast to searching for a single uninterrupted sequence.
    
    Change-Id: I709b74ea72f63eeba12876a05dac508e6a03c4f4
    Reviewed-on: https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/12463
    Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
    Reviewed-by: Dmitry Lychagin <dm...@couchbase.com>
---
 .../IntroduceSecondaryIndexInsertDeleteRule.java   |   9 +-
 .../am/AbstractIntroduceAccessMethodRule.java      |   6 +-
 .../rules/am/AccessMethodAnalysisContext.java      |   4 +
 .../optimizer/rules/am/AccessMethodUtils.java      | 218 +++------------------
 .../rules/am/ArrayIndexStructureMatcher.java       | 213 ++++++++++++++++++++
 .../rules/util/SelectInSubplanBranchCreator.java   |  24 ++-
 .../multiple-quantifiers/query1.sqlpp              |  37 ++++
 .../multiple-quantifiers/query2.sqlpp              |  38 ++++
 .../multiple-quantifiers/query3.sqlpp              |  38 ++++
 .../multiple-quantifiers/query4.sqlpp              |  41 ++++
 .../multiple-quantifiers/query5.sqlpp              |  40 ++++
 .../multiple-quantifiers/query6.sqlpp              |  45 +++++
 .../open/complex-structures/query1.sqlpp           |  49 +++++
 .../open/complex-structures/query2.sqlpp           |  49 +++++
 .../open/complex-structures/query3.sqlpp           |  50 +++++
 .../open/complex-structures/query4.sqlpp           |  51 +++++
 .../open/complex-structures/query5.sqlpp           |  49 +++++
 .../open/complex-structures/query6.sqlpp           |  49 +++++
 .../open/complex-structures/query7.sqlpp           |  50 +++++
 .../open/complex-structures/query8.sqlpp           |  51 +++++
 .../multiple-quantifiers/query1.plan               |  31 +++
 .../multiple-quantifiers/query2.plan               |  31 +++
 .../multiple-quantifiers/query3.plan               |  33 ++++
 .../multiple-quantifiers/query4.plan               |  33 ++++
 .../multiple-quantifiers/query5.plan               |  44 +++++
 .../multiple-quantifiers/query6.plan               |  44 +++++
 .../open/complex-structures/query1.plan            |  22 +++
 .../open/complex-structures/query2.plan            |  22 +++
 .../open/complex-structures/query3.plan            |  25 +++
 .../open/complex-structures/query4.plan            |  25 +++
 .../open/complex-structures/query5.plan            |  22 +++
 .../open/complex-structures/query6.plan            |  22 +++
 .../open/complex-structures/query7.plan            |  25 +++
 .../open/complex-structures/query8.plan            |  25 +++
 .../asterix/metadata/utils/ArrayIndexUtil.java     |  82 ++------
 35 files changed, 1334 insertions(+), 263 deletions(-)

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 a6ae0af..f8a6cd8 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
@@ -721,17 +721,12 @@ public class IntroduceSecondaryIndexInsertDeleteRule implements IAlgebraicRewrit
                 secondaryKeyVars.add(newVar);
 
             } else {
-                // We have an array element. The "open / nestedness" is determined by the first UNNEST field.
-                isOpenOrNestedField = workingElement.getUnnestList().get(0).size() > 1
-                        || !recordType.isClosedField(workingElement.getUnnestList().get(0).get(0));
-
-                // Walk the array path.
+                // We have an array element.  Walk the array path.
                 List<String> flatFirstFieldName = ArrayIndexUtil.getFlattenedKeyFieldNames(
                         workingElement.getUnnestList(), workingElement.getProjectList().get(0));
                 List<Boolean> firstUnnestFlags = ArrayIndexUtil.getUnnestFlags(workingElement.getUnnestList(),
                         workingElement.getProjectList().get(0));
-                ArrayIndexUtil.walkArrayPath((isOpenOrNestedField) ? null : recordType, flatFirstFieldName,
-                        firstUnnestFlags, branchCreator);
+                ArrayIndexUtil.walkArrayPath(recordType, flatFirstFieldName, firstUnnestFlags, branchCreator);
 
                 // For all other elements in the PROJECT list, add an assign.
                 for (int j = 1; j < workingElement.getProjectList().size(); j++) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
index d08cf85..8458fde 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AbstractIntroduceAccessMethodRule.java
@@ -81,7 +81,6 @@ import com.google.common.base.Strings;
  * methods.
  */
 public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRewriteRule {
-
     protected MetadataProvider metadataProvider;
 
     public abstract Map<FunctionIdentifier, List<IAccessMethod>> getAccessMethods();
@@ -829,9 +828,8 @@ public abstract class AbstractIntroduceAccessMethodRule implements IAlgebraicRew
             if (optVarIndex == -1) {
                 if (doesArrayIndexQualify && subTree.getDataSourceType() == DataSourceType.DATASOURCE_SCAN) {
                     // We may be able to apply an array index to this variable.
-                    Triple<Integer, List<String>, IAType> fieldTriplet =
-                            AccessMethodUtils.analyzeVarForArrayIndexes(assignOp, optFuncExpr, subTree, datasetMetaVar,
-                                    context, datasetIndexes, analysisCtx.getMatchedFuncExprs(), varIndex);
+                    Triple<Integer, List<String>, IAType> fieldTriplet = AccessMethodUtils
+                            .analyzeVarForArrayIndexes(datasetIndexes, optFuncExpr, subTree, context, var, analysisCtx);
                     if (fieldTriplet != null && subTree.hasDataSource()) {
                         fillIndexExprs(datasetIndexes, fieldTriplet.second, fieldTriplet.third, optFuncExpr,
                                 optFuncExprIndex, fieldTriplet.first, subTree, analysisCtx, fieldSource.intValue());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
index 924c679..7fa6de1 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodAnalysisContext.java
@@ -36,6 +36,7 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCall
  * Context for analyzing the applicability of a single access method.
  */
 public class AccessMethodAnalysisContext {
+    private final ArrayIndexStructureMatcher arrayIndexStructureMatcher = new ArrayIndexStructureMatcher();
 
     private List<IOptimizableFuncExpr> matchedFuncExprs = new ArrayList<IOptimizableFuncExpr>();
 
@@ -176,4 +177,7 @@ public class AccessMethodAnalysisContext {
         this.indexDatasetMap = indexDatasetMap;
     }
 
+    public ArrayIndexStructureMatcher getArrayIndexStructureMatcher() {
+        return arrayIndexStructureMatcher;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
index 3f4d3f2..7528ce0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/AccessMethodUtils.java
@@ -28,7 +28,6 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Set;
-import java.util.Stack;
 
 import org.apache.asterix.algebra.operators.physical.ExternalDataLookupPOperator;
 import org.apache.asterix.common.annotations.AbstractExpressionAnnotationWithIndexNames;
@@ -2967,13 +2966,17 @@ public class AccessMethodUtils {
         return Collections.emptyList();
     }
 
-    /**
-     * Determine whether an array index can be used for the given variable.
-     */
-    public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(AssignOperator assignOp,
-            IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree, LogicalVariable datasetMetaVar,
-            IOptimizationContext context, List<Index> datasetIndexes, List<IOptimizableFuncExpr> matchedFuncExprs,
-            int assignVarIndex) throws AlgebricksException {
+    public static Triple<Integer, List<String>, IAType> analyzeVarForArrayIndexes(List<Index> datasetIndexes,
+            IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree, IOptimizationContext context,
+            LogicalVariable assignVar, AccessMethodAnalysisContext analysisCtx) throws AlgebricksException {
+        // Set the logical expression we are working with.
+        final int lastMatchedDataSourceVar = subTree.getLastMatchedDataSourceVars().second;
+        if (lastMatchedDataSourceVar < 0) {
+            return null;
+        }
+        final ILogicalExpression optVarExpr =
+                optFuncExpr.getFuncExpr().getArguments().get(lastMatchedDataSourceVar).getValue();
+        optFuncExpr.setLogicalExpr(lastMatchedDataSourceVar, optVarExpr);
 
         for (Index index : datasetIndexes) {
             if (index.getIndexType() != IndexType.ARRAY) {
@@ -2988,198 +2991,39 @@ public class AccessMethodUtils {
 
                 // We have found the array field for an array index.
                 for (List<String> project : e.getProjectList()) {
-                    List<String> flattenedFieldName =
-                            ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project);
-                    List<Integer> arrayIndicator = ArrayIndexUtil.getArrayDepthIndicator(e.getUnnestList(), project);
-
-                    Triple<Integer, List<String>, IAType> fieldTriplet =
-                            matchAssignFieldInUnnestAssignStack(assignOp.getVariables().get(assignVarIndex),
-                                    assignVarIndex, optFuncExpr, subTree, datasetMetaVar, context, arrayIndicator,
-                                    flattenedFieldName, arrayIndexDetails.isOverridingKeyFieldTypes());
-
-                    // This specific field aligns with our array index.
-                    if (fieldTriplet.first > -1) {
-                        int optVarIndex = fieldTriplet.first;
-                        List<String> fieldName = fieldTriplet.second;
-                        IAType fieldType = fieldTriplet.third;
+                    List<String> flatName = ArrayIndexUtil.getFlattenedKeyFieldNames(e.getUnnestList(), project);
+                    List<Boolean> unnestFlags = ArrayIndexUtil.getUnnestFlags(e.getUnnestList(), project);
+                    analysisCtx.getArrayIndexStructureMatcher().reset(assignVar, subTree);
+                    ArrayIndexUtil.walkArrayPath(subTree.getRecordType(), flatName, unnestFlags,
+                            analysisCtx.getArrayIndexStructureMatcher());
+
+                    LogicalVariable varAfterWalk = analysisCtx.getArrayIndexStructureMatcher().getEndVar();
+                    ILogicalOperator opAfterWalk = analysisCtx.getArrayIndexStructureMatcher().getEndOperator();
+                    if (varAfterWalk != null && opAfterWalk != null) {
+                        // This specific field aligns with an array index. Verify that this variable actually exists
+                        // in our function expression.
+                        int optVarIndex = optFuncExpr.findLogicalVar(varAfterWalk);
+                        if (optVarIndex == -1) {
+                            continue;
+                        }
+                        IAType fieldType =
+                                (IAType) context.getOutputTypeEnvironment(opAfterWalk).getVarType(varAfterWalk);
+                        optFuncExpr.setSourceVar(optVarIndex, varAfterWalk);
 
                         // Remember matching subtree.
                         optFuncExpr.setOptimizableSubTree(optVarIndex, subTree);
                         MutableInt fieldSource = new MutableInt(0);
-                        optFuncExpr.setFieldName(optVarIndex, fieldName, fieldSource.intValue());
+                        optFuncExpr.setFieldName(optVarIndex, flatName, fieldSource.intValue());
                         optFuncExpr.setFieldType(optVarIndex, fieldType);
                         IAType type = (IAType) context.getOutputTypeEnvironment(subTree.getRoot())
                                 .getVarType(optFuncExpr.getLogicalVar(optVarIndex));
                         optFuncExpr.setFieldType(optVarIndex, type);
 
-                        return fieldTriplet;
+                        return new Triple<>(optVarIndex, flatName, fieldType);
                     }
                 }
             }
         }
-
         return null;
     }
-
-    /**
-     * @param assignVar Variable from lowest assign that we are trying to match (i.e. the first array step var).
-     * @param assignVarIndex Index of the variable from the lowest assign.
-     * @param optFuncExpr The function expression we are trying to optimize.
-     * @param subTree Subtree for the function expression {@code optFunExpr}.
-     * @param datasetMetaVar Meta-variable from our subtree, if any exist.
-     * @param context Context required to get the type of the found variable.
-     * @param indexArrayIndicators Depth indicators of index to match our unnest/assign stack to.
-     * @param indexFieldNames Field names of index to match our unnest/assign stack to.
-     * @param areFieldNamesInAssign True if we have an open index. False otherwise.
-     */
-    private static Triple<Integer, List<String>, IAType> matchAssignFieldInUnnestAssignStack(LogicalVariable assignVar,
-            int assignVarIndex, IOptimizableFuncExpr optFuncExpr, OptimizableOperatorSubTree subTree,
-            LogicalVariable datasetMetaVar, IOptimizationContext context, List<Integer> indexArrayIndicators,
-            List<String> indexFieldNames, boolean areFieldNamesInAssign) throws AlgebricksException {
-        Triple<Integer, List<String>, IAType> resultantTriple = new Triple<>(-1, new ArrayList<>(), null);
-        final int optVarIndex = subTree.getLastMatchedDataSourceVars().second;
-        if (optVarIndex < 0) {
-            return resultantTriple;
-        }
-        final ILogicalExpression optVarExpr = optFuncExpr.getFuncExpr().getArguments().get(optVarIndex).getValue();
-        optFuncExpr.setLogicalExpr(optVarIndex, optVarExpr);
-
-        // Build our assign / unnest stack. Do not include the very last assign (this is handled in the parent).
-        int indexOfWorkingOp = subTree.getAssignsAndUnnests().size() - 1;
-        Stack<AbstractLogicalOperator> logicalOperatorStack = new Stack<>();
-        logicalOperatorStack.addAll(subTree.getAssignsAndUnnests().subList(0, indexOfWorkingOp));
-        if (logicalOperatorStack.empty()) {
-            return resultantTriple;
-        }
-
-        // Aggregate our record paths, and pair these with their respective array indexes.
-        Pair<List<List<String>>, List<Integer>> unnestPairs =
-                ArrayIndexUtil.unnestComplexRecordPath(indexFieldNames, indexArrayIndicators);
-        AbstractLogicalOperator workingOp = null;
-        List<String> fieldNameForWorkingUnnest;
-        MutableInt fieldSource = new MutableInt(0);
-        ARecordType workingRecordType = subTree.getRecordType();
-
-        // Iterate through our array index structure. We must match the depth and field names for the caller's variable
-        // to qualify for an array-index optimization.
-        LogicalVariable varFromParent = assignVar;
-        for (int pairsIndex = 0; pairsIndex < unnestPairs.first.size(); pairsIndex++) {
-            if (logicalOperatorStack.empty()) {
-                return resultantTriple;
-            }
-            workingOp = logicalOperatorStack.pop();
-
-            // Explore our UNNEST path.
-            if (unnestPairs.second.get(pairsIndex) > 0) {
-                for (int i = (pairsIndex == 0) ? 1 : 0; i < unnestPairs.first.get(pairsIndex).size(); i++) {
-                    // Match our parent assign variable to a variable used in our working assign.
-                    assignVarIndex = findAssignVarIndex(workingOp, varFromParent);
-                    if (logicalOperatorStack.empty() || assignVarIndex == -1) {
-                        return resultantTriple;
-                    }
-                    varFromParent = ((AssignOperator) workingOp).getVariables().get(assignVarIndex);
-                    indexOfWorkingOp--;
-                    workingOp = logicalOperatorStack.pop();
-                }
-
-                // Get the field name associated with the current UNNEST.
-                if (workingOp.getOperatorTag() != LogicalOperatorTag.UNNEST) {
-                    return resultantTriple;
-                }
-                fieldNameForWorkingUnnest = getFieldNameFromSubTree(null, subTree, indexOfWorkingOp, assignVarIndex,
-                        workingRecordType, 0, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource, true);
-
-                if (!fieldNameForWorkingUnnest.equals(unnestPairs.first.get(pairsIndex))) {
-                    return resultantTriple;
-                }
-                resultantTriple.second.addAll(fieldNameForWorkingUnnest);
-
-                IAType typeIntermediate = workingRecordType.getSubFieldType(fieldNameForWorkingUnnest);
-                for (int i = 0; i < unnestPairs.second.get(pairsIndex); i++) {
-                    // If we are working with a closed index, then update our record type. For open types, we do not
-                    // need to do this as the field name is stored in the expression itself.
-                    if (!areFieldNamesInAssign && pairsIndex != unnestPairs.first.size() - 1) {
-                        typeIntermediate = TypeComputeUtils.extractListItemType(typeIntermediate);
-                        if (typeIntermediate == null) {
-                            return resultantTriple;
-                        }
-                    }
-                    boolean isIntermediateUnnestInPath = (i != unnestPairs.second.get(pairsIndex) - 1);
-                    if (!areFieldNamesInAssign && !isIntermediateUnnestInPath) {
-                        if (typeIntermediate.getTypeTag().equals(ATypeTag.OBJECT)) {
-                            workingRecordType = (ARecordType) typeIntermediate;
-                        } else if (!typeIntermediate.getTypeTag().isListType()) {
-                            return resultantTriple;
-                        }
-                    }
-
-                    // Update our parent variable. If we are in-between UNNESTs, we need to fetch the next UNNEST.
-                    if (isIntermediateUnnestInPath) {
-                        workingOp = logicalOperatorStack.pop();
-                        indexOfWorkingOp--;
-                    }
-                    varFromParent = ((UnnestOperator) workingOp).getVariable();
-                }
-            } else if (pairsIndex != 0) {
-                // We have explored an UNNEST array-path previously, and must now match a field name.
-                AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
-                indexOfWorkingOp -= unnestPairs.first.get(pairsIndex).size();
-                for (assignVarIndex = 0; assignVarIndex < workingOpAsAssign.getVariables().size(); assignVarIndex++) {
-                    // Iterate through each of our ASSIGN's field names, and try to match the index field names.
-                    fieldNameForWorkingUnnest = getFieldNameFromSubTree(null, subTree, indexOfWorkingOp, assignVarIndex,
-                            workingRecordType, 0, null, subTree.getMetaRecordType(), datasetMetaVar, fieldSource, true);
-
-                    if (fieldNameForWorkingUnnest.equals(unnestPairs.first.get(pairsIndex))) {
-                        resultantTriple.second.addAll(fieldNameForWorkingUnnest);
-                        break;
-                    }
-                }
-
-                // We have exhausted all of our ASSIGN variables, but have not matched the field name. Exit early.
-                if (assignVarIndex == workingOpAsAssign.getVariables().size()) {
-                    return resultantTriple;
-                }
-            }
-
-            indexOfWorkingOp--;
-        }
-
-        // We have found an applicable array index. Determine our optFuncIndex and fieldType.
-        if (workingOp != null && workingOp.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
-            AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
-            LogicalVariable matchedVar = workingOpAsAssign.getVariables().get(assignVarIndex);
-            if (optFuncExpr.findLogicalVar(matchedVar) > -1) {
-                resultantTriple.first = optFuncExpr.findLogicalVar(matchedVar);
-                resultantTriple.third = (IAType) context.getOutputTypeEnvironment(workingOp).getVarType(matchedVar);
-                optFuncExpr.setSourceVar(resultantTriple.first, matchedVar);
-            }
-
-        } else if (workingOp != null) {
-            UnnestOperator workingOpAsUnnest = (UnnestOperator) workingOp;
-            resultantTriple.first = optFuncExpr.findLogicalVar(workingOpAsUnnest.getVariable());
-            resultantTriple.third =
-                    (IAType) context.getOutputTypeEnvironment(workingOp).getVarType(workingOpAsUnnest.getVariable());
-            optFuncExpr.setSourceVar(resultantTriple.first, workingOpAsUnnest.getVariable());
-        }
-
-        return resultantTriple;
-    }
-
-    private static int findAssignVarIndex(AbstractLogicalOperator workingOp, LogicalVariable varFromParentAssign) {
-        if (workingOp.getOperatorTag() != LogicalOperatorTag.ASSIGN) {
-            return -1;
-        }
-        AssignOperator workingOpAsAssign = (AssignOperator) workingOp;
-
-        // Match our parent assign variable to a variable used in our working assign.
-        List<LogicalVariable> variablesUsedInWorkingAssign = new ArrayList<>();
-        for (Mutable<ILogicalExpression> assignExpr : workingOpAsAssign.getExpressions()) {
-            assignExpr.getValue().getUsedVariables(variablesUsedInWorkingAssign);
-            int pos = variablesUsedInWorkingAssign.indexOf(varFromParentAssign);
-            if (pos != -1) {
-                return pos;
-            }
-        }
-        return -1;
-    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayIndexStructureMatcher.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayIndexStructureMatcher.java
new file mode 100644
index 0000000..2e7f814
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/am/ArrayIndexStructureMatcher.java
@@ -0,0 +1,213 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.optimizer.rules.am;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.asterix.metadata.utils.ArrayIndexUtil;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AString;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.constants.AsterixConstantValue;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
+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.ConstantExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
+
+public class ArrayIndexStructureMatcher implements ArrayIndexUtil.TypeTrackerCommandExecutor {
+    private final List<AbstractLogicalOperator> logicalOperators = new ArrayList<>();
+    private int lastMatchedPosition = -1;
+    private boolean isStructureMatched = true;
+
+    private LogicalVariable varFromParent;
+    private IAType typeFromParent;
+
+    @Override
+    public void executeActionOnEachArrayStep(ARecordType startingStepRecordType, IAType workingType,
+            List<String> fieldName, boolean isFirstArrayStep, boolean isLastUnnestInIntermediateStep) {
+        // Avoiding exploring, we could not match an earlier array step.
+        if (!isStructureMatched) {
+            return;
+        }
+
+        // Match our field name and a set of ASSIGNs.
+        boolean isStructureMatchedForThisStep = matchAssignVarAndFieldName(startingStepRecordType, fieldName);
+
+        // Match an UNNEST operator with the tail of the previously matched ASSIGN.
+        if (isStructureMatchedForThisStep) {
+            isStructureMatchedForThisStep = matchUnnestVar();
+        }
+
+        // Update our flags.
+        isStructureMatched = isStructureMatched && isStructureMatchedForThisStep;
+    }
+
+    @Override
+    public void executeActionOnFinalArrayStep(ARecordType startingStepRecordType, List<String> fieldName,
+            boolean isNonArrayStep, boolean requiresOnlyOneUnnest) {
+        if (isNonArrayStep) {
+            isStructureMatched = isStructureMatched && matchAssignVarAndFieldName(startingStepRecordType, fieldName);
+        }
+
+        if (!isStructureMatched) {
+            // If no match is found, signal this to our caller.
+            varFromParent = null;
+            lastMatchedPosition = -1;
+        }
+    }
+
+    public void reset(LogicalVariable assignVar, OptimizableOperatorSubTree subTree) {
+        varFromParent = assignVar;
+        typeFromParent = null;
+
+        // We start by assuming that the structure is matched, and try to find steps where this does not hold.
+        isStructureMatched = true;
+
+        // Build our list from the ASSIGNs and UNNESTs collected in our subtree.
+        lastMatchedPosition = -1;
+        logicalOperators.clear();
+        logicalOperators.addAll(subTree.getAssignsAndUnnests());
+        Collections.reverse(logicalOperators);
+    }
+
+    public LogicalVariable getEndVar() {
+        return varFromParent;
+    }
+
+    public ILogicalOperator getEndOperator() {
+        return (lastMatchedPosition == -1) ? null : logicalOperators.get(lastMatchedPosition);
+    }
+
+    private boolean matchUnnestVar() {
+        boolean isStructureMatchedFoundForThisStep = false;
+        AbstractLogicalOperator workingOp;
+        int searchPosition = lastMatchedPosition + 1;
+
+        // Match the UNNEST variable. Ignore any ASSIGNs we run into here.
+        while (!isStructureMatchedFoundForThisStep && searchPosition < logicalOperators.size()) {
+            workingOp = logicalOperators.get(searchPosition);
+
+            if (workingOp.getOperatorTag().equals(LogicalOperatorTag.UNNEST)) {
+                UnnestOperator workingOpAsUnnest = (UnnestOperator) workingOp;
+                List<LogicalVariable> expressionUsedVariables = new ArrayList<>();
+                ILogicalExpression unnestExpr = workingOpAsUnnest.getExpressionRef().getValue();
+                unnestExpr.getUsedVariables(expressionUsedVariables);
+
+                if (expressionUsedVariables.contains(varFromParent)) {
+                    varFromParent = workingOpAsUnnest.getVariable();
+                    lastMatchedPosition = searchPosition;
+                    isStructureMatchedFoundForThisStep = true;
+
+                } else {
+                    isStructureMatchedFoundForThisStep = false;
+                }
+            }
+            searchPosition++;
+        }
+        return isStructureMatchedFoundForThisStep;
+    }
+
+    private boolean matchAssignVarAndFieldName(ARecordType recordType, List<String> fieldName) {
+        boolean isStructureMatchedForThisStep = false;
+        AbstractLogicalOperator workingOp;
+        int searchPosition;
+
+        typeFromParent = recordType;
+        for (String fieldPart : fieldName) {
+            searchPosition = lastMatchedPosition + 1;
+            isStructureMatchedForThisStep = false;
+
+            // Match the ASSIGN variable + field name. Ignore the UNNESTs we find here.
+            while (!isStructureMatchedForThisStep && searchPosition < logicalOperators.size()) {
+                workingOp = logicalOperators.get(searchPosition);
+                if (workingOp.getOperatorTag().equals(LogicalOperatorTag.ASSIGN)) {
+
+                    // Keep track of our type between matching field part calls.
+                    ARecordType typeForFieldPart =
+                            (typeFromParent instanceof ARecordType) ? (ARecordType) typeFromParent : recordType;
+
+                    // Match the specific field part.
+                    if (matchAssignVarAndFieldPart((AssignOperator) workingOp, typeForFieldPart, fieldPart)) {
+                        isStructureMatchedForThisStep = true;
+                        lastMatchedPosition = searchPosition;
+                    }
+                }
+                searchPosition++;
+            }
+        }
+        return isStructureMatchedForThisStep;
+    }
+
+    private boolean matchAssignVarAndFieldPart(AssignOperator workingOp, ARecordType recordType, String fieldPart) {
+        final List<LogicalVariable> expressionUsedVariables = new ArrayList<>();
+        for (int j = 0; j < workingOp.getExpressions().size(); j++) {
+            ILogicalExpression assignExpr = workingOp.getExpressions().get(j).getValue();
+            assignExpr.getUsedVariables(expressionUsedVariables);
+
+            boolean isVarInExpression = expressionUsedVariables.contains(varFromParent);
+            boolean isVarInOutput = workingOp.getVariables().get(j).equals(varFromParent);
+            if (isVarInExpression || isVarInOutput) {
+                // We have found a potential match. We must now map the field names from the assign to what is actually
+                // specified in the index.
+
+                if (assignExpr.getExpressionTag().equals(LogicalExpressionTag.FUNCTION_CALL)) {
+                    ScalarFunctionCallExpression assignFunc = (ScalarFunctionCallExpression) assignExpr;
+                    String fieldNameFromAssign = null;
+
+                    if (assignFunc.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_NAME)) {
+                        // Search for the field name inside the expression itself.
+                        ConstantExpression assignNameExpr =
+                                (ConstantExpression) assignFunc.getArguments().get(1).getValue();
+                        IAObject assignNameObj = ((AsterixConstantValue) assignNameExpr.getValue()).getObject();
+                        fieldNameFromAssign = ((AString) assignNameObj).getStringValue();
+
+                    } else if (assignFunc.getFunctionIdentifier().equals(BuiltinFunctions.FIELD_ACCESS_BY_INDEX)) {
+                        // Search for the field name using the type we are tracking.
+                        ConstantExpression assignIndexExpr =
+                                (ConstantExpression) assignFunc.getArguments().get(1).getValue();
+                        IAObject assignIndexObj = ((AsterixConstantValue) assignIndexExpr.getValue()).getObject();
+                        int assignIndex = ((AInt32) assignIndexObj).getIntegerValue();
+                        fieldNameFromAssign = recordType.getFieldNames()[assignIndex];
+                        typeFromParent = recordType.getFieldTypes()[assignIndex];
+                    }
+
+                    if (fieldNameFromAssign != null && fieldNameFromAssign.equals(fieldPart)) {
+                        // We have found a match.
+                        varFromParent = workingOp.getVariables().get(j);
+                        return true;
+                    }
+                }
+            }
+        }
+
+        return false;
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
index c03870f..ffe1fd4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/util/SelectInSubplanBranchCreator.java
@@ -208,8 +208,9 @@ public class SelectInSubplanBranchCreator {
         newSelectOperator.setExecutionMode(optimizableSelect.getExecutionMode());
 
         // Follow this SELECT to the root of our nested-plan branch (i.e. the NESTED-TUPLE-SOURCE).
-        ILogicalOperator workingOriginalOperator = optimizableSelect, workingNewOperator = newSelectOperator;
+        ILogicalOperator workingNewOperator = newSelectOperator;
         UnnestOperator bottommostNewUnnest = null;
+        ILogicalOperator workingOriginalOperator = optimizableSelect.getInputs().get(0).getValue();
         while (!workingOriginalOperator.getOperatorTag().equals(LogicalOperatorTag.NESTEDTUPLESOURCE)) {
             if (workingOriginalOperator.getInputs().isEmpty()) {
                 throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE,
@@ -237,8 +238,27 @@ public class SelectInSubplanBranchCreator {
                     workingNewOperator = newAssign;
                     break;
 
-                case AGGREGATE:
                 case SELECT:
+                    // If we encounter another SELECT, then we have multiple quantifiers. Transform our new SELECT to
+                    // include this condition.
+                    List<Mutable<ILogicalExpression>> selectArguments = new ArrayList<>();
+                    if (!newSelectOperator.getCondition().getValue().splitIntoConjuncts(selectArguments)) {
+                        selectArguments.add(newSelectOperator.getCondition());
+                    }
+                    if (!((SelectOperator) workingOriginalOperator).getCondition().getValue()
+                            .splitIntoConjuncts(selectArguments)) {
+                        selectArguments.add(((SelectOperator) workingOriginalOperator).getCondition());
+                    }
+                    ScalarFunctionCallExpression andCond = new ScalarFunctionCallExpression(
+                            BuiltinFunctions.getBuiltinFunctionInfo(BuiltinFunctions.AND), selectArguments);
+                    SelectOperator updatedSelectOperator = new SelectOperator(new MutableObject<>(andCond),
+                            newSelectOperator.getRetainMissing(), newSelectOperator.getMissingPlaceholderVariable());
+                    updatedSelectOperator.setSourceLocation(sourceLocation);
+                    updatedSelectOperator.getInputs().addAll(newSelectOperator.getInputs());
+                    newSelectOperator = updatedSelectOperator;
+                    break;
+
+                case AGGREGATE:
                     break;
 
                 default:
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query1.sqlpp
new file mode 100644
index 0000000..3327bd6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query1.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.
+ */
+SET               `compiler.arrayindex` "true";
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+    _id: uuid
+};
+CREATE DATASET    Dataset1 (TestType)
+PRIMARY KEY        _id AUTOGENERATED;
+CREATE INDEX      d1Idx
+ON                Dataset1 ( UNNEST items : bigint );
+
+-- Single UNNEST, open index.
+FROM       Dataset1 D1
+WHERE      SOME I IN D1.items, J IN D1.other_items
+           SATISFIES I = 1 AND J = 2
+SELECT     *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query2.sqlpp
new file mode 100644
index 0000000..bf46296
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query2.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.
+ */
+SET               `compiler.arrayindex` "true";
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+    _id:   uuid,
+    items: [bigint]
+};
+CREATE DATASET    Dataset1 (TestType)
+PRIMARY KEY        _id AUTOGENERATED;
+CREATE INDEX      d1Idx
+ON                Dataset1 ( UNNEST items );
+
+-- Single UNNEST, closed index.
+FROM       Dataset1 D1
+WHERE      SOME I IN D1.items, J IN D1.other_items
+           SATISFIES I = 1 AND J = 2
+SELECT     *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query3.sqlpp
new file mode 100644
index 0000000..03b0d72
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query3.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.
+ */
+SET               `compiler.arrayindex` "true";
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+    _id: uuid
+};
+CREATE DATASET    Dataset1 (TestType)
+PRIMARY KEY        _id AUTOGENERATED;
+CREATE INDEX      d1Idx
+ON                Dataset1 ( UNNEST items
+                             SELECT item : bigint );
+
+-- Single UNNEST w/ SELECT, open index.
+FROM       Dataset1 D1
+WHERE      SOME I IN D1.items, J IN D1.other_items
+           SATISFIES I.item = 1 AND J.item = 2
+SELECT     *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query4.sqlpp
new file mode 100644
index 0000000..0bef658
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query4.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET               `compiler.arrayindex` "true";
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    items: [{
+        item: bigint
+    }]
+};
+CREATE DATASET    Dataset1 (TestType)
+PRIMARY KEY        _id AUTOGENERATED;
+CREATE INDEX      d1Idx
+ON                Dataset1 ( UNNEST items
+                             SELECT item );
+
+-- Single UNNEST w/ SELECT, closed index.
+FROM       Dataset1 D1
+WHERE      SOME I IN D1.items, J IN D1.other_items
+           SATISFIES I.item = 1 AND J.item = 2
+SELECT     *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query5.sqlpp
new file mode 100644
index 0000000..5f6fcb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query5.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET               `compiler.arrayindex` "true";
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+    _id: uuid
+};
+CREATE DATASET    Dataset1 (TestType)
+PRIMARY KEY        _id AUTOGENERATED;
+CREATE INDEX      d1Idx
+ON                Dataset1 ( UNNEST outer_items
+                             UNNEST inner_items
+                             SELECT item : bigint );
+
+-- Double UNNEST w/ SELECT, open index.
+FROM       Dataset1 D1
+WHERE      SOME I1 IN D1.outer_items, J IN D1.other_items
+           SATISFIES J.item = 2 AND ( SOME I2 IN I1.inner_items, J2 IN I1.other_inner_items
+                                      SATISFIES I2.item = 1 AND J2.item = 2 )
+SELECT     *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query6.sqlpp
new file mode 100644
index 0000000..7e529dc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-quantified-queries/multiple-quantifiers/query6.sqlpp
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET               `compiler.arrayindex` "true";
+
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    outer_items: [{
+        inner_items: [{
+            item: bigint
+        }]
+    }]
+};
+CREATE DATASET    Dataset1 (TestType)
+PRIMARY KEY        _id AUTOGENERATED;
+CREATE INDEX      d1Idx
+ON                Dataset1 ( UNNEST outer_items
+                             UNNEST inner_items
+                             SELECT item );
+
+-- Double UNNEST w/ SELECT, open index.
+FROM       Dataset1 D1
+WHERE      SOME I1 IN D1.outer_items, J IN D1.other_items
+           SATISFIES J.item = 2 AND ( SOME I2 IN I1.inner_items, J2 IN I1.other_inner_items
+                                      SATISFIES I2.item = 1 AND J2.item = 2 )
+SELECT     *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query1.sqlpp
new file mode 100644
index 0000000..3ecf642
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query1.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.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Fully open index 1. Unnest flags: [0, 1, 0]
+CREATE INDEX      testIndex1
+ON                TestDataset ( UNNEST open_field_1.open_field_2
+                                SELECT open_field_3a : bigint );
+
+FROM    TestDataset D
+UNNEST  D.open_field_1.open_field_2 F
+WHERE   F.open_field_3a > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query2.sqlpp
new file mode 100644
index 0000000..d2eef20
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query2.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.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Fully open index 2. Unnest flags: [0, 1, 0, 0]
+CREATE INDEX      testIndex2
+ON                TestDataset ( UNNEST open_field_1.open_field_2
+                                SELECT open_field_3b.open_field_4 : bigint );
+
+FROM    TestDataset D
+UNNEST  D.open_field_1.open_field_2 F
+WHERE   F.open_field_3b.open_field_4 > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query3.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query3.sqlpp
new file mode 100644
index 0000000..39e504c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query3.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.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Fully open index 3. Unnest flags: [0, 1, 0, 1]
+CREATE INDEX      testIndex3
+ON                TestDataset ( UNNEST open_field_1.open_field_2
+                                UNNEST open_field_3c.open_field_4a : bigint );
+
+FROM    TestDataset D
+UNNEST  D.open_field_1.open_field_2 F1
+UNNEST  F1.open_field_3c.open_field_4a F2
+WHERE   F2 > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query4.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query4.sqlpp
new file mode 100644
index 0000000..fea1d41
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query4.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Fully open index 4. Unnest flags: [0, 1, 0, 1, 0]
+CREATE INDEX      testIndex4
+ON                TestDataset ( UNNEST open_field_1.open_field_2
+                                UNNEST open_field_3c.open_field_4b
+                                SELECT open_field_5 : bigint );
+
+FROM    TestDataset D
+UNNEST  D.open_field_1.open_field_2 F1
+UNNEST  F1.open_field_3c.open_field_4b F2
+WHERE   F2.open_field_5 > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query5.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query5.sqlpp
new file mode 100644
index 0000000..10f7ab7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query5.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.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Partially open index 1.  [0, 1, 0]
+CREATE INDEX      testIndex1c
+ON                TestDataset ( UNNEST closed_field_1.open_field_2
+                                SELECT open_field_3a : bigint );
+
+FROM    TestDataset D
+UNNEST  D.closed_field_1.open_field_2 F
+WHERE   F.open_field_3a > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query6.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query6.sqlpp
new file mode 100644
index 0000000..637c458
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query6.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.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Partially open index 2. Unnest flags: [0, 1, 0, 0]
+CREATE INDEX      testIndex2c
+ON                TestDataset ( UNNEST closed_field_1.closed_field_2
+                                SELECT open_field_3b.open_field_4 : bigint );
+
+FROM    TestDataset D
+UNNEST  D.closed_field_1.closed_field_2 F
+WHERE   F.open_field_3b.open_field_4 > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query7.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query7.sqlpp
new file mode 100644
index 0000000..ba97b1b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query7.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.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Partially open index 3. Unnest flags: [0, 1, 0, 1]
+CREATE INDEX      testIndex3c
+ON                TestDataset ( UNNEST closed_field_1.closed_field_2
+                                UNNEST closed_field_3.open_field_4a : bigint );
+
+FROM    TestDataset D
+UNNEST  D.closed_field_1.closed_field_2 F1
+UNNEST  F1.closed_field_3.open_field_4a F2
+WHERE   F2 > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query8.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query8.sqlpp
new file mode 100644
index 0000000..f36ac3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/array-index/select-unnest-queries/open/complex-structures/query8.sqlpp
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+SET               `compiler.arrayindex` "true";
+DROP DATAVERSE    TestDataverse IF EXISTS;
+CREATE DATAVERSE  TestDataverse;
+USE               TestDataverse;
+CREATE TYPE       TestType AS {
+    _id: uuid,
+    closed_field_1: {
+        closed_field_2: [{
+            closed_field_3: {
+                closed_field_4: [{
+                    closed_field_5: bigint
+                }],
+                closed_field_4a: bigint,
+                closed_field_4b: [ bigint ]
+            },
+            closed_field_3a: bigint
+        }]
+    }
+};
+CREATE DATASET    TestDataset (TestType)
+PRIMARY KEY       _id AUTOGENERATED;
+
+-- Partially open index 4. Unnest flags: [0, 1, 0, 1, 0]
+CREATE INDEX      testIndex4c
+ON                TestDataset ( UNNEST closed_field_1.closed_field_2
+                                UNNEST closed_field_3.closed_field_4
+                                SELECT open_field_5 : bigint );
+
+FROM    TestDataset D
+UNNEST  D.closed_field_1.closed_field_2 F1
+UNNEST  F1.closed_field_3.closed_field_4 F2
+WHERE   F2.open_field_5 > 0
+SELECT  *;
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan
new file mode 100644
index 0000000..62b49d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query1.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan
new file mode 100644
index 0000000..62b49d7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query2.plan
@@ -0,0 +1,31 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- UNNEST  |LOCAL|
+                              -- STREAM_SELECT  |LOCAL|
+                                -- UNNEST  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan
new file mode 100644
index 0000000..24b6a5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query3.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- ASSIGN  |LOCAL|
+                                    -- UNNEST  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan
new file mode 100644
index 0000000..24b6a5a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query4.plan
@@ -0,0 +1,33 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- ASSIGN  |LOCAL|
+                              -- UNNEST  |LOCAL|
+                                -- STREAM_SELECT  |LOCAL|
+                                  -- ASSIGN  |LOCAL|
+                                    -- UNNEST  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$50(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan
new file mode 100644
index 0000000..7992dcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query5.plan
@@ -0,0 +1,44 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- SUBPLAN  |LOCAL|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- ASSIGN  |LOCAL|
+                                            -- UNNEST  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- ASSIGN  |LOCAL|
+                                                  -- UNNEST  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- STREAM_SELECT  |LOCAL|
+                                -- ASSIGN  |LOCAL|
+                                  -- UNNEST  |LOCAL|
+                                    -- ASSIGN  |LOCAL|
+                                      -- UNNEST  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan
new file mode 100644
index 0000000..7992dcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-quantified-queries/multiple-quantifiers/query6.plan
@@ -0,0 +1,44 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- SUBPLAN  |PARTITIONED|
+                      {
+                        -- AGGREGATE  |LOCAL|
+                          -- STREAM_SELECT  |LOCAL|
+                            -- SUBPLAN  |LOCAL|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- ASSIGN  |LOCAL|
+                                            -- UNNEST  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- ASSIGN  |LOCAL|
+                                                  -- UNNEST  |LOCAL|
+                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- STREAM_SELECT  |LOCAL|
+                                -- ASSIGN  |LOCAL|
+                                  -- UNNEST  |LOCAL|
+                                    -- ASSIGN  |LOCAL|
+                                      -- UNNEST  |LOCAL|
+                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                      }
+                -- ASSIGN  |PARTITIONED|
+                  -- STREAM_PROJECT  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- BTREE_SEARCH (TestDataverse.Dataset1.Dataset1)  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STABLE_SORT [$$81(ASC)]  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH (TestDataverse.Dataset1.d1Idx)  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan
new file mode 100644
index 0000000..0d24b2d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query1.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan
new file mode 100644
index 0000000..3585861
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query2.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan
new file mode 100644
index 0000000..679b2a0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query3.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan
new file mode 100644
index 0000000..3f26cee
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query4.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan
new file mode 100644
index 0000000..f1e0798
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query5.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$34(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex1c)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan
new file mode 100644
index 0000000..a31846b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query6.plan
@@ -0,0 +1,22 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex2c)  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan
new file mode 100644
index 0000000..4ee8831
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query7.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$46(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex3c)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan
new file mode 100644
index 0000000..c512a31
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/array-index/select-unnest-queries/open/complex-structures/query8.plan
@@ -0,0 +1,25 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_SELECT  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- UNNEST  |PARTITIONED|
+              -- ASSIGN  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- UNNEST  |PARTITIONED|
+                    -- ASSIGN  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH (TestDataverse.TestDataset.TestDataset)  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- PRE_SORTED_DISTINCT_BY  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$48(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH (TestDataverse.TestDataset.testIndex4c)  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
index 0778ad9..ab181cb 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/utils/ArrayIndexUtil.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.metadata.utils;
 
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.asterix.common.exceptions.AsterixException;
@@ -236,62 +237,6 @@ public class ArrayIndexUtil {
     }
 
     /**
-     * @deprecated Use new unnestList and projectList scheme.
-     */
-    public static List<Integer> getArrayDepthIndicator(List<List<String>> unnestList, List<String> projectList) {
-        if (unnestList == null) {
-            // A simple element has a flat set of depth indicators.
-            List<Integer> depthIndicator = new ArrayList<>();
-            for (String ignored : projectList) {
-                depthIndicator.add(0);
-            }
-            return depthIndicator;
-
-        } else {
-            List<Integer> depthIndicatorPrefix = new ArrayList<>();
-            for (List<String> unnestField : unnestList) {
-                for (int i = 0; i < unnestField.size() - 1; i++) {
-                    depthIndicatorPrefix.add(0);
-                }
-                depthIndicatorPrefix.add(1);
-            }
-
-            if (projectList == null) {
-                // Stop here. The prefix is the indicator itself.
-                return depthIndicatorPrefix;
-
-            } else {
-                List<Integer> depthIndicator = new ArrayList<>(depthIndicatorPrefix);
-                for (int i = 0; i < projectList.size(); i++) {
-                    depthIndicator.add(0);
-                }
-                return depthIndicator;
-            }
-        }
-    }
-
-    /**
-     * @deprecated Use new unnestList and projectList scheme.
-     * @return The record paths and non-zero depth indicators associated each record of fields from an array index.
-     */
-    public static Pair<List<List<String>>, List<Integer>> unnestComplexRecordPath(List<String> fieldName,
-            List<Integer> depthIndicators) {
-        List<List<String>> resultantPaths = new ArrayList<>();
-        List<Integer> resultantArrayIndicators = new ArrayList<>();
-        List<String> workingRecordPath = new ArrayList<>();
-        for (int i = 0; i < depthIndicators.size(); i++) {
-            workingRecordPath.add(fieldName.get(i));
-
-            if (i == depthIndicators.size() - 1 || depthIndicators.get(i) > 0) {
-                resultantArrayIndicators.add(depthIndicators.get(i));
-                resultantPaths.add(workingRecordPath);
-                workingRecordPath = new ArrayList<>();
-            }
-        }
-        return new Pair<>(resultantPaths, resultantArrayIndicators);
-    }
-
-    /**
      * Traverse each distinct record path and invoke the appropriate commands for each scenario. Here, we keep track
      * of the record/list type at each step and give this to each command.
      */
@@ -301,22 +246,33 @@ public class ArrayIndexUtil {
         List<List<String>> fieldNamesPerArray = arrayPath.fieldNamesPerArray;
         List<Boolean> unnestFlagsPerArray = arrayPath.unnestFlagsPerArray;
 
-        // If we are given no base record type, then we do not need to keep track of the record type. We are solely 
+        // If we are given no base record type, then we do not need to keep track of the record type. We are solely
         // using this walk for its flags.
         boolean isTrackingType = baseRecordType != null;
-
         IAType workingType = baseRecordType;
+
         for (int i = 0; i < fieldNamesPerArray.size(); i++) {
-            ARecordType startingStepRecordType = null;
+            ARecordType startingStepRecordType = (isTrackingType) ? (ARecordType) workingType : null;
             if (isTrackingType) {
                 if (!workingType.getTypeTag().equals(ATypeTag.OBJECT)) {
                     throw new AsterixException(ErrorCode.COMPILATION_ERROR, "Mismatched record type to depth-"
                             + "indicators. Expected record type, but got: " + workingType.getTypeTag());
                 }
-                startingStepRecordType = (ARecordType) workingType;
-                workingType = Index.getNonNullableOpenFieldType(
-                        startingStepRecordType.getSubFieldType(fieldNamesPerArray.get(i)), fieldNamesPerArray.get(i),
-                        startingStepRecordType).first;
+
+                ARecordType intermediateRecordType = startingStepRecordType;
+                List<String> fieldName = fieldNamesPerArray.get(i);
+                for (String fieldPart : fieldName) {
+                    // Determine whether we have an open field or not. Extract the type appropriately.
+                    isTrackingType = isTrackingType && intermediateRecordType.doesFieldExist(fieldPart);
+                    if (isTrackingType) {
+                        workingType = Index.getNonNullableOpenFieldType(intermediateRecordType.getFieldType(fieldPart),
+                                Collections.singletonList(fieldPart), intermediateRecordType).first;
+                        if (workingType instanceof ARecordType) {
+                            // We have an intermediate step, set our record step for the next loop iteration.
+                            intermediateRecordType = (ARecordType) workingType;
+                        }
+                    }
+                }
             }
 
             if (unnestFlagsPerArray.get(i)) {